From ff4d6214f19292d90076d8ff6ec86a885861c1af Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Fri, 10 Jul 2020 10:06:41 +0200 Subject: [PATCH 1/6] experimenting with pruning of relations --- .../dhp/oa/provision/PrepareRelationsJob.java | 344 +++++++++--------- 1 file changed, 174 insertions(+), 170 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 eb63d4423..e1f7386e9 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 @@ -59,200 +59,204 @@ import scala.Tuple2; */ public class PrepareRelationsJob { - private static final Logger log = LoggerFactory.getLogger(PrepareRelationsJob.class); + private static final Logger log = LoggerFactory.getLogger(PrepareRelationsJob.class); - private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); - public static final int MAX_RELS = 100; + public static final int MAX_RELS = 100; - public static final int DEFAULT_NUM_PARTITIONS = 3000; + public static final int DEFAULT_NUM_PARTITIONS = 3000; - public static void main(String[] args) throws Exception { - String jsonConfiguration = IOUtils - .toString( - PrepareRelationsJob.class - .getResourceAsStream( - "/eu/dnetlib/dhp/oa/provision/input_params_prepare_relations.json")); - final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration); - parser.parseArgument(args); + public static void main(String[] args) throws Exception { + String jsonConfiguration = IOUtils + .toString( + PrepareRelationsJob.class + .getResourceAsStream( + "/eu/dnetlib/dhp/oa/provision/input_params_prepare_relations.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); + Boolean isSparkSessionManaged = Optional + .ofNullable(parser.get("isSparkSessionManaged")) + .map(Boolean::valueOf) + .orElse(Boolean.TRUE); + log.info("isSparkSessionManaged: {}", isSparkSessionManaged); - String inputRelationsPath = parser.get("inputRelationsPath"); - log.info("inputRelationsPath: {}", inputRelationsPath); + String inputRelationsPath = parser.get("inputRelationsPath"); + log.info("inputRelationsPath: {}", inputRelationsPath); - String outputPath = parser.get("outputPath"); - log.info("outputPath: {}", outputPath); + String outputPath = parser.get("outputPath"); + log.info("outputPath: {}", outputPath); - int relPartitions = Optional - .ofNullable(parser.get("relPartitions")) - .map(Integer::valueOf) - .orElse(DEFAULT_NUM_PARTITIONS); - log.info("relPartitions: {}", relPartitions); + int relPartitions = Optional + .ofNullable(parser.get("relPartitions")) + .map(Integer::valueOf) + .orElse(DEFAULT_NUM_PARTITIONS); + log.info("relPartitions: {}", relPartitions); - Set relationFilter = Optional - .ofNullable(parser.get("relationFilter")) - .map(s -> Sets.newHashSet(Splitter.on(",").split(s))) - .orElse(new HashSet<>()); - log.info("relationFilter: {}", relationFilter); + Set relationFilter = Optional + .ofNullable(parser.get("relationFilter")) + .map(s -> Sets.newHashSet(Splitter.on(",").split(s))) + .orElse(new HashSet<>()); + log.info("relationFilter: {}", relationFilter); - int maxRelations = Optional - .ofNullable(parser.get("maxRelations")) - .map(Integer::valueOf) - .orElse(MAX_RELS); - log.info("maxRelations: {}", maxRelations); + int maxRelations = Optional + .ofNullable(parser.get("maxRelations")) + .map(Integer::valueOf) + .orElse(MAX_RELS); + log.info("maxRelations: {}", maxRelations); - SparkConf conf = new SparkConf(); - conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer"); - conf.registerKryoClasses(ProvisionModelSupport.getModelClasses()); + SparkConf conf = new SparkConf(); + conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer"); + conf.registerKryoClasses(ProvisionModelSupport.getModelClasses()); - runWithSparkSession( - conf, - isSparkSessionManaged, - spark -> { - removeOutputDir(spark, outputPath); - prepareRelationsRDD( - spark, inputRelationsPath, outputPath, relationFilter, maxRelations, relPartitions); - }); + runWithSparkSession( + conf, + isSparkSessionManaged, + spark -> { + removeOutputDir(spark, outputPath); + prepareRelationsRDD( + spark, inputRelationsPath, outputPath, relationFilter, maxRelations, relPartitions); + }); + } + + /** + * 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 + * @param relPartitions number of partitions for the output RDD + */ + private static void prepareRelationsRDD(SparkSession spark, String inputRelationsPath, String outputPath, + Set relationFilter, int maxRelations, int relPartitions) { + + JavaRDD rels = readPathRelationRDD(spark, inputRelationsPath); + + JavaRDD pruned = pruneRels( + pruneRels(rels, relationFilter, maxRelations, relPartitions, (Function) r -> r.getSource()), + relationFilter, maxRelations, relPartitions, (Function) r -> r.getTarget()); + spark + .createDataset(pruned.rdd(), Encoders.bean(Relation.class)) + .repartition(relPartitions) + .write() + .mode(SaveMode.Overwrite) + .parquet(outputPath); + } + + private static JavaRDD pruneRels(JavaRDD rels, Set relationFilter, int maxRelations, int relPartitions, Function idFn) { + return rels + .filter(rel -> rel.getDataInfo().getDeletedbyinference() == false) + .filter(rel -> relationFilter.contains(rel.getRelClass()) == false) + .mapToPair(r -> new Tuple2<>(SortableRelationKey.create(r, idFn.call(r)), r)) + .repartitionAndSortWithinPartitions(new RelationPartitioner(relPartitions)) + .groupBy(Tuple2::_1) + .map(Tuple2::_2) + .map(t -> Iterables.limit(t, maxRelations)) + .flatMap(Iterable::iterator).map(Tuple2::_2); } - /** - * 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 - * @param relPartitions number of partitions for the output RDD - */ - private static void prepareRelationsRDD(SparkSession spark, String inputRelationsPath, String outputPath, - Set relationFilter, int maxRelations, int relPartitions) { + // experimental + private static void prepareRelationsDataset( + SparkSession spark, String inputRelationsPath, String outputPath, Set relationFilter, int maxRelations, + int relPartitions) { + spark + .read() + .textFile(inputRelationsPath) + .repartition(relPartitions) + .map( + (MapFunction) s -> OBJECT_MAPPER.readValue(s, Relation.class), + Encoders.kryo(Relation.class)) + .filter((FilterFunction) rel -> rel.getDataInfo().getDeletedbyinference() == false) + .filter((FilterFunction) rel -> relationFilter.contains(rel.getRelClass()) == false) + .groupByKey( + (MapFunction) Relation::getSource, + Encoders.STRING()) + .agg(new RelationAggregator(maxRelations).toColumn()) + .flatMap( + (FlatMapFunction, Relation>) t -> Iterables + .limit(t._2().getRelations(), maxRelations) + .iterator(), + Encoders.bean(Relation.class)) + .repartition(relPartitions) + .write() + .mode(SaveMode.Overwrite) + .parquet(outputPath); + } - // group by SOURCE and apply limit - RDD bySource = readPathRelationRDD(spark, inputRelationsPath) - .filter(rel -> rel.getDataInfo().getDeletedbyinference() == false) - .filter(rel -> relationFilter.contains(rel.getRelClass()) == false) - .mapToPair(r -> new Tuple2<>(SortableRelationKey.create(r, r.getSource()), r)) - .repartitionAndSortWithinPartitions(new RelationPartitioner(relPartitions)) - .groupBy(Tuple2::_1) - .map(Tuple2::_2) - .map(t -> Iterables.limit(t, maxRelations)) - .flatMap(Iterable::iterator) - .map(Tuple2::_2) - .rdd(); + public static class RelationAggregator + extends Aggregator { - spark - .createDataset(bySource, Encoders.bean(Relation.class)) - .repartition(relPartitions) - .write() - .mode(SaveMode.Overwrite) - .parquet(outputPath); - } + private int maxRelations; - // experimental - private static void prepareRelationsDataset( - SparkSession spark, String inputRelationsPath, String outputPath, Set relationFilter, int maxRelations, - int relPartitions) { - spark - .read() - .textFile(inputRelationsPath) - .repartition(relPartitions) - .map( - (MapFunction) s -> OBJECT_MAPPER.readValue(s, Relation.class), - Encoders.kryo(Relation.class)) - .filter((FilterFunction) rel -> rel.getDataInfo().getDeletedbyinference() == false) - .filter((FilterFunction) rel -> relationFilter.contains(rel.getRelClass()) == false) - .groupByKey( - (MapFunction) Relation::getSource, - Encoders.STRING()) - .agg(new RelationAggregator(maxRelations).toColumn()) - .flatMap( - (FlatMapFunction, Relation>) t -> Iterables - .limit(t._2().getRelations(), maxRelations) - .iterator(), - Encoders.bean(Relation.class)) - .repartition(relPartitions) - .write() - .mode(SaveMode.Overwrite) - .parquet(outputPath); - } + public RelationAggregator(int maxRelations) { + this.maxRelations = maxRelations; + } - public static class RelationAggregator - extends Aggregator { + @Override + public RelationList zero() { + return new RelationList(); + } - private int maxRelations; + @Override + public RelationList reduce(RelationList b, Relation a) { + b.getRelations().add(a); + return getSortableRelationList(b); + } - public RelationAggregator(int maxRelations) { - this.maxRelations = maxRelations; - } + @Override + public RelationList merge(RelationList b1, RelationList b2) { + b1.getRelations().addAll(b2.getRelations()); + return getSortableRelationList(b1); + } - @Override - public RelationList zero() { - return new RelationList(); - } + @Override + public RelationList finish(RelationList r) { + return getSortableRelationList(r); + } - @Override - public RelationList reduce(RelationList b, Relation a) { - b.getRelations().add(a); - return getSortableRelationList(b); - } + private RelationList getSortableRelationList(RelationList b1) { + RelationList sr = new RelationList(); + sr + .setRelations( + b1 + .getRelations() + .stream() + .limit(maxRelations) + .collect(Collectors.toCollection(() -> new PriorityQueue<>(new RelationComparator())))); + return sr; + } - @Override - public RelationList merge(RelationList b1, RelationList b2) { - b1.getRelations().addAll(b2.getRelations()); - return getSortableRelationList(b1); - } + @Override + public Encoder bufferEncoder() { + return Encoders.kryo(RelationList.class); + } - @Override - public RelationList finish(RelationList r) { - return getSortableRelationList(r); - } + @Override + public Encoder outputEncoder() { + return Encoders.kryo(RelationList.class); + } + } - private RelationList getSortableRelationList(RelationList b1) { - RelationList sr = new RelationList(); - sr - .setRelations( - b1 - .getRelations() - .stream() - .limit(maxRelations) - .collect(Collectors.toCollection(() -> new PriorityQueue<>(new RelationComparator())))); - return sr; - } + /** + * Reads a JavaRDD of eu.dnetlib.dhp.oa.provision.model.SortableRelation objects from a newline delimited json text + * file, + * + * @param spark + * @param inputPath + * @return the JavaRDD containing all the relationships + */ + private static JavaRDD readPathRelationRDD( + SparkSession spark, final String inputPath) { + JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext()); + return sc.textFile(inputPath).map(s -> OBJECT_MAPPER.readValue(s, Relation.class)); + } - @Override - public Encoder bufferEncoder() { - return Encoders.kryo(RelationList.class); - } - - @Override - public Encoder outputEncoder() { - return Encoders.kryo(RelationList.class); - } - } - - /** - * Reads a JavaRDD of eu.dnetlib.dhp.oa.provision.model.SortableRelation objects from a newline delimited json text - * file, - * - * @param spark - * @param inputPath - * @return the JavaRDD containing all the relationships - */ - private static JavaRDD readPathRelationRDD( - SparkSession spark, final String inputPath) { - JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext()); - return sc.textFile(inputPath).map(s -> OBJECT_MAPPER.readValue(s, Relation.class)); - } - - private static void removeOutputDir(SparkSession spark, String path) { - HdfsSupport.remove(path, spark.sparkContext().hadoopConfiguration()); - } + private static void removeOutputDir(SparkSession spark, String path) { + HdfsSupport.remove(path, spark.sparkContext().hadoopConfiguration()); + } } From b21866a2da7be44106713c3f3982e3ee646d22d4 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Fri, 10 Jul 2020 13:59:48 +0200 Subject: [PATCH 2/6] allow to set different to relations cut points by source and by target; adjusted weight assigned to relationship types --- .../dhp/oa/provision/PrepareRelationsJob.java | 358 +++++++++--------- .../provision/model/SortableRelationKey.java | 22 +- .../input_params_prepare_relations.json | 13 +- 3 files changed, 206 insertions(+), 187 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 e1f7386e9..da0a81021 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 @@ -59,204 +59,216 @@ import scala.Tuple2; */ public class PrepareRelationsJob { - private static final Logger log = LoggerFactory.getLogger(PrepareRelationsJob.class); + private static final Logger log = LoggerFactory.getLogger(PrepareRelationsJob.class); - private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); - public static final int MAX_RELS = 100; + public static final int MAX_RELS = 100; - public static final int DEFAULT_NUM_PARTITIONS = 3000; + public static final int DEFAULT_NUM_PARTITIONS = 3000; - public static void main(String[] args) throws Exception { - String jsonConfiguration = IOUtils - .toString( - PrepareRelationsJob.class - .getResourceAsStream( - "/eu/dnetlib/dhp/oa/provision/input_params_prepare_relations.json")); - final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration); - parser.parseArgument(args); + public static void main(String[] args) throws Exception { + String jsonConfiguration = IOUtils + .toString( + PrepareRelationsJob.class + .getResourceAsStream( + "/eu/dnetlib/dhp/oa/provision/input_params_prepare_relations.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); + Boolean isSparkSessionManaged = Optional + .ofNullable(parser.get("isSparkSessionManaged")) + .map(Boolean::valueOf) + .orElse(Boolean.TRUE); + log.info("isSparkSessionManaged: {}", isSparkSessionManaged); - String inputRelationsPath = parser.get("inputRelationsPath"); - log.info("inputRelationsPath: {}", inputRelationsPath); + String inputRelationsPath = parser.get("inputRelationsPath"); + log.info("inputRelationsPath: {}", inputRelationsPath); - String outputPath = parser.get("outputPath"); - log.info("outputPath: {}", outputPath); + String outputPath = parser.get("outputPath"); + log.info("outputPath: {}", outputPath); - int relPartitions = Optional - .ofNullable(parser.get("relPartitions")) - .map(Integer::valueOf) - .orElse(DEFAULT_NUM_PARTITIONS); - log.info("relPartitions: {}", relPartitions); + int relPartitions = Optional + .ofNullable(parser.get("relPartitions")) + .map(Integer::valueOf) + .orElse(DEFAULT_NUM_PARTITIONS); + log.info("relPartitions: {}", relPartitions); - Set relationFilter = Optional - .ofNullable(parser.get("relationFilter")) - .map(s -> Sets.newHashSet(Splitter.on(",").split(s))) - .orElse(new HashSet<>()); - log.info("relationFilter: {}", relationFilter); + Set relationFilter = Optional + .ofNullable(parser.get("relationFilter")) + .map(s -> Sets.newHashSet(Splitter.on(",").split(s))) + .orElse(new HashSet<>()); + log.info("relationFilter: {}", relationFilter); - int maxRelations = Optional - .ofNullable(parser.get("maxRelations")) - .map(Integer::valueOf) - .orElse(MAX_RELS); - log.info("maxRelations: {}", maxRelations); + int sourceMaxRelations = Optional + .ofNullable(parser.get("sourceMaxRelations")) + .map(Integer::valueOf) + .orElse(MAX_RELS); + log.info("sourceMaxRelations: {}", sourceMaxRelations); - SparkConf conf = new SparkConf(); - conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer"); - conf.registerKryoClasses(ProvisionModelSupport.getModelClasses()); + int targetMaxRelations = Optional + .ofNullable(parser.get("targetMaxRelations")) + .map(Integer::valueOf) + .orElse(MAX_RELS); + log.info("targetMaxRelations: {}", targetMaxRelations); - runWithSparkSession( - conf, - isSparkSessionManaged, - spark -> { - removeOutputDir(spark, outputPath); - prepareRelationsRDD( - spark, inputRelationsPath, outputPath, relationFilter, maxRelations, relPartitions); - }); - } + SparkConf conf = new SparkConf(); + conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer"); + conf.registerKryoClasses(ProvisionModelSupport.getModelClasses()); - /** - * 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 - * @param relPartitions number of partitions for the output RDD - */ - private static void prepareRelationsRDD(SparkSession spark, String inputRelationsPath, String outputPath, - Set relationFilter, int maxRelations, int relPartitions) { - - JavaRDD rels = readPathRelationRDD(spark, inputRelationsPath); - - JavaRDD pruned = pruneRels( - pruneRels(rels, relationFilter, maxRelations, relPartitions, (Function) r -> r.getSource()), - relationFilter, maxRelations, relPartitions, (Function) r -> r.getTarget()); - spark - .createDataset(pruned.rdd(), Encoders.bean(Relation.class)) - .repartition(relPartitions) - .write() - .mode(SaveMode.Overwrite) - .parquet(outputPath); - } - - private static JavaRDD pruneRels(JavaRDD rels, Set relationFilter, int maxRelations, int relPartitions, Function idFn) { - return rels - .filter(rel -> rel.getDataInfo().getDeletedbyinference() == false) - .filter(rel -> relationFilter.contains(rel.getRelClass()) == false) - .mapToPair(r -> new Tuple2<>(SortableRelationKey.create(r, idFn.call(r)), r)) - .repartitionAndSortWithinPartitions(new RelationPartitioner(relPartitions)) - .groupBy(Tuple2::_1) - .map(Tuple2::_2) - .map(t -> Iterables.limit(t, maxRelations)) - .flatMap(Iterable::iterator).map(Tuple2::_2); + runWithSparkSession( + conf, + isSparkSessionManaged, + spark -> { + removeOutputDir(spark, outputPath); + prepareRelationsRDD( + spark, inputRelationsPath, outputPath, relationFilter, sourceMaxRelations, targetMaxRelations, + relPartitions); + }); } - // experimental - private static void prepareRelationsDataset( - SparkSession spark, String inputRelationsPath, String outputPath, Set relationFilter, int maxRelations, - int relPartitions) { - spark - .read() - .textFile(inputRelationsPath) - .repartition(relPartitions) - .map( - (MapFunction) s -> OBJECT_MAPPER.readValue(s, Relation.class), - Encoders.kryo(Relation.class)) - .filter((FilterFunction) rel -> rel.getDataInfo().getDeletedbyinference() == false) - .filter((FilterFunction) rel -> relationFilter.contains(rel.getRelClass()) == false) - .groupByKey( - (MapFunction) Relation::getSource, - Encoders.STRING()) - .agg(new RelationAggregator(maxRelations).toColumn()) - .flatMap( - (FlatMapFunction, Relation>) t -> Iterables - .limit(t._2().getRelations(), maxRelations) - .iterator(), - Encoders.bean(Relation.class)) - .repartition(relPartitions) - .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 sourceMaxRelations maximum number of allowed outgoing edges grouping by relation.source + * @param targetMaxRelations maximum number of allowed outgoing edges grouping by relation.target + * @param relPartitions number of partitions for the output RDD + */ + private static void prepareRelationsRDD(SparkSession spark, String inputRelationsPath, String outputPath, + Set relationFilter, int sourceMaxRelations, int targetMaxRelations, int relPartitions) { - public static class RelationAggregator - extends Aggregator { + JavaRDD rels = readPathRelationRDD(spark, inputRelationsPath) + .filter(rel -> rel.getDataInfo().getDeletedbyinference() == false) + .filter(rel -> relationFilter.contains(rel.getRelClass()) == false); - private int maxRelations; + JavaRDD pruned = pruneRels( + pruneRels( + rels, + sourceMaxRelations, relPartitions, (Function) r -> r.getSource()), + targetMaxRelations, relPartitions, (Function) r -> r.getTarget()); + spark + .createDataset(pruned.rdd(), Encoders.bean(Relation.class)) + .repartition(relPartitions) + .write() + .mode(SaveMode.Overwrite) + .parquet(outputPath); + } - public RelationAggregator(int maxRelations) { - this.maxRelations = maxRelations; - } + private static JavaRDD pruneRels(JavaRDD rels, int maxRelations, + int relPartitions, Function idFn) { + return rels + .mapToPair(r -> new Tuple2<>(SortableRelationKey.create(r, idFn.call(r)), r)) + .repartitionAndSortWithinPartitions(new RelationPartitioner(relPartitions)) + .groupBy(Tuple2::_1) + .map(Tuple2::_2) + .map(t -> Iterables.limit(t, maxRelations)) + .flatMap(Iterable::iterator) + .map(Tuple2::_2); + } - @Override - public RelationList zero() { - return new RelationList(); - } + // experimental + private static void prepareRelationsDataset( + SparkSession spark, String inputRelationsPath, String outputPath, Set relationFilter, int maxRelations, + int relPartitions) { + spark + .read() + .textFile(inputRelationsPath) + .repartition(relPartitions) + .map( + (MapFunction) s -> OBJECT_MAPPER.readValue(s, Relation.class), + Encoders.kryo(Relation.class)) + .filter((FilterFunction) rel -> rel.getDataInfo().getDeletedbyinference() == false) + .filter((FilterFunction) rel -> relationFilter.contains(rel.getRelClass()) == false) + .groupByKey( + (MapFunction) Relation::getSource, + Encoders.STRING()) + .agg(new RelationAggregator(maxRelations).toColumn()) + .flatMap( + (FlatMapFunction, Relation>) t -> Iterables + .limit(t._2().getRelations(), maxRelations) + .iterator(), + Encoders.bean(Relation.class)) + .repartition(relPartitions) + .write() + .mode(SaveMode.Overwrite) + .parquet(outputPath); + } - @Override - public RelationList reduce(RelationList b, Relation a) { - b.getRelations().add(a); - return getSortableRelationList(b); - } + public static class RelationAggregator + extends Aggregator { - @Override - public RelationList merge(RelationList b1, RelationList b2) { - b1.getRelations().addAll(b2.getRelations()); - return getSortableRelationList(b1); - } + private int maxRelations; - @Override - public RelationList finish(RelationList r) { - return getSortableRelationList(r); - } + public RelationAggregator(int maxRelations) { + this.maxRelations = maxRelations; + } - private RelationList getSortableRelationList(RelationList b1) { - RelationList sr = new RelationList(); - sr - .setRelations( - b1 - .getRelations() - .stream() - .limit(maxRelations) - .collect(Collectors.toCollection(() -> new PriorityQueue<>(new RelationComparator())))); - return sr; - } + @Override + public RelationList zero() { + return new RelationList(); + } - @Override - public Encoder bufferEncoder() { - return Encoders.kryo(RelationList.class); - } + @Override + public RelationList reduce(RelationList b, Relation a) { + b.getRelations().add(a); + return getSortableRelationList(b); + } - @Override - public Encoder outputEncoder() { - return Encoders.kryo(RelationList.class); - } - } + @Override + public RelationList merge(RelationList b1, RelationList b2) { + b1.getRelations().addAll(b2.getRelations()); + return getSortableRelationList(b1); + } - /** - * Reads a JavaRDD of eu.dnetlib.dhp.oa.provision.model.SortableRelation objects from a newline delimited json text - * file, - * - * @param spark - * @param inputPath - * @return the JavaRDD containing all the relationships - */ - private static JavaRDD readPathRelationRDD( - SparkSession spark, final String inputPath) { - JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext()); - return sc.textFile(inputPath).map(s -> OBJECT_MAPPER.readValue(s, Relation.class)); - } + @Override + public RelationList finish(RelationList r) { + return getSortableRelationList(r); + } - private static void removeOutputDir(SparkSession spark, String path) { - HdfsSupport.remove(path, spark.sparkContext().hadoopConfiguration()); - } + private RelationList getSortableRelationList(RelationList b1) { + RelationList sr = new RelationList(); + sr + .setRelations( + b1 + .getRelations() + .stream() + .limit(maxRelations) + .collect(Collectors.toCollection(() -> new PriorityQueue<>(new RelationComparator())))); + return sr; + } + + @Override + public Encoder bufferEncoder() { + return Encoders.kryo(RelationList.class); + } + + @Override + public Encoder outputEncoder() { + return Encoders.kryo(RelationList.class); + } + } + + /** + * Reads a JavaRDD of eu.dnetlib.dhp.oa.provision.model.SortableRelation objects from a newline delimited json text + * file, + * + * @param spark + * @param inputPath + * @return the JavaRDD containing all the relationships + */ + private static JavaRDD readPathRelationRDD( + SparkSession spark, final String inputPath) { + JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext()); + return sc.textFile(inputPath).map(s -> OBJECT_MAPPER.readValue(s, Relation.class)); + } + + private static void removeOutputDir(SparkSession spark, String path) { + HdfsSupport.remove(path, spark.sparkContext().hadoopConfiguration()); + } } diff --git a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/model/SortableRelationKey.java b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/model/SortableRelationKey.java index bf7f9330d..bd7b4d78e 100644 --- a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/model/SortableRelationKey.java +++ b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/model/SortableRelationKey.java @@ -16,18 +16,18 @@ public class SortableRelationKey implements Comparable, Ser private static final Map weights = Maps.newHashMap(); static { - weights.put("outcome", 0); - weights.put("supplement", 1); - weights.put("review", 2); - weights.put("citation", 3); - weights.put("affiliation", 4); - weights.put("relationship", 5); - weights.put("publicationDataset", 6); - weights.put("similarity", 7); + weights.put("participation", 0); - weights.put("provision", 8); - weights.put("participation", 9); - weights.put("dedup", 10); + weights.put("outcome", 1); + weights.put("affiliation", 2); + weights.put("dedup", 3); + weights.put("publicationDataset", 4); + weights.put("citation", 5); + weights.put("supplement", 6); + weights.put("review", 7); + weights.put("relationship", 8); + weights.put("provision", 9); + weights.put("similarity", 10); } private static final long serialVersionUID = 3232323; diff --git a/dhp-workflows/dhp-graph-provision/src/main/resources/eu/dnetlib/dhp/oa/provision/input_params_prepare_relations.json b/dhp-workflows/dhp-graph-provision/src/main/resources/eu/dnetlib/dhp/oa/provision/input_params_prepare_relations.json index 71b2becc4..33fa1dc8d 100644 --- a/dhp-workflows/dhp-graph-provision/src/main/resources/eu/dnetlib/dhp/oa/provision/input_params_prepare_relations.json +++ b/dhp-workflows/dhp-graph-provision/src/main/resources/eu/dnetlib/dhp/oa/provision/input_params_prepare_relations.json @@ -30,9 +30,16 @@ "paramRequired": false }, { - "paramName": "mr", - "paramLongName": "maxRelations", - "paramDescription": "maximum number of relations allowed for a each entity", + "paramName": "smr", + "paramLongName": "sourceMaxRelations", + "paramDescription": "maximum number of relations allowed for a each entity grouping by source", + "paramRequired": false + }, + { + "paramName": "tmr", + "paramLongName": "targetMaxRelations", + "paramDescription": "maximum number of relations allowed for a each entity grouping by target", "paramRequired": false } + ] From 4c3836f62e3358d173163aef8de11ead52fcc707 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Fri, 10 Jul 2020 19:00:44 +0200 Subject: [PATCH 3/6] materialize the related entities before joining them --- .../CreateRelatedEntitiesJob_phase1.java | 25 ++++++++++++++++--- .../CreateRelatedEntitiesJob_phase2.java | 17 +++++-------- .../dhp/oa/provision/ProvisionConstants.java | 14 +++++++++++ 3 files changed, 42 insertions(+), 14 deletions(-) create mode 100644 dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/ProvisionConstants.java diff --git a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/CreateRelatedEntitiesJob_phase1.java b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/CreateRelatedEntitiesJob_phase1.java index 80b800017..57dca7bb1 100644 --- a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/CreateRelatedEntitiesJob_phase1.java +++ b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/CreateRelatedEntitiesJob_phase1.java @@ -9,6 +9,7 @@ import java.util.Optional; import java.util.stream.Collectors; import org.apache.commons.io.IOUtils; +import org.apache.commons.lang3.StringUtils; import org.apache.spark.SparkConf; import org.apache.spark.api.java.function.MapFunction; import org.apache.spark.sql.Dataset; @@ -115,11 +116,21 @@ public class CreateRelatedEntitiesJob_phase1 { Encoders.tuple(Encoders.STRING(), Encoders.kryo(Relation.class))) .cache(); - Dataset> entities = readPathEntity(spark, inputEntityPath, clazz) + final String relatedEntityPath = outputPath + "_relatedEntity"; + readPathEntity(spark, inputEntityPath, clazz) .filter("dataInfo.invisible == false") .map( (MapFunction) value -> asRelatedEntity(value, clazz), Encoders.kryo(RelatedEntity.class)) + .repartition(5000) + .write() + .mode(SaveMode.Overwrite) + .parquet(relatedEntityPath); + + Dataset> entities = spark + .read() + .load(relatedEntityPath) + .as(Encoders.kryo(RelatedEntity.class)) .map( (MapFunction>) e -> new Tuple2<>(e.getId(), e), Encoders.tuple(Encoders.STRING(), Encoders.kryo(RelatedEntity.class))) @@ -165,13 +176,21 @@ public class CreateRelatedEntitiesJob_phase1 { Result result = (Result) entity; if (result.getTitle() != null && !result.getTitle().isEmpty()) { - re.setTitle(result.getTitle().stream().findFirst().get()); + final StructuredProperty title = result.getTitle().stream().findFirst().get(); + title.setValue(StringUtils.left(title.getValue(), ProvisionConstants.MAX_TITLE_LENGTH)); + re.setTitle(title); } re.setDateofacceptance(getValue(result.getDateofacceptance())); re.setPublisher(getValue(result.getPublisher())); re.setResulttype(result.getResulttype()); - re.setInstances(result.getInstance()); + re + .setInstances( + result + .getInstance() + .stream() + .limit(ProvisionConstants.MAX_INSTANCES) + .collect(Collectors.toList())); // TODO still to be mapped // re.setCodeRepositoryUrl(j.read("$.coderepositoryurl")); diff --git a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/CreateRelatedEntitiesJob_phase2.java b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/CreateRelatedEntitiesJob_phase2.java index bfcc648a3..7e175121e 100644 --- a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/CreateRelatedEntitiesJob_phase2.java +++ b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/CreateRelatedEntitiesJob_phase2.java @@ -61,12 +61,6 @@ public class CreateRelatedEntitiesJob_phase2 { private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); - private static final int MAX_EXTERNAL_ENTITIES = 50; - private static final int MAX_AUTHORS = 200; - private static final int MAX_AUTHOR_FULLNAME_LENGTH = 1000; - private static final int MAX_TITLE_LENGTH = 5000; - private static final int MAX_ABSTRACT_LENGTH = 100000; - public static void main(String[] args) throws Exception { String jsonConfiguration = IOUtils @@ -246,15 +240,15 @@ public class CreateRelatedEntitiesJob_phase2 { List refs = r .getExternalReference() .stream() - .limit(MAX_EXTERNAL_ENTITIES) + .limit(ProvisionConstants.MAX_EXTERNAL_ENTITIES) .collect(Collectors.toList()); r.setExternalReference(refs); } if (r.getAuthor() != null) { List authors = Lists.newArrayList(); for (Author a : r.getAuthor()) { - a.setFullname(StringUtils.left(a.getFullname(), MAX_AUTHOR_FULLNAME_LENGTH)); - if (authors.size() < MAX_AUTHORS || hasORCID(a)) { + a.setFullname(StringUtils.left(a.getFullname(), ProvisionConstants.MAX_AUTHOR_FULLNAME_LENGTH)); + if (authors.size() < ProvisionConstants.MAX_AUTHORS || hasORCID(a)) { authors.add(a); } } @@ -266,7 +260,7 @@ public class CreateRelatedEntitiesJob_phase2 { .stream() .filter(Objects::nonNull) .map(d -> { - d.setValue(StringUtils.left(d.getValue(), MAX_ABSTRACT_LENGTH)); + d.setValue(StringUtils.left(d.getValue(), ProvisionConstants.MAX_ABSTRACT_LENGTH)); return d; }) .collect(Collectors.toList()); @@ -278,9 +272,10 @@ public class CreateRelatedEntitiesJob_phase2 { .stream() .filter(Objects::nonNull) .map(t -> { - t.setValue(StringUtils.left(t.getValue(), MAX_TITLE_LENGTH)); + t.setValue(StringUtils.left(t.getValue(), ProvisionConstants.MAX_TITLE_LENGTH)); return t; }) + .limit(ProvisionConstants.MAX_TITLES) .collect(Collectors.toList()); r.setTitle(titles); } diff --git a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/ProvisionConstants.java b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/ProvisionConstants.java new file mode 100644 index 000000000..9bc3706cd --- /dev/null +++ b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/ProvisionConstants.java @@ -0,0 +1,14 @@ + +package eu.dnetlib.dhp.oa.provision; + +public class ProvisionConstants { + + public static final int MAX_EXTERNAL_ENTITIES = 50; + public static final int MAX_AUTHORS = 200; + public static final int MAX_AUTHOR_FULLNAME_LENGTH = 1000; + public static final int MAX_TITLE_LENGTH = 5000; + public static final int MAX_TITLES = 10; + public static final int MAX_ABSTRACT_LENGTH = 100000; + public static final int MAX_INSTANCES = 10; + +} From cc77446dc4d092503bff388994a005e17e61dcfc Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Fri, 10 Jul 2020 19:01:50 +0200 Subject: [PATCH 4/6] added dbSchema parameter to the raw_db workflow --- .../eu/dnetlib/dhp/oa/graph/raw_db/oozie_app/workflow.xml | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/raw_db/oozie_app/workflow.xml b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/raw_db/oozie_app/workflow.xml index 575f9229e..eea8d0a5a 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/raw_db/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/raw_db/oozie_app/workflow.xml @@ -16,6 +16,11 @@ postgresPassword the password postgres + + dbSchema + beta + the database schema according to the D-Net infrastructure (beta or production) + isLookupUrl the address of the lookUp service @@ -93,6 +98,7 @@ --postgresUser${postgresUser} --postgresPassword${postgresPassword} --isLookupUrl${isLookupUrl} + --dbschema${dbSchema} @@ -109,6 +115,7 @@ --postgresUser${postgresUser} --postgresPassword${postgresPassword} --isLookupUrl${isLookupUrl} + --dbschema${dbSchema} --actionclaims From 06c1913062541062b4b5ed49044870acc1d3bd56 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Fri, 10 Jul 2020 19:03:33 +0200 Subject: [PATCH 5/6] added different limits for grouping by source and by target, incremented spark.sql.shuffle.partitions for the join operations --- .../dhp/oa/provision/oozie_app/workflow.xml | 40 ++++++++++--------- 1 file changed, 22 insertions(+), 18 deletions(-) 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 faa81ad64..32bf7ce83 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 @@ -18,8 +18,12 @@ filter applied reading relations (by relClass) - maxRelations - maximum number of relations allowed for a each entity + sourceMaxRelations + maximum number of relations allowed for a each entity grouping by source + + + targetMaxRelations + maximum number of relations allowed for a each entity grouping by target otherDsTypeId @@ -133,7 +137,8 @@ --inputRelationsPath${inputGraphRootPath}/relation --outputPath${workingDir}/relation - --maxRelations${maxRelations} + --sourceMaxRelations${sourceMaxRelations} + --targetMaxRelations${targetMaxRelations} --relationFilter${relationFilter} --relPartitions5000 @@ -166,7 +171,7 @@ --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} - --conf spark.sql.shuffle.partitions=7680 + --conf spark.sql.shuffle.partitions=15000 --conf spark.network.timeout=${sparkNetworkTimeout} --inputRelationsPath${workingDir}/relation @@ -193,7 +198,7 @@ --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} - --conf spark.sql.shuffle.partitions=3840 + --conf spark.sql.shuffle.partitions=15000 --conf spark.network.timeout=${sparkNetworkTimeout} --inputRelationsPath${workingDir}/relation @@ -220,7 +225,7 @@ --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} - --conf spark.sql.shuffle.partitions=3840 + --conf spark.sql.shuffle.partitions=10000 --conf spark.network.timeout=${sparkNetworkTimeout} --inputRelationsPath${workingDir}/relation @@ -247,7 +252,7 @@ --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} - --conf spark.sql.shuffle.partitions=3840 + --conf spark.sql.shuffle.partitions=5000 --conf spark.network.timeout=${sparkNetworkTimeout} --inputRelationsPath${workingDir}/relation @@ -274,7 +279,7 @@ --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} - --conf spark.sql.shuffle.partitions=3840 + --conf spark.sql.shuffle.partitions=5000 --conf spark.network.timeout=${sparkNetworkTimeout} --inputRelationsPath${workingDir}/relation @@ -301,7 +306,7 @@ --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} - --conf spark.sql.shuffle.partitions=3840 + --conf spark.sql.shuffle.partitions=5000 --conf spark.network.timeout=${sparkNetworkTimeout} --inputRelationsPath${workingDir}/relation @@ -328,7 +333,7 @@ --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} - --conf spark.sql.shuffle.partitions=3840 + --conf spark.sql.shuffle.partitions=5000 --conf spark.network.timeout=${sparkNetworkTimeout} --inputRelationsPath${workingDir}/relation @@ -367,7 +372,7 @@ --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} - --conf spark.sql.shuffle.partitions=15360 + --conf spark.sql.shuffle.partitions=15000 --conf spark.network.timeout=${sparkNetworkTimeout} --inputEntityPath${inputGraphRootPath}/publication @@ -395,7 +400,7 @@ --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} - --conf spark.sql.shuffle.partitions=7680 + --conf spark.sql.shuffle.partitions=10000 --conf spark.network.timeout=${sparkNetworkTimeout} --inputEntityPath${inputGraphRootPath}/dataset @@ -423,7 +428,7 @@ --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} - --conf spark.sql.shuffle.partitions=7680 + --conf spark.sql.shuffle.partitions=10000 --conf spark.network.timeout=${sparkNetworkTimeout} --inputEntityPath${inputGraphRootPath}/otherresearchproduct @@ -451,7 +456,7 @@ --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} - --conf spark.sql.shuffle.partitions=3840 + --conf spark.sql.shuffle.partitions=5000 --conf spark.network.timeout=${sparkNetworkTimeout} --inputEntityPath${inputGraphRootPath}/software @@ -479,7 +484,7 @@ --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} - --conf spark.sql.shuffle.partitions=7680 + --conf spark.sql.shuffle.partitions=8000 --conf spark.network.timeout=${sparkNetworkTimeout} --inputEntityPath${inputGraphRootPath}/datasource @@ -507,7 +512,7 @@ --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} - --conf spark.sql.shuffle.partitions=7680 + --conf spark.sql.shuffle.partitions=10000 --conf spark.network.timeout=${sparkNetworkTimeout} --inputEntityPath${inputGraphRootPath}/organization @@ -535,7 +540,7 @@ --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} - --conf spark.sql.shuffle.partitions=3840 + --conf spark.sql.shuffle.partitions=5000 --conf spark.network.timeout=${sparkNetworkTimeout} --inputEntityPath${inputGraphRootPath}/project @@ -607,5 +612,4 @@ - \ No newline at end of file From b7a39731a6385457e4defd7eeac08edf208266a8 Mon Sep 17 00:00:00 2001 From: Alessia Bardi Date: Sun, 12 Jul 2020 19:28:56 +0200 Subject: [PATCH 6/6] assert, not print --- .../src/test/java/eu/dnetlib/dhp/oa/dedup/EntityMergerTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/EntityMergerTest.java b/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/EntityMergerTest.java index 513e14f07..0b718ecdc 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/EntityMergerTest.java +++ b/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/EntityMergerTest.java @@ -52,7 +52,7 @@ public class EntityMergerTest implements Serializable { Software merged = DedupRecordFactory .entityMerger(dedupId, softwares.iterator(), 0, dataInfo, Software.class); - System.out.println(merged.getBestaccessright().getClassid()); + assertEquals(merged.getBestaccessright().getClassid(), "OPEN SOURCE"); } @Test