refactoring

This commit is contained in:
Miriam Baglioni 2020-04-23 12:40:44 +02:00
parent d8dc31d4af
commit 769aa8178a
2 changed files with 163 additions and 122 deletions

View File

@ -1,37 +1,36 @@
package eu.dnetlib.dhp.projecttoresult; package eu.dnetlib.dhp.projecttoresult;
import static eu.dnetlib.dhp.PropagationConstant.*;
import static eu.dnetlib.dhp.PropagationConstant.getConstraintList;
import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkHiveSession;
import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.gson.Gson; import com.google.gson.Gson;
import eu.dnetlib.dhp.application.ArgumentApplicationParser; import eu.dnetlib.dhp.application.ArgumentApplicationParser;
import eu.dnetlib.dhp.countrypropagation.PrepareDatasourceCountryAssociation; import eu.dnetlib.dhp.countrypropagation.PrepareDatasourceCountryAssociation;
import eu.dnetlib.dhp.schema.oaf.Relation; import eu.dnetlib.dhp.schema.oaf.Relation;
import java.util.Arrays;
import java.util.List;
import org.apache.commons.io.IOUtils; import org.apache.commons.io.IOUtils;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.io.compress.GzipCodec;
import org.apache.spark.SparkConf; import org.apache.spark.SparkConf;
import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.sql.*; import org.apache.spark.sql.*;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
import java.util.Arrays;
import java.util.List;
import static eu.dnetlib.dhp.PropagationConstant.*;
import static eu.dnetlib.dhp.PropagationConstant.getConstraintList;
import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkHiveSession;
public class PrepareProjectResultsAssociation { public class PrepareProjectResultsAssociation {
private static final Logger log = LoggerFactory.getLogger(PrepareDatasourceCountryAssociation.class); private static final Logger log =
LoggerFactory.getLogger(PrepareDatasourceCountryAssociation.class);
private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
public static void main(String[] args) throws Exception{ public static void main(String[] args) throws Exception {
String jsonConfiguration = IOUtils.toString(PrepareProjectResultsAssociation.class String jsonConfiguration =
.getResourceAsStream("/eu/dnetlib/dhp/projecttoresult/input_prepareprojecttoresult_parameters.json")); IOUtils.toString(
PrepareProjectResultsAssociation.class.getResourceAsStream(
"/eu/dnetlib/dhp/projecttoresult/input_prepareprojecttoresult_parameters.json"));
final ArgumentApplicationParser parser = new ArgumentApplicationParser( final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration);
jsonConfiguration);
parser.parseArgument(args); parser.parseArgument(args);
@ -53,69 +52,88 @@ public class PrepareProjectResultsAssociation {
SparkConf conf = new SparkConf(); SparkConf conf = new SparkConf();
conf.set("hive.metastore.uris", parser.get("hive_metastore_uris")); conf.set("hive.metastore.uris", parser.get("hive_metastore_uris"));
runWithSparkHiveSession(conf, isSparkSessionManaged, runWithSparkHiveSession(
conf,
isSparkSessionManaged,
spark -> { spark -> {
// removeOutputDir(spark, potentialUpdatePath); // removeOutputDir(spark, potentialUpdatePath);
// removeOutputDir(spark, alreadyLinkedPath); // removeOutputDir(spark, alreadyLinkedPath);
prepareResultProjProjectResults(spark, inputPath, potentialUpdatePath, alreadyLinkedPath, allowedsemrel); prepareResultProjProjectResults(
spark,
inputPath,
potentialUpdatePath,
alreadyLinkedPath,
allowedsemrel);
}); });
} }
private static void prepareResultProjProjectResults(SparkSession spark, String inputPath, String potentialUpdatePath, private static void prepareResultProjProjectResults(
String alreadyLinkedPath, List<String> allowedsemrel) { SparkSession spark,
String inputPath,
String potentialUpdatePath,
String alreadyLinkedPath,
List<String> allowedsemrel) {
JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); JavaSparkContext sc = new JavaSparkContext(spark.sparkContext());
Dataset<Relation> relation = spark.createDataset(sc.textFile(inputPath ) Dataset<Relation> relation =
.map(item -> OBJECT_MAPPER.readValue(item, Relation.class)).rdd(), Encoders.bean(Relation.class)); spark.createDataset(
sc.textFile(inputPath)
.map(item -> OBJECT_MAPPER.readValue(item, Relation.class))
.rdd(),
Encoders.bean(Relation.class));
relation.createOrReplaceTempView("relation"); relation.createOrReplaceTempView("relation");
String query = "SELECT source, target " + String query =
" FROM relation " + "SELECT source, target "
" WHERE datainfo.deletedbyinference = false " + + " FROM relation "
" AND relClass = '" + RELATION_RESULT_PROJECT_REL_CLASS + "'"; + " WHERE datainfo.deletedbyinference = false "
+ " AND relClass = '"
+ RELATION_RESULT_PROJECT_REL_CLASS
+ "'";
Dataset<Row> resproj_relation = spark.sql(query); Dataset<Row> resproj_relation = spark.sql(query);
resproj_relation.createOrReplaceTempView("resproj_relation"); resproj_relation.createOrReplaceTempView("resproj_relation");
query ="SELECT projectId, collect_set(resId) resultSet " + query =
"FROM (" + "SELECT projectId, collect_set(resId) resultSet "
" SELECT r1.target resId, r2.target projectId " + + "FROM ("
" FROM (SELECT source, target " + + " SELECT r1.target resId, r2.target projectId "
" FROM relation " + + " FROM (SELECT source, target "
" WHERE datainfo.deletedbyinference = false " + + " FROM relation "
getConstraintList(" relClass = '", allowedsemrel ) + ") r1" + + " WHERE datainfo.deletedbyinference = false "
" JOIN resproj_relation r2 " + + getConstraintList(" relClass = '", allowedsemrel)
" ON r1.source = r2.source " + + ") r1"
" ) tmp " + + " JOIN resproj_relation r2 "
"GROUP BY projectId "; + " ON r1.source = r2.source "
+ " ) tmp "
spark.sql(query).as(Encoders.bean(ProjectResultSet.class)) + "GROUP BY projectId ";
.toJSON()
.write()
.mode(SaveMode.Overwrite)
.option("compression","gzip")
.text(potentialUpdatePath);
// .toJavaRDD()
// .map(r -> OBJECT_MAPPER.writeValueAsString(r))
// .saveAsTextFile(potentialUpdatePath, GzipCodec.class);
query = "SELECT target projectId, collect_set(source) resultSet " +
"FROM resproj_relation " +
"GROUP BY target";
spark.sql(query) spark.sql(query)
.as(Encoders.bean(ProjectResultSet.class)) .as(Encoders.bean(ProjectResultSet.class))
.toJSON() .toJSON()
.write() .write()
.mode(SaveMode.Overwrite) .mode(SaveMode.Overwrite)
.option("compression","gzip") .option("compression", "gzip")
.text(alreadyLinkedPath); .text(potentialUpdatePath);
// .toJavaRDD() // .toJavaRDD()
// .map(r -> OBJECT_MAPPER.writeValueAsString(r)) // .map(r -> OBJECT_MAPPER.writeValueAsString(r))
// .saveAsTextFile(alreadyLinkedPath, GzipCodec.class); // .saveAsTextFile(potentialUpdatePath, GzipCodec.class);
query =
"SELECT target projectId, collect_set(source) resultSet "
+ "FROM resproj_relation "
+ "GROUP BY target";
spark.sql(query)
.as(Encoders.bean(ProjectResultSet.class))
.toJSON()
.write()
.mode(SaveMode.Overwrite)
.option("compression", "gzip")
.text(alreadyLinkedPath);
// .toJavaRDD()
// .map(r -> OBJECT_MAPPER.writeValueAsString(r))
// .saveAsTextFile(alreadyLinkedPath, GzipCodec.class);
} }
} }

