forked from D-Net/dnet-hadoop
Merge pull request 'Rewrite SparkPropagateRelation exploiting Dataframe API' (#330) from propagate_relation_rewrite into beta
Reviewed-on: D-Net/dnet-hadoop#330
This commit is contained in:
commit
0515d81c7c
|
@ -1,19 +1,6 @@
|
|||
|
||||
package eu.dnetlib.dhp.oa.dedup;
|
||||
|
||||
import static org.apache.spark.sql.functions.col;
|
||||
|
||||
import java.util.Objects;
|
||||
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.commons.lang3.StringUtils;
|
||||
import org.apache.spark.SparkConf;
|
||||
import org.apache.spark.api.java.function.FilterFunction;
|
||||
import org.apache.spark.api.java.function.MapFunction;
|
||||
import org.apache.spark.sql.*;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
|
||||
import eu.dnetlib.dhp.schema.common.ModelConstants;
|
||||
import eu.dnetlib.dhp.schema.common.ModelSupport;
|
||||
|
@ -21,198 +8,149 @@ import eu.dnetlib.dhp.schema.oaf.DataInfo;
|
|||
import eu.dnetlib.dhp.schema.oaf.Relation;
|
||||
import eu.dnetlib.dhp.utils.ISLookupClientFactory;
|
||||
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService;
|
||||
import org.apache.commons.beanutils.BeanUtils;
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.commons.lang3.StringUtils;
|
||||
import org.apache.spark.SparkConf;
|
||||
import org.apache.spark.api.java.function.FilterFunction;
|
||||
import org.apache.spark.api.java.function.MapFunction;
|
||||
import org.apache.spark.api.java.function.ReduceFunction;
|
||||
import org.apache.spark.sql.*;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
import scala.Tuple2;
|
||||
import scala.Tuple3;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.Iterator;
|
||||
import java.util.Objects;
|
||||
|
||||
import static org.apache.spark.sql.functions.col;
|
||||
|
||||
public class SparkPropagateRelation extends AbstractSparkAction {
|
||||
|
||||
private static final Logger log = LoggerFactory.getLogger(SparkPropagateRelation.class);
|
||||
private static final Logger log = LoggerFactory.getLogger(SparkPropagateRelation.class);
|
||||
|
||||
enum FieldType {
|
||||
SOURCE, TARGET
|
||||
}
|
||||
private static Encoder<Relation> REL_BEAN_ENC = Encoders.bean(Relation.class);
|
||||
|
||||
public SparkPropagateRelation(ArgumentApplicationParser parser, SparkSession spark) {
|
||||
super(parser, spark);
|
||||
}
|
||||
private static Encoder<Relation> REL_KRYO_ENC = Encoders.kryo(Relation.class);
|
||||
|
||||
public static void main(String[] args) throws Exception {
|
||||
ArgumentApplicationParser parser = new ArgumentApplicationParser(
|
||||
IOUtils
|
||||
.toString(
|
||||
SparkPropagateRelation.class
|
||||
.getResourceAsStream(
|
||||
"/eu/dnetlib/dhp/oa/dedup/propagateRelation_parameters.json")));
|
||||
public SparkPropagateRelation(ArgumentApplicationParser parser, SparkSession spark) {
|
||||
super(parser, spark);
|
||||
}
|
||||
|
||||
parser.parseArgument(args);
|
||||
public static void main(String[] args) throws Exception {
|
||||
ArgumentApplicationParser parser = new ArgumentApplicationParser(
|
||||
IOUtils
|
||||
.toString(
|
||||
SparkPropagateRelation.class
|
||||
.getResourceAsStream(
|
||||
"/eu/dnetlib/dhp/oa/dedup/propagateRelation_parameters.json")));
|
||||
|
||||
SparkConf conf = new SparkConf();
|
||||
conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer");
|
||||
conf.registerKryoClasses(ModelSupport.getOafModelClasses());
|
||||
parser.parseArgument(args);
|
||||
|
||||
new SparkPropagateRelation(parser, getSparkSession(conf))
|
||||
.run(ISLookupClientFactory.getLookUpService(parser.get("isLookUpUrl")));
|
||||
}
|
||||
SparkConf conf = new SparkConf();
|
||||
conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer");
|
||||
conf.registerKryoClasses(ModelSupport.getOafModelClasses());
|
||||
|
||||
@Override
|
||||
public void run(ISLookUpService isLookUpService) {
|
||||
new SparkPropagateRelation(parser, getSparkSession(conf))
|
||||
.run(ISLookupClientFactory.getLookUpService(parser.get("isLookUpUrl")));
|
||||
}
|
||||
|
||||
final String graphBasePath = parser.get("graphBasePath");
|
||||
final String workingPath = parser.get("workingPath");
|
||||
final String graphOutputPath = parser.get("graphOutputPath");
|
||||
@Override
|
||||
public void run(ISLookUpService isLookUpService) {
|
||||
|
||||
log.info("graphBasePath: '{}'", graphBasePath);
|
||||
log.info("workingPath: '{}'", workingPath);
|
||||
log.info("graphOutputPath: '{}'", graphOutputPath);
|
||||
final String graphBasePath = parser.get("graphBasePath");
|
||||
final String workingPath = parser.get("workingPath");
|
||||
final String graphOutputPath = parser.get("graphOutputPath");
|
||||
|
||||
final String outputRelationPath = DedupUtility.createEntityPath(graphOutputPath, "relation");
|
||||
removeOutputDir(spark, outputRelationPath);
|
||||
log.info("graphBasePath: '{}'", graphBasePath);
|
||||
log.info("workingPath: '{}'", workingPath);
|
||||
log.info("graphOutputPath: '{}'", graphOutputPath);
|
||||
|
||||
Dataset<Relation> mergeRels = spark
|
||||
.read()
|
||||
.load(DedupUtility.createMergeRelPath(workingPath, "*", "*"))
|
||||
.as(Encoders.bean(Relation.class));
|
||||
final String outputRelationPath = DedupUtility.createEntityPath(graphOutputPath, "relation");
|
||||
removeOutputDir(spark, outputRelationPath);
|
||||
|
||||
// <mergedObjectID, dedupID>
|
||||
Dataset<Tuple2<String, String>> mergedIds = mergeRels
|
||||
.where(col("relClass").equalTo(ModelConstants.MERGES))
|
||||
.select(col("source"), col("target"))
|
||||
.distinct()
|
||||
.map(
|
||||
(MapFunction<Row, Tuple2<String, String>>) r -> new Tuple2<>(r.getString(1), r.getString(0)),
|
||||
Encoders.tuple(Encoders.STRING(), Encoders.STRING()))
|
||||
.cache();
|
||||
Dataset<Relation> mergeRels = spark
|
||||
.read()
|
||||
.load(DedupUtility.createMergeRelPath(workingPath, "*", "*"))
|
||||
.as(REL_BEAN_ENC);
|
||||
|
||||
final String relationPath = DedupUtility.createEntityPath(graphBasePath, "relation");
|
||||
// <mergedObjectID, dedupID>
|
||||
Dataset<Row> mergedIds = mergeRels
|
||||
.where(col("relClass").equalTo(ModelConstants.MERGES))
|
||||
.select(col("source").as("dedupID"), col("target").as("mergedObjectID"))
|
||||
.distinct()
|
||||
.cache();
|
||||
|
||||
Dataset<Relation> rels = spark.read().textFile(relationPath).map(patchRelFn(), Encoders.bean(Relation.class));
|
||||
Dataset<Row> allRels = spark.read()
|
||||
.schema(REL_BEAN_ENC.schema())
|
||||
.json(DedupUtility.createEntityPath(graphBasePath, "relation"));
|
||||
|
||||
Dataset<Relation> newRels = createNewRels(rels, mergedIds, getFixRelFn());
|
||||
Dataset<Relation> dedupedRels = allRels
|
||||
.joinWith(mergedIds, allRels.col("source").equalTo(mergedIds.col("mergedObjectID")), "left_outer")
|
||||
.joinWith(mergedIds, col("_1.target").equalTo(mergedIds.col("mergedObjectID")), "left_outer")
|
||||
.select("_1._1", "_1._2.dedupID", "_2.dedupID")
|
||||
.as(Encoders.tuple(REL_BEAN_ENC, Encoders.STRING(), Encoders.STRING()))
|
||||
.flatMap(SparkPropagateRelation::addInferredRelations, REL_KRYO_ENC);
|
||||
|
||||
Dataset<Relation> updated = processDataset(
|
||||
processDataset(rels, mergedIds, FieldType.SOURCE, getDeletedFn()),
|
||||
mergedIds,
|
||||
FieldType.TARGET,
|
||||
getDeletedFn());
|
||||
Dataset<Relation> processedRelations = distinctRelations(dedupedRels.union(mergeRels.map((MapFunction<Relation, Relation>) r -> r, REL_KRYO_ENC)))
|
||||
.filter((FilterFunction<Relation>) r -> !Objects.equals(r.getSource(), r.getTarget()));
|
||||
|
||||
save(
|
||||
distinctRelations(
|
||||
newRels
|
||||
.union(updated)
|
||||
.union(mergeRels)
|
||||
.map((MapFunction<Relation, Relation>) r -> r, Encoders.kryo(Relation.class)))
|
||||
.filter((FilterFunction<Relation>) r -> !Objects.equals(r.getSource(), r.getTarget())),
|
||||
outputRelationPath, SaveMode.Overwrite);
|
||||
}
|
||||
save(processedRelations, outputRelationPath, SaveMode.Overwrite);
|
||||
}
|
||||
|
||||
private Dataset<Relation> distinctRelations(Dataset<Relation> rels) {
|
||||
return rels
|
||||
.filter(getRelationFilterFunction())
|
||||
.groupByKey(
|
||||
(MapFunction<Relation, String>) r -> String
|
||||
.join(r.getSource(), r.getTarget(), r.getRelType(), r.getSubRelType(), r.getRelClass()),
|
||||
Encoders.STRING())
|
||||
.agg(new RelationAggregator().toColumn())
|
||||
.map((MapFunction<Tuple2<String, Relation>, Relation>) Tuple2::_2, Encoders.bean(Relation.class));
|
||||
}
|
||||
private static Iterator<Relation> addInferredRelations(Tuple3<Relation, String, String> t) throws Exception {
|
||||
Relation existingRel = t._1();
|
||||
String newSource = t._2();
|
||||
String newTarget = t._3();
|
||||
|
||||
// redirect the relations to the dedupID
|
||||
private static Dataset<Relation> createNewRels(
|
||||
Dataset<Relation> rels, // all the relations to be redirected
|
||||
Dataset<Tuple2<String, String>> mergedIds, // merge rels: <mergedObjectID, dedupID>
|
||||
MapFunction<Tuple2<Tuple2<Tuple3<String, Relation, String>, Tuple2<String, String>>, Tuple2<String, String>>, Relation> mapRel) {
|
||||
if (newSource == null && newTarget == null) {
|
||||
return Collections.singleton(t._1()).iterator();
|
||||
}
|
||||
|
||||
// <sourceID, relation, targetID>
|
||||
Dataset<Tuple3<String, Relation, String>> mapped = rels
|
||||
.map(
|
||||
(MapFunction<Relation, Tuple3<String, Relation, String>>) r -> new Tuple3<>(getId(r, FieldType.SOURCE),
|
||||
r, getId(r, FieldType.TARGET)),
|
||||
Encoders.tuple(Encoders.STRING(), Encoders.kryo(Relation.class), Encoders.STRING()));
|
||||
// update existing relation
|
||||
if (existingRel.getDataInfo() == null) {
|
||||
existingRel.setDataInfo(new DataInfo());
|
||||
}
|
||||
existingRel.getDataInfo().setDeletedbyinference(true);
|
||||
|
||||
// < <sourceID, relation, target>, <sourceID, dedupID> >
|
||||
Dataset<Tuple2<Tuple3<String, Relation, String>, Tuple2<String, String>>> relSource = mapped
|
||||
.joinWith(mergedIds, mapped.col("_1").equalTo(mergedIds.col("_1")), "left_outer");
|
||||
// Create new relation inferred by dedupIDs
|
||||
Relation inferredRel = (Relation) BeanUtils.cloneBean(existingRel);
|
||||
|
||||
// < <<sourceID, relation, targetID>, <sourceID, dedupID>>, <targetID, dedupID> >
|
||||
Dataset<Tuple2<Tuple2<Tuple3<String, Relation, String>, Tuple2<String, String>>, Tuple2<String, String>>> relSourceTarget = relSource
|
||||
.joinWith(mergedIds, relSource.col("_1._3").equalTo(mergedIds.col("_1")), "left_outer");
|
||||
inferredRel.setDataInfo((DataInfo) BeanUtils.cloneBean(existingRel.getDataInfo()));
|
||||
inferredRel.getDataInfo().setDeletedbyinference(false);
|
||||
|
||||
return relSourceTarget
|
||||
.filter(
|
||||
(FilterFunction<Tuple2<Tuple2<Tuple3<String, Relation, String>, Tuple2<String, String>>, Tuple2<String, String>>>) r -> r
|
||||
._1()
|
||||
._1() != null || r._2() != null)
|
||||
.map(mapRel, Encoders.bean(Relation.class))
|
||||
.distinct();
|
||||
}
|
||||
if (newSource != null)
|
||||
inferredRel.setSource(newSource);
|
||||
|
||||
private static Dataset<Relation> processDataset(
|
||||
Dataset<Relation> rels,
|
||||
Dataset<Tuple2<String, String>> mergedIds,
|
||||
FieldType type,
|
||||
MapFunction<Tuple2<Tuple2<String, Relation>, Tuple2<String, String>>, Relation> mapFn) {
|
||||
final Dataset<Tuple2<String, Relation>> mapped = rels
|
||||
.map(
|
||||
(MapFunction<Relation, Tuple2<String, Relation>>) r -> new Tuple2<>(getId(r, type), r),
|
||||
Encoders.tuple(Encoders.STRING(), Encoders.kryo(Relation.class)));
|
||||
return mapped
|
||||
.joinWith(mergedIds, mapped.col("_1").equalTo(mergedIds.col("_1")), "left_outer")
|
||||
.map(mapFn, Encoders.bean(Relation.class));
|
||||
}
|
||||
if (newTarget != null)
|
||||
inferredRel.setTarget(newTarget);
|
||||
|
||||
private FilterFunction<Relation> getRelationFilterFunction() {
|
||||
return r -> StringUtils.isNotBlank(r.getSource()) ||
|
||||
StringUtils.isNotBlank(r.getTarget()) ||
|
||||
StringUtils.isNotBlank(r.getRelType()) ||
|
||||
StringUtils.isNotBlank(r.getSubRelType()) ||
|
||||
StringUtils.isNotBlank(r.getRelClass());
|
||||
}
|
||||
return Arrays.asList(existingRel, inferredRel).iterator();
|
||||
}
|
||||
|
||||
private static String getId(Relation r, FieldType type) {
|
||||
switch (type) {
|
||||
case SOURCE:
|
||||
return r.getSource();
|
||||
case TARGET:
|
||||
return r.getTarget();
|
||||
default:
|
||||
throw new IllegalArgumentException("");
|
||||
}
|
||||
}
|
||||
|
||||
private static MapFunction<Tuple2<Tuple2<Tuple3<String, Relation, String>, Tuple2<String, String>>, Tuple2<String, String>>, Relation> getFixRelFn() {
|
||||
return value -> {
|
||||
|
||||
Relation r = value._1()._1()._2();
|
||||
String newSource = value._1()._2() != null ? value._1()._2()._2() : null;
|
||||
String newTarget = value._2() != null ? value._2()._2() : null;
|
||||
|
||||
if (r.getDataInfo() == null) {
|
||||
r.setDataInfo(new DataInfo());
|
||||
}
|
||||
r.getDataInfo().setDeletedbyinference(false);
|
||||
|
||||
if (newSource != null)
|
||||
r.setSource(newSource);
|
||||
|
||||
if (newTarget != null)
|
||||
r.setTarget(newTarget);
|
||||
|
||||
return r;
|
||||
};
|
||||
}
|
||||
|
||||
private static MapFunction<Tuple2<Tuple2<String, Relation>, Tuple2<String, String>>, Relation> getDeletedFn() {
|
||||
return value -> {
|
||||
if (value._2() != null) {
|
||||
Relation r = value._1()._2();
|
||||
if (r.getDataInfo() == null) {
|
||||
r.setDataInfo(new DataInfo());
|
||||
}
|
||||
r.getDataInfo().setDeletedbyinference(true);
|
||||
return r;
|
||||
}
|
||||
return value._1()._2();
|
||||
};
|
||||
}
|
||||
private Dataset<Relation> distinctRelations(Dataset<Relation> rels) {
|
||||
return rels
|
||||
.filter(getRelationFilterFunction())
|
||||
.groupByKey(
|
||||
(MapFunction<Relation, String>) r -> String
|
||||
.join(" ", r.getSource(), r.getTarget(), r.getRelType(), r.getSubRelType(), r.getRelClass()),
|
||||
Encoders.STRING())
|
||||
.reduceGroups((ReduceFunction<Relation>) (b, a) -> {
|
||||
b.mergeFrom(a);
|
||||
return b;
|
||||
}
|
||||
)
|
||||
.map((MapFunction<Tuple2<String, Relation>, Relation>) Tuple2::_2, REL_BEAN_ENC);
|
||||
}
|
||||
|
||||
private FilterFunction<Relation> getRelationFilterFunction() {
|
||||
return r -> StringUtils.isNotBlank(r.getSource()) ||
|
||||
StringUtils.isNotBlank(r.getTarget()) ||
|
||||
StringUtils.isNotBlank(r.getRelType()) ||
|
||||
StringUtils.isNotBlank(r.getSubRelType()) ||
|
||||
StringUtils.isNotBlank(r.getRelClass());
|
||||
}
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue