dnet-hadoop/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/CreateRelatedEntitiesJob_ph...

183 lines
8.3 KiB
Java
Raw Normal View History

2020-04-04 14:03:43 +02:00
package eu.dnetlib.dhp.oa.provision;
import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession;
import static eu.dnetlib.dhp.oa.provision.utils.GraphMappingUtils.*;
2020-04-04 14:03:43 +02:00
import com.fasterxml.jackson.databind.ObjectMapper;
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
import eu.dnetlib.dhp.common.HdfsSupport;
import eu.dnetlib.dhp.oa.provision.model.EntityRelEntity;
2020-04-06 15:33:31 +02:00
import eu.dnetlib.dhp.oa.provision.model.RelatedEntity;
2020-04-04 14:03:43 +02:00
import eu.dnetlib.dhp.oa.provision.model.SortableRelation;
import eu.dnetlib.dhp.schema.common.EntityType;
import eu.dnetlib.dhp.schema.common.ModelSupport;
2020-04-06 15:33:31 +02:00
import eu.dnetlib.dhp.schema.oaf.OafEntity;
import java.util.Optional;
2020-04-04 14:03:43 +02:00
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.MapFunction;
import org.apache.spark.sql.Dataset;
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 scala.Tuple2;
/**
* Joins the graph nodes by resolving the links of distance = 1 to create an adjacency list of
* linked objects. The operation considers all the entity types (publication, dataset, software,
* ORP, project, datasource, organization, and all the possible relationships (similarity links
* produced by the Dedup process are excluded).
2020-04-04 14:03:43 +02:00
*
* <p>The operation is implemented by sequentially joining one entity type at time (E) with the
* relationships (R), and again by E, finally grouped by E.id;
2020-04-04 14:03:43 +02:00
*
* <p>The workflow is organized in different parts aimed to to reduce the complexity of the
* operation 1) PrepareRelationsJob: only consider relationships that are not virtually deleted
* ($.dataInfo.deletedbyinference == false), each entity can be linked at most to 100 other objects
2020-04-04 14:03:43 +02:00
*
* <p>2) JoinRelationEntityByTargetJob: (phase 1): prepare tuples [relation - target entity] (R -
* T): for each entity type E_i map E_i as RelatedEntity T_i to simplify the model and extracting
* only the necessary information join (R.target = T_i.id) save the tuples (R_i, T_i) (phase 2):
* create the union of all the entity types E, hash by id read the tuples (R, T), hash by R.source
* join E.id = (R, T).source, where E becomes the Source Entity S save the tuples (S, R, T)
2020-04-04 14:03:43 +02:00
*
* <p>3) AdjacencyListBuilderJob: given the tuple (S - R - T) we need to group by S.id -> List [ R -
* T ], mapping the result as JoinedEntity
2020-04-04 14:03:43 +02:00
*
* <p>4) XmlConverterJob: convert the JoinedEntities as XML records
2020-04-04 14:03:43 +02:00
*/
public class CreateRelatedEntitiesJob_phase1 {
private static final Logger log =
LoggerFactory.getLogger(CreateRelatedEntitiesJob_phase1.class);
2020-04-04 14:03:43 +02:00
private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
public static void main(String[] args) throws Exception {
String jsonConfiguration =
IOUtils.toString(
PrepareRelationsJob.class.getResourceAsStream(
"/eu/dnetlib/dhp/oa/provision/input_params_related_entities_pahase1.json"));
2020-04-04 14:03:43 +02:00
final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration);
parser.parseArgument(args);
Boolean isSparkSessionManaged =
Optional.ofNullable(parser.get("isSparkSessionManaged"))
.map(Boolean::valueOf)
.orElse(Boolean.TRUE);
2020-04-04 14:03:43 +02:00
log.info("isSparkSessionManaged: {}", isSparkSessionManaged);
String inputRelationsPath = parser.get("inputRelationsPath");
log.info("inputRelationsPath: {}", inputRelationsPath);
String inputEntityPath = parser.get("inputEntityPath");
log.info("inputEntityPath: {}", inputEntityPath);
String outputPath = parser.get("outputPath");
log.info("outputPath: {}", outputPath);
String graphTableClassName = parser.get("graphTableClassName");
log.info("graphTableClassName: {}", graphTableClassName);
Class<? extends OafEntity> entityClazz =
(Class<? extends OafEntity>) Class.forName(graphTableClassName);
2020-04-04 14:03:43 +02:00
SparkConf conf = new SparkConf();
conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer");
conf.registerKryoClasses(ModelSupport.getOafModelClasses());
2020-04-04 14:03:43 +02:00
runWithSparkSession(
conf,
isSparkSessionManaged,
2020-04-04 14:03:43 +02:00
spark -> {
removeOutputDir(spark, outputPath);
joinRelationEntity(
spark, inputRelationsPath, inputEntityPath, entityClazz, outputPath);
2020-04-04 14:03:43 +02:00
});
}
private static <E extends OafEntity> void joinRelationEntity(
SparkSession spark,
String inputRelationsPath,
String inputEntityPath,
Class<E> entityClazz,
String outputPath) {
Dataset<Tuple2<String, SortableRelation>> relsByTarget =
readPathRelation(spark, inputRelationsPath)
.filter(
(FilterFunction<SortableRelation>)
value ->
value.getDataInfo().getDeletedbyinference()
== false)
.map(
(MapFunction<SortableRelation, Tuple2<String, SortableRelation>>)
r -> new Tuple2<>(r.getTarget(), r),
Encoders.tuple(
Encoders.STRING(), Encoders.kryo(SortableRelation.class)))
.cache();
Dataset<Tuple2<String, RelatedEntity>> entities =
readPathEntity(spark, inputEntityPath, entityClazz)
.map(
(MapFunction<E, RelatedEntity>)
value -> asRelatedEntity(value, entityClazz),
Encoders.bean(RelatedEntity.class))
.map(
(MapFunction<RelatedEntity, Tuple2<String, RelatedEntity>>)
e -> new Tuple2<>(e.getId(), e),
Encoders.tuple(
Encoders.STRING(), Encoders.kryo(RelatedEntity.class)))
.cache();
2020-04-04 14:03:43 +02:00
relsByTarget
.joinWith(entities, entities.col("_1").equalTo(relsByTarget.col("_1")), "inner")
.map(
(MapFunction<
Tuple2<
Tuple2<String, SortableRelation>,
Tuple2<String, RelatedEntity>>,
EntityRelEntity>)
t -> new EntityRelEntity(t._1()._2(), t._2()._2()),
2020-04-04 14:03:43 +02:00
Encoders.bean(EntityRelEntity.class))
.write()
2020-04-06 08:59:58 +02:00
.mode(SaveMode.Overwrite)
.parquet(outputPath + "/" + EntityType.fromClass(entityClazz));
2020-04-04 14:03:43 +02:00
}
private static <E extends OafEntity> Dataset<E> readPathEntity(
SparkSession spark, String inputEntityPath, Class<E> entityClazz) {
2020-04-04 14:03:43 +02:00
log.info("Reading Graph table from: {}", inputEntityPath);
return spark.read()
2020-04-04 14:03:43 +02:00
.textFile(inputEntityPath)
.map(
(MapFunction<String, E>)
value -> OBJECT_MAPPER.readValue(value, entityClazz),
Encoders.bean(entityClazz));
2020-04-04 14:03:43 +02:00
}
/**
* Reads a Dataset of eu.dnetlib.dhp.oa.provision.model.SortableRelation objects from a newline
* delimited json text file,
2020-04-04 14:03:43 +02:00
*
* @param spark
* @param relationPath
* @return the Dataset<SortableRelation> containing all the relationships
*/
private static Dataset<SortableRelation> readPathRelation(
SparkSession spark, final String relationPath) {
2020-04-04 14:03:43 +02:00
log.info("Reading relations from: {}", relationPath);
return spark.read().load(relationPath).as(Encoders.bean(SortableRelation.class));
2020-04-04 14:03:43 +02:00
}
private static void removeOutputDir(SparkSession spark, String path) {
HdfsSupport.remove(path, spark.sparkContext().hadoopConfiguration());
}
}