View File

@ -1,31 +1,34 @@
package eu.dnetlib.dhp.projecttoresult; package eu.dnetlib.dhp.projecttoresult;
import static eu.dnetlib.dhp.PropagationConstant.*;
import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession;
import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.ObjectMapper;
import eu.dnetlib.dhp.application.ArgumentApplicationParser; import eu.dnetlib.dhp.application.ArgumentApplicationParser;
import eu.dnetlib.dhp.countrypropagation.PrepareDatasourceCountryAssociation; import eu.dnetlib.dhp.countrypropagation.PrepareDatasourceCountryAssociation;
import eu.dnetlib.dhp.schema.oaf.Relation; import eu.dnetlib.dhp.schema.oaf.Relation;
import java.util.ArrayList;
import java.util.List;
import org.apache.commons.io.IOUtils; import org.apache.commons.io.IOUtils;
import org.apache.spark.SparkConf; import org.apache.spark.SparkConf;
import org.apache.spark.sql.*; import org.apache.spark.sql.*;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
import java.util.ArrayList;
import java.util.List;
import static eu.dnetlib.dhp.PropagationConstant.*;
import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession;
public class SparkResultToProjectThroughSemRelJob3 { public class SparkResultToProjectThroughSemRelJob3 {
private static final Logger log = LoggerFactory.getLogger(PrepareDatasourceCountryAssociation.class); private static final Logger log =
LoggerFactory.getLogger(PrepareDatasourceCountryAssociation.class);
private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
public static void main(String[] args) throws Exception { public static void main(String[] args) throws Exception {
String jsonConfiguration = IOUtils.toString(SparkResultToProjectThroughSemRelJob3.class String jsonConfiguration =
.getResourceAsStream("/eu/dnetlib/dhp/projecttoresult/input_projecttoresult_parameters.json")); IOUtils.toString(
SparkResultToProjectThroughSemRelJob3.class.getResourceAsStream(
"/eu/dnetlib/dhp/projecttoresult/input_projecttoresult_parameters.json"));
final ArgumentApplicationParser parser = new ArgumentApplicationParser( final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration);
jsonConfiguration);
parser.parseArgument(args); parser.parseArgument(args);
@ -49,84 +52,104 @@ public class SparkResultToProjectThroughSemRelJob3 {
SparkConf conf = new SparkConf(); SparkConf conf = new SparkConf();
runWithSparkSession(
runWithSparkSession(conf, isSparkSessionManaged, conf,
isSparkSessionManaged,
spark -> { spark -> {
if(isTest(parser)) { if (isTest(parser)) {
removeOutputDir(spark, outputPath); removeOutputDir(spark, outputPath);
} }
execPropagation(spark, outputPath, alreadyLinkedPath, potentialUpdatePath, writeUpdates, saveGraph); execPropagation(
spark,
outputPath,
alreadyLinkedPath,
potentialUpdatePath,
writeUpdates,
saveGraph);
}); });
} }
private static void execPropagation(
SparkSession spark,
String outputPath,
String alreadyLinkedPath,
String potentialUpdatePath,
Boolean writeUpdate,
Boolean saveGraph) {
private static void execPropagation(SparkSession spark, String outputPath, String alreadyLinkedPath, String potentialUpdatePath, Dataset<ProjectResultSet> toaddrelations =
Boolean writeUpdate, Boolean saveGraph){ readAssocProjectResults(spark, potentialUpdatePath);
Dataset<ProjectResultSet> alreadyLinked = readAssocProjectResults(spark, alreadyLinkedPath);
Dataset<ProjectResultSet> toaddrelations = readAssocProjectResults(spark, potentialUpdatePath); if (writeUpdate) {
Dataset<ProjectResultSet> alreadyLinked = readAssocProjectResults(spark, alreadyLinkedPath);
if(writeUpdate){
toaddrelations toaddrelations
.toJSON() .toJSON()
.write() .write()
.mode(SaveMode.Overwrite) .mode(SaveMode.Overwrite)
.option("compression","gzip") .option("compression", "gzip")
.text(outputPath +"/potential_updates"); .text(outputPath + "/potential_updates");
} }
if (saveGraph){ if (saveGraph) {
getNewRelations(alreadyLinked, toaddrelations) getNewRelations(alreadyLinked, toaddrelations)
.toJSON() .toJSON()
.write() .write()
.mode(SaveMode.Append) .mode(SaveMode.Append)
.option("compression", "gzip") .option("compression", "gzip")
.text(outputPath); .text(outputPath);
} }
} }
private static Dataset<Relation> getNewRelations(Dataset<ProjectResultSet> alreadyLinked, private static Dataset<Relation> getNewRelations(
Dataset<ProjectResultSet> toaddrelations){ Dataset<ProjectResultSet> alreadyLinked, Dataset<ProjectResultSet> toaddrelations) {
return toaddrelations return toaddrelations
.joinWith(alreadyLinked, toaddrelations.col("projectId").equalTo(alreadyLinked.col("projectId")), "left") .joinWith(
.flatMap(value -> { alreadyLinked,
List<Relation> new_relations = new ArrayList<>(); toaddrelations.col("projectId").equalTo(alreadyLinked.col("projectId")),
ProjectResultSet potential_update = value._1(); "left")
ProjectResultSet already_linked = value._2(); .flatMap(
String projId = already_linked.getProjectId(); value -> {
potential_update List<Relation> new_relations = new ArrayList<>();
.getResultSet() ProjectResultSet potential_update = value._1();
.stream() ProjectResultSet already_linked = value._2();
.forEach(rId -> { String projId = already_linked.getProjectId();
if (!already_linked.getResultSet().contains(rId)){ potential_update.getResultSet().stream()
new_relations.add(getRelation(rId, projId, RELATION_RESULT_PROJECT_REL_CLASS, RELATION_RESULTPROJECT_REL_TYPE, .forEach(
RELATION_RESULTPROJECT_SUBREL_TYPE, PROPAGATION_DATA_INFO_TYPE, rId -> {
PROPAGATION_RELATION_RESULT_PROJECT_SEM_REL_CLASS_ID, if (!already_linked.getResultSet().contains(rId)) {
PROPAGATION_RELATION_RESULT_PROJECT_SEM_REL_CLASS_NAME)); new_relations.add(
new_relations.add(getRelation(projId, rId, RELATION_PROJECT_RESULT_REL_CLASS, RELATION_RESULTPROJECT_REL_TYPE, getRelation(
RELATION_RESULTPROJECT_SUBREL_TYPE, PROPAGATION_DATA_INFO_TYPE, rId,
PROPAGATION_RELATION_RESULT_PROJECT_SEM_REL_CLASS_ID, projId,
PROPAGATION_RELATION_RESULT_PROJECT_SEM_REL_CLASS_NAME)); RELATION_RESULT_PROJECT_REL_CLASS,
} RELATION_RESULTPROJECT_REL_TYPE,
}); RELATION_RESULTPROJECT_SUBREL_TYPE,
return new_relations.iterator(); PROPAGATION_DATA_INFO_TYPE,
PROPAGATION_RELATION_RESULT_PROJECT_SEM_REL_CLASS_ID,
} PROPAGATION_RELATION_RESULT_PROJECT_SEM_REL_CLASS_NAME));
,Encoders.bean(Relation.class)); new_relations.add(
getRelation(
projId,
rId,
RELATION_PROJECT_RESULT_REL_CLASS,
} RELATION_RESULTPROJECT_REL_TYPE,
RELATION_RESULTPROJECT_SUBREL_TYPE,
private static Dataset<ProjectResultSet> readAssocProjectResults(SparkSession spark, String potentialUpdatePath) { PROPAGATION_DATA_INFO_TYPE,
return spark PROPAGATION_RELATION_RESULT_PROJECT_SEM_REL_CLASS_ID,
.read() PROPAGATION_RELATION_RESULT_PROJECT_SEM_REL_CLASS_NAME));
.textFile(potentialUpdatePath) }
.map(value -> OBJECT_MAPPER.readValue(value, ProjectResultSet.class), Encoders.bean(ProjectResultSet.class)); });
return new_relations.iterator();
},
Encoders.bean(Relation.class));
} }
private static Dataset<ProjectResultSet> readAssocProjectResults(
SparkSession spark, String potentialUpdatePath) {
return spark.read()
.textFile(potentialUpdatePath)
.map(
value -> OBJECT_MAPPER.readValue(value, ProjectResultSet.class),
Encoders.bean(ProjectResultSet.class));
}
} }