From dbfb9c19fe12181c61ef0435937b298a863ed555 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Thu, 21 May 2020 10:00:14 +0200 Subject: [PATCH] minor changes --- .../dhp/oa/provision/PrepareRelationsJob.java | 82 ++++++++++++------- .../dhp/oa/provision/oozie_app/workflow.xml | 9 +- 2 files changed, 59 insertions(+), 32 deletions(-) diff --git a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/PrepareRelationsJob.java b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/PrepareRelationsJob.java index 32a20d62c..72d68a389 100644 --- a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/PrepareRelationsJob.java +++ b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/PrepareRelationsJob.java @@ -17,7 +17,6 @@ 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.apache.spark.sql.sources.In; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -31,7 +30,6 @@ import eu.dnetlib.dhp.application.ArgumentApplicationParser; import eu.dnetlib.dhp.common.HdfsSupport; import eu.dnetlib.dhp.oa.provision.model.SortableRelation; import eu.dnetlib.dhp.oa.provision.utils.RelationPartitioner; -import scala.Int; import scala.Tuple2; /** @@ -112,26 +110,74 @@ public class PrepareRelationsJob { isSparkSessionManaged, spark -> { removeOutputDir(spark, outputPath); - prepareRelationsRDDFromPaths( + prepareRelationsRDD( spark, inputRelationsPath, outputPath, relationFilter, relPartitions, maxRelations); }); } - private static void prepareRelationsFromPaths( - SparkSession spark, String inputRelationsPath, String outputPath, Set relationFilter) { + /** + * Dataset based implementation that prepares the graph relations by limiting the number of outgoing links and + * filtering the relation types according to the given criteria. + * + * @param spark the spark session + * @param inputRelationsPath source path for the graph relations + * @param outputPath output path for the processed relations + * @param relationFilter set of relation filters applied to the `relClass` field + * @param maxRelations maximum number of allowed outgoing edges + */ + private static void prepareRelations( + SparkSession spark, String inputRelationsPath, String outputPath, Set relationFilter, + int maxRelations) { readPathRelation(spark, inputRelationsPath) .filter("dataInfo.deletedbyinference == false") + .filter((FilterFunction) rel -> !relationFilter.contains(rel.getRelClass())) .groupByKey( (MapFunction) value -> value.getSource(), Encoders.STRING()) .flatMapGroups( (FlatMapGroupsFunction) (key, values) -> Iterators - .limit(values, MAX_RELS), + .limit(values, maxRelations), Encoders.bean(SortableRelation.class)) .write() .mode(SaveMode.Overwrite) .parquet(outputPath); } + /** + * RDD based implementation that prepares the graph relations by limiting the number of outgoing links and filtering + * the relation types according to the given criteria. Moreover, outgoing links kept within the given limit are + * prioritized according to the weights indicated in eu.dnetlib.dhp.oa.provision.model.SortableRelation. + * + * @param spark the spark session + * @param inputRelationsPath source path for the graph relations + * @param outputPath output path for the processed relations + * @param relationFilter set of relation filters applied to the `relClass` field + * @param maxRelations maximum number of allowed outgoing edges + */ + // TODO work in progress + private static void prepareRelationsRDD( + SparkSession spark, String inputRelationsPath, String outputPath, Set relationFilter, int relPartitions, + int maxRelations) { + JavaRDD rels = readPathRelationRDD(spark, inputRelationsPath).repartition(relPartitions); + RelationPartitioner partitioner = new RelationPartitioner(rels.getNumPartitions()); + + // only consider those that are not virtually deleted + RDD d = rels + .filter(rel -> !rel.getDataInfo().getDeletedbyinference()) + .filter(rel -> !relationFilter.contains(rel.getRelClass())) + .mapToPair( + (PairFunction) rel -> new Tuple2<>(rel, rel)) + .groupByKey(partitioner) + .map(group -> Iterables.limit(group._2(), maxRelations)) + .flatMap(group -> group.iterator()) + .rdd(); + + spark + .createDataset(d, Encoders.bean(SortableRelation.class)) + .write() + .mode(SaveMode.Overwrite) + .parquet(outputPath); + } + /** * Reads a Dataset of eu.dnetlib.dhp.oa.provision.model.SortableRelation objects from a newline delimited json text * file, @@ -150,30 +196,6 @@ public class PrepareRelationsJob { Encoders.bean(SortableRelation.class)); } - // TODO work in progress - private static void prepareRelationsRDDFromPaths( - SparkSession spark, String inputRelationsPath, String outputPath, Set relationFilter, int relPartitions, - int maxRelations) { - JavaRDD rels = readPathRelationRDD(spark, inputRelationsPath).repartition(relPartitions); - - // only consider those that are not virtually deleted - RDD d = rels - .filter(rel -> !rel.getDataInfo().getDeletedbyinference()) - .filter(rel -> !relationFilter.contains(rel.getRelClass())) - .mapToPair( - (PairFunction) rel -> new Tuple2<>(rel, rel)) - .groupByKey(new RelationPartitioner(rels.getNumPartitions())) - .map(group -> Iterables.limit(group._2(), maxRelations)) - .flatMap(group -> group.iterator()) - .rdd(); - - spark - .createDataset(d, Encoders.bean(SortableRelation.class)) - .write() - .mode(SaveMode.Overwrite) - .parquet(outputPath); - } - private static JavaRDD readPathRelationRDD( SparkSession spark, final String inputPath) { JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext()); diff --git a/dhp-workflows/dhp-graph-provision/src/main/resources/eu/dnetlib/dhp/oa/provision/oozie_app/workflow.xml b/dhp-workflows/dhp-graph-provision/src/main/resources/eu/dnetlib/dhp/oa/provision/oozie_app/workflow.xml index 9600ccbaa..fd8f5ba89 100644 --- a/dhp-workflows/dhp-graph-provision/src/main/resources/eu/dnetlib/dhp/oa/provision/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-graph-provision/src/main/resources/eu/dnetlib/dhp/oa/provision/oozie_app/workflow.xml @@ -80,6 +80,11 @@ spark2EventLogDir spark 2.* event log dir location + + sparkNetworkTimeout + configures spark.network.timeout + + @@ -357,7 +362,7 @@ --inputGraphRootPath${inputGraphRootPath} --inputRelatedEntitiesPath${workingDir}/join_partial --outputPath${workingDir}/join_entities - --numPartitions12000 + --numPartitions24000 @@ -381,7 +386,7 @@ --conf spark.sql.shuffle.partitions=7680 --conf spark.network.timeout=${sparkNetworkTimeout} - --inputPath ${workingDir}/join_entities + --inputPath${workingDir}/join_entities --outputPath${workingDir}/joined