forked from D-Net/dnet-hadoop
allow to set different to relations cut points by source and by target; adjusted weight assigned to relationship types
This commit is contained in:
parent
ff4d6214f1
commit
b21866a2da
|
@ -59,204 +59,216 @@ 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 sourceMaxRelations = Optional
|
||||||
.ofNullable(parser.get("maxRelations"))
|
.ofNullable(parser.get("sourceMaxRelations"))
|
||||||
.map(Integer::valueOf)
|
.map(Integer::valueOf)
|
||||||
.orElse(MAX_RELS);
|
.orElse(MAX_RELS);
|
||||||
log.info("maxRelations: {}", maxRelations);
|
log.info("sourceMaxRelations: {}", sourceMaxRelations);
|
||||||
|
|
||||||
SparkConf conf = new SparkConf();
|
int targetMaxRelations = Optional
|
||||||
conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer");
|
.ofNullable(parser.get("targetMaxRelations"))
|
||||||
conf.registerKryoClasses(ProvisionModelSupport.getModelClasses());
|
.map(Integer::valueOf)
|
||||||
|
.orElse(MAX_RELS);
|
||||||
|
log.info("targetMaxRelations: {}", targetMaxRelations);
|
||||||
|
|
||||||
runWithSparkSession(
|
SparkConf conf = new SparkConf();
|
||||||
conf,
|
conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer");
|
||||||
isSparkSessionManaged,
|
conf.registerKryoClasses(ProvisionModelSupport.getModelClasses());
|
||||||
spark -> {
|
|
||||||
removeOutputDir(spark, outputPath);
|
|
||||||
prepareRelationsRDD(
|
|
||||||
spark, inputRelationsPath, outputPath, relationFilter, maxRelations, relPartitions);
|
|
||||||
});
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
runWithSparkSession(
|
||||||
* RDD based implementation that prepares the graph relations by limiting the number of outgoing links and filtering
|
conf,
|
||||||
* the relation types according to the given criteria. Moreover, outgoing links kept within the given limit are
|
isSparkSessionManaged,
|
||||||
* prioritized according to the weights indicated in eu.dnetlib.dhp.oa.provision.model.SortableRelation.
|
spark -> {
|
||||||
*
|
removeOutputDir(spark, outputPath);
|
||||||
* @param spark the spark session
|
prepareRelationsRDD(
|
||||||
* @param inputRelationsPath source path for the graph relations
|
spark, inputRelationsPath, outputPath, relationFilter, sourceMaxRelations, targetMaxRelations,
|
||||||
* @param outputPath output path for the processed relations
|
relPartitions);
|
||||||
* @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
|
/**
|
||||||
private static void prepareRelationsDataset(
|
* RDD based implementation that prepares the graph relations by limiting the number of outgoing links and filtering
|
||||||
SparkSession spark, String inputRelationsPath, String outputPath, Set<String> relationFilter, int maxRelations,
|
* the relation types according to the given criteria. Moreover, outgoing links kept within the given limit are
|
||||||
int relPartitions) {
|
* prioritized according to the weights indicated in eu.dnetlib.dhp.oa.provision.model.SortableRelation.
|
||||||
spark
|
*
|
||||||
.read()
|
* @param spark the spark session
|
||||||
.textFile(inputRelationsPath)
|
* @param inputRelationsPath source path for the graph relations
|
||||||
.repartition(relPartitions)
|
* @param outputPath output path for the processed relations
|
||||||
.map(
|
* @param relationFilter set of relation filters applied to the `relClass` field
|
||||||
(MapFunction<String, Relation>) s -> OBJECT_MAPPER.readValue(s, Relation.class),
|
* @param sourceMaxRelations maximum number of allowed outgoing edges grouping by relation.source
|
||||||
Encoders.kryo(Relation.class))
|
* @param targetMaxRelations maximum number of allowed outgoing edges grouping by relation.target
|
||||||
.filter((FilterFunction<Relation>) rel -> rel.getDataInfo().getDeletedbyinference() == false)
|
* @param relPartitions number of partitions for the output RDD
|
||||||
.filter((FilterFunction<Relation>) rel -> relationFilter.contains(rel.getRelClass()) == false)
|
*/
|
||||||
.groupByKey(
|
private static void prepareRelationsRDD(SparkSession spark, String inputRelationsPath, String outputPath,
|
||||||
(MapFunction<Relation, String>) Relation::getSource,
|
Set<String> relationFilter, int sourceMaxRelations, int targetMaxRelations, int relPartitions) {
|
||||||
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
|
JavaRDD<Relation> rels = readPathRelationRDD(spark, inputRelationsPath)
|
||||||
extends Aggregator<Relation, RelationList, RelationList> {
|
.filter(rel -> rel.getDataInfo().getDeletedbyinference() == false)
|
||||||
|
.filter(rel -> relationFilter.contains(rel.getRelClass()) == false);
|
||||||
|
|
||||||
private int maxRelations;
|
JavaRDD<Relation> pruned = pruneRels(
|
||||||
|
pruneRels(
|
||||||
|
rels,
|
||||||
|
sourceMaxRelations, relPartitions, (Function<Relation, String>) r -> r.getSource()),
|
||||||
|
targetMaxRelations, relPartitions, (Function<Relation, String>) r -> r.getTarget());
|
||||||
|
spark
|
||||||
|
.createDataset(pruned.rdd(), Encoders.bean(Relation.class))
|
||||||
|
.repartition(relPartitions)
|
||||||
|
.write()
|
||||||
|
.mode(SaveMode.Overwrite)
|
||||||
|
.parquet(outputPath);
|
||||||
|
}
|
||||||
|
|
||||||
public RelationAggregator(int maxRelations) {
|
private static JavaRDD<Relation> pruneRels(JavaRDD<Relation> rels, int maxRelations,
|
||||||
this.maxRelations = maxRelations;
|
int relPartitions, Function<Relation, String> 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
|
// experimental
|
||||||
public RelationList zero() {
|
private static void prepareRelationsDataset(
|
||||||
return new RelationList();
|
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);
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
public static class RelationAggregator
|
||||||
public RelationList reduce(RelationList b, Relation a) {
|
extends Aggregator<Relation, RelationList, RelationList> {
|
||||||
b.getRelations().add(a);
|
|
||||||
return getSortableRelationList(b);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
private int maxRelations;
|
||||||
public RelationList merge(RelationList b1, RelationList b2) {
|
|
||||||
b1.getRelations().addAll(b2.getRelations());
|
|
||||||
return getSortableRelationList(b1);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
public RelationAggregator(int maxRelations) {
|
||||||
public RelationList finish(RelationList r) {
|
this.maxRelations = maxRelations;
|
||||||
return getSortableRelationList(r);
|
}
|
||||||
}
|
|
||||||
|
|
||||||
private RelationList getSortableRelationList(RelationList b1) {
|
@Override
|
||||||
RelationList sr = new RelationList();
|
public RelationList zero() {
|
||||||
sr
|
return new RelationList();
|
||||||
.setRelations(
|
}
|
||||||
b1
|
|
||||||
.getRelations()
|
|
||||||
.stream()
|
|
||||||
.limit(maxRelations)
|
|
||||||
.collect(Collectors.toCollection(() -> new PriorityQueue<>(new RelationComparator()))));
|
|
||||||
return sr;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Encoder<RelationList> bufferEncoder() {
|
public RelationList reduce(RelationList b, Relation a) {
|
||||||
return Encoders.kryo(RelationList.class);
|
b.getRelations().add(a);
|
||||||
}
|
return getSortableRelationList(b);
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Encoder<RelationList> outputEncoder() {
|
public RelationList merge(RelationList b1, RelationList b2) {
|
||||||
return Encoders.kryo(RelationList.class);
|
b1.getRelations().addAll(b2.getRelations());
|
||||||
}
|
return getSortableRelationList(b1);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
@Override
|
||||||
* Reads a JavaRDD of eu.dnetlib.dhp.oa.provision.model.SortableRelation objects from a newline delimited json text
|
public RelationList finish(RelationList r) {
|
||||||
* file,
|
return getSortableRelationList(r);
|
||||||
*
|
}
|
||||||
* @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) {
|
private RelationList getSortableRelationList(RelationList b1) {
|
||||||
HdfsSupport.remove(path, spark.sparkContext().hadoopConfiguration());
|
RelationList sr = new RelationList();
|
||||||
}
|
sr
|
||||||
|
.setRelations(
|
||||||
|
b1
|
||||||
|
.getRelations()
|
||||||
|
.stream()
|
||||||
|
.limit(maxRelations)
|
||||||
|
.collect(Collectors.toCollection(() -> new PriorityQueue<>(new RelationComparator()))));
|
||||||
|
return sr;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Encoder<RelationList> bufferEncoder() {
|
||||||
|
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());
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -16,18 +16,18 @@ public class SortableRelationKey implements Comparable<SortableRelationKey>, Ser
|
||||||
private static final Map<String, Integer> weights = Maps.newHashMap();
|
private static final Map<String, Integer> weights = Maps.newHashMap();
|
||||||
|
|
||||||
static {
|
static {
|
||||||
weights.put("outcome", 0);
|
weights.put("participation", 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("provision", 8);
|
weights.put("outcome", 1);
|
||||||
weights.put("participation", 9);
|
weights.put("affiliation", 2);
|
||||||
weights.put("dedup", 10);
|
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;
|
private static final long serialVersionUID = 3232323;
|
||||||
|
|
|
@ -30,9 +30,16 @@
|
||||||
"paramRequired": false
|
"paramRequired": false
|
||||||
},
|
},
|
||||||
{
|
{
|
||||||
"paramName": "mr",
|
"paramName": "smr",
|
||||||
"paramLongName": "maxRelations",
|
"paramLongName": "sourceMaxRelations",
|
||||||
"paramDescription": "maximum number of relations allowed for a each entity",
|
"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
|
"paramRequired": false
|
||||||
}
|
}
|
||||||
|
|
||||||
]
|
]
|
||||||
|
|
Loading…
Reference in New Issue