experimenting with pruning of relations

This commit is contained in:
Claudio Atzori 2020-07-10 10:06:41 +02:00
parent 67e1d222b6
commit ff4d6214f1
1 changed files with 174 additions and 170 deletions

View File

@ -59,200 +59,204 @@ import scala.Tuple2;
*/ */
public class PrepareRelationsJob { 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 { public static void main(String[] args) throws Exception {
String jsonConfiguration = IOUtils String jsonConfiguration = IOUtils
.toString( .toString(
PrepareRelationsJob.class PrepareRelationsJob.class
.getResourceAsStream( .getResourceAsStream(
"/eu/dnetlib/dhp/oa/provision/input_params_prepare_relations.json")); "/eu/dnetlib/dhp/oa/provision/input_params_prepare_relations.json"));
final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration); final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration);
parser.parseArgument(args); parser.parseArgument(args);
Boolean isSparkSessionManaged = Optional Boolean isSparkSessionManaged = Optional
.ofNullable(parser.get("isSparkSessionManaged")) .ofNullable(parser.get("isSparkSessionManaged"))
.map(Boolean::valueOf) .map(Boolean::valueOf)
.orElse(Boolean.TRUE); .orElse(Boolean.TRUE);
log.info("isSparkSessionManaged: {}", isSparkSessionManaged); log.info("isSparkSessionManaged: {}", isSparkSessionManaged);
String inputRelationsPath = parser.get("inputRelationsPath"); String inputRelationsPath = parser.get("inputRelationsPath");
log.info("inputRelationsPath: {}", inputRelationsPath); log.info("inputRelationsPath: {}", inputRelationsPath);
String outputPath = parser.get("outputPath"); String outputPath = parser.get("outputPath");
log.info("outputPath: {}", outputPath); log.info("outputPath: {}", outputPath);
int relPartitions = Optional int relPartitions = Optional
.ofNullable(parser.get("relPartitions")) .ofNullable(parser.get("relPartitions"))
.map(Integer::valueOf) .map(Integer::valueOf)
.orElse(DEFAULT_NUM_PARTITIONS); .orElse(DEFAULT_NUM_PARTITIONS);
log.info("relPartitions: {}", relPartitions); log.info("relPartitions: {}", relPartitions);
Set<String> relationFilter = Optional Set<String> relationFilter = Optional
.ofNullable(parser.get("relationFilter")) .ofNullable(parser.get("relationFilter"))
.map(s -> Sets.newHashSet(Splitter.on(",").split(s))) .map(s -> Sets.newHashSet(Splitter.on(",").split(s)))
.orElse(new HashSet<>()); .orElse(new HashSet<>());
log.info("relationFilter: {}", relationFilter); log.info("relationFilter: {}", relationFilter);
int maxRelations = Optional int maxRelations = Optional
.ofNullable(parser.get("maxRelations")) .ofNullable(parser.get("maxRelations"))
.map(Integer::valueOf) .map(Integer::valueOf)
.orElse(MAX_RELS); .orElse(MAX_RELS);
log.info("maxRelations: {}", maxRelations); log.info("maxRelations: {}", maxRelations);
SparkConf conf = new SparkConf(); SparkConf conf = new SparkConf();
conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer"); conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer");
conf.registerKryoClasses(ProvisionModelSupport.getModelClasses()); conf.registerKryoClasses(ProvisionModelSupport.getModelClasses());
runWithSparkSession( runWithSparkSession(
conf, conf,
isSparkSessionManaged, isSparkSessionManaged,
spark -> { spark -> {
removeOutputDir(spark, outputPath); removeOutputDir(spark, outputPath);
prepareRelationsRDD( prepareRelationsRDD(
spark, inputRelationsPath, outputPath, relationFilter, maxRelations, relPartitions); 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<String> relationFilter, int maxRelations, int relPartitions) {
JavaRDD<Relation> rels = readPathRelationRDD(spark, inputRelationsPath);
JavaRDD<Relation> pruned = pruneRels(
pruneRels(rels, relationFilter, maxRelations, relPartitions, (Function<Relation, String>) r -> r.getSource()),
relationFilter, maxRelations, relPartitions, (Function<Relation, String>) r -> r.getTarget());
spark
.createDataset(pruned.rdd(), Encoders.bean(Relation.class))
.repartition(relPartitions)
.write()
.mode(SaveMode.Overwrite)
.parquet(outputPath);
}
private static JavaRDD<Relation> pruneRels(JavaRDD<Relation> rels, Set<String> relationFilter, int maxRelations, int relPartitions, Function<Relation, String> 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);
} }
/** // experimental
* RDD based implementation that prepares the graph relations by limiting the number of outgoing links and filtering private static void prepareRelationsDataset(
* the relation types according to the given criteria. Moreover, outgoing links kept within the given limit are SparkSession spark, String inputRelationsPath, String outputPath, Set<String> relationFilter, int maxRelations,
* prioritized according to the weights indicated in eu.dnetlib.dhp.oa.provision.model.SortableRelation. int relPartitions) {
* spark
* @param spark the spark session .read()
* @param inputRelationsPath source path for the graph relations .textFile(inputRelationsPath)
* @param outputPath output path for the processed relations .repartition(relPartitions)
* @param relationFilter set of relation filters applied to the `relClass` field .map(
* @param maxRelations maximum number of allowed outgoing edges (MapFunction<String, Relation>) s -> OBJECT_MAPPER.readValue(s, Relation.class),
* @param relPartitions number of partitions for the output RDD Encoders.kryo(Relation.class))
*/ .filter((FilterFunction<Relation>) rel -> rel.getDataInfo().getDeletedbyinference() == false)
private static void prepareRelationsRDD(SparkSession spark, String inputRelationsPath, String outputPath, .filter((FilterFunction<Relation>) rel -> relationFilter.contains(rel.getRelClass()) == false)
Set<String> relationFilter, int maxRelations, int relPartitions) { .groupByKey(
(MapFunction<Relation, String>) Relation::getSource,
Encoders.STRING())
.agg(new RelationAggregator(maxRelations).toColumn())
.flatMap(
(FlatMapFunction<Tuple2<String, RelationList>, 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 public static class RelationAggregator
RDD<Relation> bySource = readPathRelationRDD(spark, inputRelationsPath) extends Aggregator<Relation, RelationList, RelationList> {
.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();
spark private int maxRelations;
.createDataset(bySource, Encoders.bean(Relation.class))
.repartition(relPartitions)
.write()
.mode(SaveMode.Overwrite)
.parquet(outputPath);
}
// experimental public RelationAggregator(int maxRelations) {
private static void prepareRelationsDataset( this.maxRelations = maxRelations;
SparkSession spark, String inputRelationsPath, String outputPath, Set<String> relationFilter, int maxRelations, }
int relPartitions) {
spark
.read()
.textFile(inputRelationsPath)
.repartition(relPartitions)
.map(
(MapFunction<String, Relation>) s -> OBJECT_MAPPER.readValue(s, Relation.class),
Encoders.kryo(Relation.class))
.filter((FilterFunction<Relation>) rel -> rel.getDataInfo().getDeletedbyinference() == false)
.filter((FilterFunction<Relation>) rel -> relationFilter.contains(rel.getRelClass()) == false)
.groupByKey(
(MapFunction<Relation, String>) Relation::getSource,
Encoders.STRING())
.agg(new RelationAggregator(maxRelations).toColumn())
.flatMap(
(FlatMapFunction<Tuple2<String, RelationList>, Relation>) t -> Iterables
.limit(t._2().getRelations(), maxRelations)
.iterator(),
Encoders.bean(Relation.class))
.repartition(relPartitions)
.write()
.mode(SaveMode.Overwrite)
.parquet(outputPath);
}
public static class RelationAggregator @Override
extends Aggregator<Relation, RelationList, RelationList> { 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) { @Override
this.maxRelations = maxRelations; public RelationList merge(RelationList b1, RelationList b2) {
} b1.getRelations().addAll(b2.getRelations());
return getSortableRelationList(b1);
}
@Override @Override
public RelationList zero() { public RelationList finish(RelationList r) {
return new RelationList(); return getSortableRelationList(r);
} }
@Override private RelationList getSortableRelationList(RelationList b1) {
public RelationList reduce(RelationList b, Relation a) { RelationList sr = new RelationList();
b.getRelations().add(a); sr
return getSortableRelationList(b); .setRelations(
} b1
.getRelations()
.stream()
.limit(maxRelations)
.collect(Collectors.toCollection(() -> new PriorityQueue<>(new RelationComparator()))));
return sr;
}
@Override @Override
public RelationList merge(RelationList b1, RelationList b2) { public Encoder<RelationList> bufferEncoder() {
b1.getRelations().addAll(b2.getRelations()); return Encoders.kryo(RelationList.class);
return getSortableRelationList(b1); }
}
@Override @Override
public RelationList finish(RelationList r) { public Encoder<RelationList> outputEncoder() {
return getSortableRelationList(r); return Encoders.kryo(RelationList.class);
} }
}
private RelationList getSortableRelationList(RelationList b1) { /**
RelationList sr = new RelationList(); * Reads a JavaRDD of eu.dnetlib.dhp.oa.provision.model.SortableRelation objects from a newline delimited json text
sr * file,
.setRelations( *
b1 * @param spark
.getRelations() * @param inputPath
.stream() * @return the JavaRDD<SortableRelation> containing all the relationships
.limit(maxRelations) */
.collect(Collectors.toCollection(() -> new PriorityQueue<>(new RelationComparator())))); private static JavaRDD<Relation> readPathRelationRDD(
return sr; SparkSession spark, final String inputPath) {
} JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext());
return sc.textFile(inputPath).map(s -> OBJECT_MAPPER.readValue(s, Relation.class));
}
@Override private static void removeOutputDir(SparkSession spark, String path) {
public Encoder<RelationList> bufferEncoder() { HdfsSupport.remove(path, spark.sparkContext().hadoopConfiguration());
return Encoders.kryo(RelationList.class); }
}
@Override
public Encoder<RelationList> 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<SortableRelation> containing all the relationships
*/
private static JavaRDD<Relation> 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());
}
} }