forked from D-Net/dnet-hadoop
experimenting with pruning of relations
This commit is contained in:
parent
67e1d222b6
commit
ff4d6214f1
|
@ -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());
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
Loading…
Reference in New Issue