[DUMP CSV] tested file to execute the dump of the relations with semantcis Cites from nodes belonging to a selected communities. It also dumps the relationships result_communities and prepare the ground for the dump of the results.

This commit is contained in:
Miriam Baglioni 2023-05-16 14:20:45 +02:00
parent 44a256fc90
commit 2ed76d4662
1 changed files with 180 additions and 138 deletions

View File

@ -1,12 +1,15 @@
package eu.dnetlib.dhp.oa.graph.dump.csv;
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 static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession;
import java.io.Serializable;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
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;
@ -17,165 +20,204 @@ 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;
import java.io.Serializable;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import java.util.Optional;
import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession;
/**
* @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"));
private static final Logger log = LoggerFactory.getLogger(SparkSelectResultsAndDumpRelations.class);
final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration);
parser.parseArgument(args);
public static void main(String[] args) throws Exception {
String jsonConfiguration = IOUtils
.toString(
SparkSelectResultsAndDumpRelations.class
.getResourceAsStream(
"/eu/dnetlib/dhp/oa/graph/dump/input_dump_csv_step1.json"));
Boolean isSparkSessionManaged = Optional
.ofNullable(parser.get("isSparkSessionManaged"))
.map(Boolean::valueOf)
.orElse(Boolean.TRUE);
log.info("isSparkSessionManaged: {}", isSparkSessionManaged);
final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration);
parser.parseArgument(args);
final String inputPath = parser.get("sourcePath");
log.info("inputPath: {}", inputPath);
Boolean isSparkSessionManaged = Optional
.ofNullable(parser.get("isSparkSessionManaged"))
.map(Boolean::valueOf)
.orElse(Boolean.TRUE);
log.info("isSparkSessionManaged: {}", isSparkSessionManaged);
final String outputPath = parser.get("outputPath");
log.info("outputPath: {}", outputPath);
final String inputPath = parser.get("isLoo");
log.info("inputPath: {}", inputPath);
final String workingPath = parser.get("workingPath");
final String outputPath = parser.get("outputPath");
log.info("outputPath: {}", outputPath);
List<String> communityList = null;
Optional<String> communities = Optional.ofNullable(parser.get("communities"));
if (communities.isPresent()) {
communityList = Arrays.asList(communities.get().split(";"));
}
final String workingPath = parser.get("workingPath");
SparkConf conf = new SparkConf();
List<String> communityList = null;
Optional<String> communities = Optional.ofNullable(parser.get("communities"));
if(communities.isPresent()){
communityList = Arrays.asList(communities.get().split(";"));
}
List<String> finalCommunityList = communityList;
runWithSparkSession(
conf,
isSparkSessionManaged,
spark -> {
Utils.removeOutputDir(spark, outputPath);
run(spark, inputPath, outputPath, workingPath, finalCommunityList);
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) {
}
private static <R extends Result> 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 + "/communityResultIds");
writeCommunityRelatedIds(spark, inputPath + "/dataset", Dataset.class, communityList, workingPath + "/communityResultIds");
writeCommunityRelatedIds(spark, inputPath + "/software", Software.class, communityList, workingPath + "/communityResultIds" );
writeCommunityRelatedIds(spark, inputPath + "/otherresearchproduct", OtherResearchProduct.class, communityList, workingPath + "/communityResultIds");
//write the relations result communities
writeCommunityResultRelations(spark, inputPath + "/publication", Publication.class, communityList, workingPath + "/communityResultIds");
writeCommunityResultRelations(spark, inputPath + "/dataset", Dataset.class, communityList, workingPath + "/communityResultIds");
writeCommunityResultRelations(spark, inputPath + "/software", Software.class, communityList, workingPath + "/communityResultIds" );
writeCommunityResultRelations(spark, inputPath + "/otherresearchproduct", OtherResearchProduct.class, communityList, workingPath + "/communityResultIds");
//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(outputPath + "/communityResultIds").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")
.mode(SaveMode.Overwrite)
.csv(outputPath + "/relation");
// 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);
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;
}
// 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));
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-> {
List<CSVRELCommunityResult> ret = new ArrayList<>();
for(Context context : p.getContext()){
String cId = context.getId().contains("::") ? context.getId().substring(0, context.getId().indexOf("::")) : context.getId();
if (communityList.contains(cId)){
CSVRELCommunityResult crc = new CSVRELCommunityResult();
crc.setResult_id(p.getId());
crc.setCommunity_id(DHPUtils.md5(cId));
}
}
return ret.iterator();
}, Encoders.bean(CSVRELCommunityResult.class) )
.write()
.option("compression","gzip")
.mode(SaveMode.Append)
.text(outputPath );
}
// 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")
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>) p-> p.getId(), Encoders.STRING() )
.write()
.option("compression","gzip")
.mode(SaveMode.Append)
.text(outputPath );
}
.text(workingPath + "/resultIds");
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("::")))));
}
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 -> {
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)) {
CSVRELCommunityResult crc = new CSVRELCommunityResult();
crc.setResult_id(p.getId());
crc.setCommunity_id(DHPUtils.md5(cId));
ret.add(crc);
}
}
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("::")))));
}
}