dhp-graph-dump/dump/src/main/java/eu/dnetlib/dhp/oa/graph/dump/filterentities/SelectConnectedEntities.java

349 lines
12 KiB
Java

package eu.dnetlib.dhp.oa.graph.dump.filterentities;
import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession;
import java.io.Serializable;
import java.util.Arrays;
import java.util.Optional;
import java.util.function.Predicate;
import java.util.stream.Collectors;
import java.util.stream.Stream;
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.apache.spark.sql.types.DataType;
import org.apache.spark.sql.types.DataTypes;
import org.apache.spark.sql.types.StructType;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import com.fasterxml.jackson.core.JsonProcessingException;
import com.fasterxml.jackson.databind.ObjectMapper;
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 -> {
selectConnectedEntities2(spark, inputPath, filterPath, workingDir);
});
}
private static void selectConnectedEntities2(SparkSession spark, String inputPath, String filterPath,
String workingDir) {
final StructType rp = new StructType()
.add(
"dataInfo", new StructType()
.add("deletedbyinference", DataTypes.BooleanType))
.add("source", DataTypes.StringType)
.add("target", DataTypes.StringType);
Dataset<String> resultIds = spark.emptyDataset(Encoders.STRING());
for (EntityType entity : ModelSupport.entityTypes.keySet())
if (ModelSupport.isResult(entity))
resultIds = resultIds
.union(
spark
.read()
.parquet(filterPath + entity.name() + "_ids")
.select("id")
.as(Encoders.STRING()));
Dataset<Row> relation = spark
.read()
.schema(rp)
.json(inputPath + "relation")
.filter(("datainfo.deletedbyinference != true"))
.drop("datainfo");
Dataset<Row> matchingRels = relation
.join(
resultIds, relation
.col("source")
.equalTo(resultIds.col("value")),
"leftsemi")
.select("target")
.distinct()
;
Dataset<Row> organization = spark
.read()
.schema(Encoders.bean(Organization.class).schema())
.json(inputPath + "organization")
.filter("datainfo.deletedbyinference != true ");
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"));
organization
.join(matchingRels, organization.col("id").equalTo(matchingRels.col("target")), "leftsemi")
.write()
.mode(SaveMode.Overwrite)
.option("compression", "gzip")
.json(workingDir + "organization");
projects
.join(matchingRels, projects.col("id").equalTo(matchingRels.col("target")), "leftsemi")
.write()
.mode(SaveMode.Overwrite)
.option("compression", "gzip")
.json(workingDir + "/project");
Dataset<Row> datasources = spark
.read()
.schema(Encoders.bean(Datasource.class).schema())
.json(inputPath + "datasource")
.filter("datainfo.deletedbyinference != true");
final Dataset<String> datasourceReferencedIds = getDatasourceReferenceIdDataset(spark, workingDir);
datasources
.join(
datasourceReferencedIds, datasourceReferencedIds.col("value").equalTo(datasources.col("id")),
"left_semi")
.write()
.mode(SaveMode.Overwrite)
.option("compression", "gzip")
.json(workingDir + "datasource");
}
// private static <R extends Result> void selectConnectedEntities(SparkSession spark, String inputPath,
// String filterPath,
// String workingDir) throws JsonProcessingException {
//
// Dataset<String> resultIds = spark.emptyDataset(Encoders.STRING());
// for (EntityType entity : ModelSupport.entityTypes.keySet())
// if (ModelSupport.isResult(entity))
// resultIds = resultIds
// .union(
// spark
// .read()
// .parquet(filterPath + entity.name() + "_ids")
// .select("id")
// .as(Encoders.STRING()));
//
// 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<Row> relationSource = relation
// .join(resultIds, resultIds.col("value").equalTo(relation.col("source")), "left_semi");
// relationSource
// .join(resultIds, resultIds.col("value").equalTo(relation.col("target")), "left_semi")
// .write()
// .option("compression", "gzip")
// .mode(SaveMode.Overwrite)
// .json(workingDir + "resultrelation");
////
//// // write relations between results and organizations
// relationSource
// .joinWith(organizations, relation.col("target").equalTo(organizations.col("id")), "left_semi")
// .write()
// .mode(SaveMode.Overwrite)
// .option("compression", "gzip")
// .json(workingDir + "organizaitonrelation");
//
// relationSource
// .joinWith(projects, relation.col("target").equalTo(projects.col("id")), "left_semi")
// .write()
// .mode(SaveMode.Overwrite)
// .option("compression", "gzip")
// .json(workingDir + "projectrelation");
//
// // write organizations linked to results in the set
//
// organizations
// .join(relationSource, relationSource.col("target").equalTo(organizations.col("id")), "left_semi")
// .write()
// .mode(SaveMode.Overwrite)
// .option("compression", "gzip")
// .json(workingDir + "organization");
//
// // write projects linked to results in the set
// projects
// .join(relationSource, relationSource.col("target").equalTo(projects.col("id")))
// .write()
// .mode(SaveMode.Overwrite)
// .option("compression", "gzip")
// .json(workingDir + "project");
//
// // read the results and select all the distinct instance.hostedbykey
// final Dataset<String> datasourceReferencedIds = getDatasourceReferenceIdDataset(spark, workingDir);
// // join with the datasources and write the datasource in the join
// datasources
// .joinWith(
// datasourceReferencedIds, datasourceReferencedIds.col("value").equalTo(datasources.col("id")),
// "left_semi")
// .write()
// .mode(SaveMode.Overwrite)
// .option("compression", "gzip")
// .json(workingDir + "datasource");
//
// // selecting relations between organizations and projects in the selected set
// StructType tp = StructType.fromDDL("`id` STRING");
// Dataset<Row> organizationSbs = spark
// .read()
// .schema(tp)
// .json(workingDir + "organization")
// .select("id");
//
// Dataset<Row> projectSbs = spark
// .read()
// .schema(tp)
// .json(workingDir + "project")
// .select("id");
////
// Dataset<Row> tmpRel;
// tmpRel = relation
// .join(
// organizationSbs, organizationSbs
// .col("id")
// .equalTo(relation.col("source")),
// "left_semi");
// tmpRel
// .join(projectSbs, tmpRel.col("target").equalTo(projectSbs.col("id")), "left_semi")
// .write()
// .mode(SaveMode.Overwrite)
// .option("compression", "gzip")
// .json(workingDir + "orgprojelation");
//
// // selecting relations between datasources and organizations in the selected set
// Dataset<Row> datasourceSbs = spark
// .read()
// .schema(tp)
// .json(workingDir + "datasource")
// .select("id");
//
// tmpRel = relation
// .join(datasourceSbs, datasourceSbs.col("id").equalTo(relation.col("source")), "left_semi");
// tmpRel
// .join(organizationSbs, tmpRel.col("target").equalTo(organizationSbs.col("id")), "left_semi")
// .write()
// .mode(SaveMode.Overwrite)
// .option("compression", "gzip")
// .json(workingDir + "datsorgrelation");
//
// Utils
// .readPath(spark, workingDir + "resultrelation", Relation.class)
// .union(Utils.readPath(spark, workingDir + "organizaitonrelation", Relation.class))
// .union(Utils.readPath(spark, workingDir + "projectrelation", Relation.class))
// .union(Utils.readPath(spark, workingDir + "orgprojelation", Relation.class))
// .union(Utils.readPath(spark, workingDir + "datsorgrelation", Relation.class))
// .write()
// .mode(SaveMode.Overwrite)
// .option("compression", "gzip")
// .json(workingDir + "relation");
//
// }
private static <R extends Result> Dataset<String> getDatasourceReferenceIdDataset(SparkSession spark,
String workingDir) {
Dataset<String> datasourceReferencedIds = spark.emptyDataset(Encoders.STRING());
for (EntityType entity : ModelSupport.entityTypes.keySet())
if (ModelSupport.isResult(entity)) {
Class<R> resultClazz = ModelSupport.entityTypes.get(entity);
datasourceReferencedIds = datasourceReferencedIds
.union(
Utils
.readPath(spark, workingDir + entity.name(), resultClazz)
.flatMap(
(FlatMapFunction<R, String>) r -> r
.getInstance()
.stream()
.filter(i -> i.getHostedby() != null && i.getHostedby().getKey() != null)
.map(i -> i.getHostedby().getKey())
.collect(Collectors.toList())
.iterator(),
Encoders.STRING()));
datasourceReferencedIds = datasourceReferencedIds
.union(
Utils
.readPath(spark, workingDir + entity.name(), resultClazz)
.flatMap(
(FlatMapFunction<R, String>) r -> r
.getInstance()
.stream()
.filter(i -> i.getCollectedfrom() != null && i.getCollectedfrom().getKey() != null)
.map(i -> i.getCollectedfrom().getKey())
.collect(Collectors.toList())
.iterator(),
Encoders.STRING()));
}
datasourceReferencedIds = datasourceReferencedIds.distinct();
return datasourceReferencedIds;
}
}