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

228 lines
7.9 KiB
Java

package eu.dnetlib.dhp.oa.graph.dump.csv;
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.model.CSVCitation;
import eu.dnetlib.dhp.oa.graph.dump.csv.model.CSVRELCommunityResult;
import eu.dnetlib.dhp.schema.common.ModelConstants;
import eu.dnetlib.dhp.schema.oaf.*;
import eu.dnetlib.dhp.utils.DHPUtils;
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", Publication.class, communityList, workingPath + COMMUNITY_RESULT_IDS);
writeCommunityRelatedIds(
spark, inputPath + "/dataset", Dataset.class, communityList, workingPath + COMMUNITY_RESULT_IDS);
writeCommunityRelatedIds(
spark, inputPath + "/software", Software.class, communityList, workingPath + COMMUNITY_RESULT_IDS);
writeCommunityRelatedIds(
spark, inputPath + "/otherresearchproduct", OtherResearchProduct.class, communityList,
workingPath + COMMUNITY_RESULT_IDS);
// write the relations result communities
writeCommunityResultRelations(
spark, inputPath + "/publication", Publication.class, communityList, outputPath + RESULT_COMMUNITY_TABLE);
writeCommunityResultRelations(
spark, inputPath + "/dataset", Dataset.class, communityList, outputPath + RESULT_COMMUNITY_TABLE);
writeCommunityResultRelations(
spark, inputPath + "/software", Software.class, communityList, outputPath + RESULT_COMMUNITY_TABLE);
writeCommunityResultRelations(
spark, inputPath + "/otherresearchproduct", OtherResearchProduct.class, communityList,
outputPath + RESULT_COMMUNITY_TABLE);
// 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 results target of the selected relations having as source one of the results related to the
// communities
org.apache.spark.sql.Dataset<String> resultIds = spark
.read()
.textFile(workingPath + COMMUNITY_RESULT_IDS)
.distinct();
resultIds
.joinWith(relations, resultIds.col("value").equalTo(relations.col("source")), "left")
.flatMap((FlatMapFunction<Tuple2<String, Relation>, String>) t2 -> {
if (Optional.ofNullable(t2._2()).isPresent()) {
return Arrays.asList(t2._1(), t2._2().getTarget()).iterator();
} else {
return Arrays.asList(t2._1()).iterator();
}
}, Encoders.STRING())
.distinct()
.write()
.mode(SaveMode.Overwrite)
.option("compression", "gzip")
.text(workingPath + "/resultIds");
resultIds
.joinWith(relations, resultIds.col("value").equalTo(relations.col("source")))
.map(
(MapFunction<Tuple2<String, Relation>, CSVCitation>) t2 -> mapToCitation(t2._2()),
Encoders.bean(CSVCitation.class))
.write()
.option("compression", "gzip")
.option("header", "true")
.option("delimiter", Constants.SEP)
.mode(SaveMode.Overwrite)
.csv(outputPath + "/relation");
}
private static CSVCitation mapToCitation(Relation relation) {
CSVCitation ret = new CSVCitation();
ret.setId(DHPUtils.md5(relation.getSource() + relation.getRelClass().toLowerCase() + relation.getTarget()));
ret.setResult_id_cites(relation.getSource());
ret.setResult_id_cited(relation.getTarget());
return ret;
}
private static <R extends Result> void writeCommunityResultRelations(SparkSession spark, String inputPath,
Class<R> clazz, List<String> communityList, String outputPath) {
Utils
.readPath(spark, inputPath, clazz)
.filter(
(FilterFunction<R>) p -> !p.getDataInfo().getDeletedbyinference() &&
!p.getDataInfo().getInvisible())
.flatMap((FlatMapFunction<R, CSVRELCommunityResult>) p -> {
Set<String> inserted = new HashSet<>();
List<CSVRELCommunityResult> ret = new ArrayList<>();
for (String context : p
.getContext()
.stream()
.map(Context::getId)
.distinct()
.collect(Collectors.toList())) {
String cId = context.contains("::")
? context.substring(0, context.indexOf("::"))
: context;
if (communityList.contains(cId) && !inserted.contains(cId)) {
CSVRELCommunityResult crc = new CSVRELCommunityResult();
crc.setResult_id(p.getId());
crc.setCommunity_id(DHPUtils.md5(cId));
ret.add(crc);
inserted.add(cId);
}
}
return ret.iterator();
}, Encoders.bean(CSVRELCommunityResult.class))
.write()
.option("compression", "gzip")
.mode(SaveMode.Append)
.option("header", "true")
.option("delimiter", Constants.SEP)
.csv(outputPath);
}
private static <R extends Result> void writeCommunityRelatedIds(SparkSession spark, String inputPath,
Class<R> clazz, List<String> communityList, String outputPath) {
Utils
.readPath(spark, inputPath, clazz)
.filter(
(FilterFunction<R>) p -> !p.getDataInfo().getDeletedbyinference() &&
!p.getDataInfo().getInvisible() &&
isRelatedToCommunities(p, communityList))
.map((MapFunction<R, String>) Result::getId, Encoders.STRING())
.write()
.option("compression", "gzip")
.mode(SaveMode.Append)
.text(outputPath);
}
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("::")))));
}
}