dhp-graph-dump/dump/src/main/java/eu/dnetlib/dhp/oa/graph/dump/serafeim/SparkSelectResultsAndDumpRe...

242 lines
8.6 KiB
Java

package eu.dnetlib.dhp.oa.graph.dump.serafeim;
import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession;
import java.io.Serializable;
import java.util.*;
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.Encoders;
import org.apache.spark.sql.SaveMode;
import org.apache.spark.sql.SparkSession;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
import eu.dnetlib.dhp.oa.graph.dump.Utils;
import eu.dnetlib.dhp.oa.graph.dump.csv.Constants;
import eu.dnetlib.dhp.schema.common.ModelConstants;
import eu.dnetlib.dhp.schema.oaf.*;
import scala.Tuple2;
/**
* @author miriam.baglioni
* @Date 04/05/23
*/
//STEP 2
public class SparkSelectResultsAndDumpRelations implements Serializable {
private static final Logger log = LoggerFactory.getLogger(SparkSelectResultsAndDumpRelations.class);
private static String RESULT_COMMUNITY_TABLE = "/result_community";
private static String COMMUNITY_RESULT_IDS = "/communityResultIds";
public static void main(String[] args) throws Exception {
String jsonConfiguration = IOUtils
.toString(
SparkSelectResultsAndDumpRelations.class
.getResourceAsStream(
"/eu/dnetlib/dhp/oa/graph/dump/input_dump_csv_ste2.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 outputPath = parser.get("outputPath");
log.info("outputPath: {}", outputPath);
final String workingPath = parser.get("workingPath");
List<String> communityList = null;
Optional<String> communities = Optional.ofNullable(parser.get("communities"));
if (communities.isPresent()) {
communityList = Arrays.asList(communities.get().split(";"));
}
SparkConf conf = new SparkConf();
List<String> finalCommunityList = communityList;
runWithSparkSession(
conf,
isSparkSessionManaged,
spark -> {
Utils.removeOutputDir(spark, outputPath);
run(spark, inputPath, outputPath, workingPath, finalCommunityList);
});
}
private static void run(SparkSession spark, String inputPath, String outputPath,
String workingPath,
List<String> communityList) {
// select the result ids related to the set of communities considered
writeCommunityRelatedIds(
spark, inputPath, Publication.class, communityList, workingPath, "publication");
writeCommunityRelatedIds(
spark, inputPath, Dataset.class, communityList, workingPath, "dataset");
writeCommunityRelatedIds(
spark, inputPath, Software.class, communityList, workingPath, "software");
writeCommunityRelatedIds(
spark, inputPath, OtherResearchProduct.class, communityList,
workingPath, "otherresearchproduct");
// select the relations with semantics cites
org.apache.spark.sql.Dataset<Relation> relations = Utils
.readPath(spark, inputPath + "/relation", Relation.class)
.filter(
(FilterFunction<Relation>) r -> !r.getDataInfo().getDeletedbyinference() &&
r.getRelClass().equals(ModelConstants.CITES));
// select the relations having as source one of the results related to the
// communities
org.apache.spark.sql.Dataset<String> communityResultIds = spark
.read()
.textFile(workingPath + COMMUNITY_RESULT_IDS)
.distinct();
Utils
.readPath(spark, inputPath + "/publication", Publication.class)
.filter(
(FilterFunction<Publication>) p -> !p.getDataInfo().getDeletedbyinference()
&& !p.getDataInfo().getInvisible())
.map((MapFunction<Publication, String>) p -> p.getId(), Encoders.STRING())
.union(
Utils
.readPath(spark, inputPath + "/dataset", Dataset.class)
.filter(
(FilterFunction<Dataset>) p -> !p.getDataInfo().getDeletedbyinference()
&& !p.getDataInfo().getInvisible())
.map((MapFunction<Dataset, String>) p -> p.getId(), Encoders.STRING()))
.union(
Utils
.readPath(spark, inputPath + "/software", Software.class)
.filter(
(FilterFunction<Software>) p -> !p.getDataInfo().getDeletedbyinference()
&& !p.getDataInfo().getInvisible())
.map((MapFunction<Software, String>) p -> p.getId(), Encoders.STRING()))
.union(
Utils
.readPath(spark, inputPath + "/otherresearchproduct", OtherResearchProduct.class)
.filter(
(FilterFunction<OtherResearchProduct>) p -> !p.getDataInfo().getDeletedbyinference()
&& !p.getDataInfo().getInvisible())
.map((MapFunction<OtherResearchProduct, String>) p -> p.getId(), Encoders.STRING()))
.write()
.mode(SaveMode.Overwrite)
.option("compression", "gzip")
.text(workingPath + "/resultIds");
org.apache.spark.sql.Dataset<String> resultIds = spark.read().textFile(workingPath + "/resultIds");
org.apache.spark.sql.Dataset<Relation> oksource = communityResultIds
.joinWith(relations, communityResultIds.col("value").equalTo(relations.col("source")))
.map(
(MapFunction<Tuple2<String, Relation>, Relation>) t2 -> t2._2(),
Encoders.bean(Relation.class));
oksource
.joinWith(resultIds, oksource.col("target").equalTo(resultIds.col("value")))
.map((MapFunction<Tuple2<Relation, String>, Relation>) t2 -> t2._1(), Encoders.bean(Relation.class))
.write()
.option("compression", "gzip")
.mode(SaveMode.Overwrite)
.json(outputPath + "/relation");
writeNodes(
spark, inputPath + "/publication", Publication.class, outputPath + "/publication",
outputPath + "/relation", workingPath);
writeNodes(
spark, inputPath + "/dataset", Dataset.class, outputPath + "/dataset", outputPath + "/relation",
workingPath);
writeNodes(
spark, inputPath + "/software", Software.class, outputPath + "/software", outputPath + "/relation",
workingPath);
writeNodes(
spark, inputPath + "/otherresearchproduct", OtherResearchProduct.class,
outputPath + "/otherresearchproduct", outputPath + "/relation", workingPath);
}
private static <R extends Result> void writeNodes(SparkSession spark, String inputPath, Class<R> clazz,
String outputPath, String relationPath, String workingPath) {
org.apache.spark.sql.Dataset<Relation> citingRelations = Utils.readPath(spark, relationPath, Relation.class);
org.apache.spark.sql.Dataset<R> result = Utils
.readPath(spark, inputPath, clazz)
.filter(
(FilterFunction<R>) p -> !p.getDataInfo().getDeletedbyinference() &&
!p.getDataInfo().getInvisible());
// take the distinct result id for source and target of the relations
citingRelations
.flatMap(
(FlatMapFunction<Relation, String>) r -> Arrays
.asList(r.getSource(), r.getTarget())
.iterator(),
Encoders.STRING())
.distinct()
.write()
.option("compression", "gzip")
.mode(SaveMode.Overwrite)
.text(workingPath + "/relationIds");
org.apache.spark.sql.Dataset<String> relationIds = spark.read().textFile(workingPath + "/relationIds");
relationIds
.joinWith(result, relationIds.col("value").equalTo(result.col("id")))
.map((MapFunction<Tuple2<String, R>, R>) t2 -> t2._2(), Encoders.bean(clazz))
.write()
.option("compression", "gzip")
.mode(SaveMode.Overwrite)
.json(outputPath);
}
private static <R extends Result> void writeCommunityRelatedIds(SparkSession spark, String inputPath,
Class<R> clazz, List<String> communityList, String outputPath, String resultType) {
org.apache.spark.sql.Dataset<R> results = Utils
.readPath(spark, inputPath + "/" + resultType, clazz)
.filter(
(FilterFunction<R>) p -> !p.getDataInfo().getDeletedbyinference() &&
!p.getDataInfo().getInvisible() &&
isRelatedToCommunities(p, communityList));
results
.map((MapFunction<R, String>) Result::getId, Encoders.STRING())
.write()
.option("compression", "gzip")
.mode(SaveMode.Append)
.text(outputPath + COMMUNITY_RESULT_IDS);
// results
// // .repartition(10000)
// .write()
// .option("compression", "gzip")
// .mode(SaveMode.Append)
// .json(outputPath + "/" + resultType);
}
private static <R extends Result> boolean isRelatedToCommunities(R p, List<String> communityList) {
return p
.getContext()
.stream()
.anyMatch(
c -> communityList.contains(c.getId()) ||
(c.getId().contains("::")
&& communityList.contains(c.getId().substring(0, c.getId().indexOf("::")))));
}
}