forked from D-Net/dnet-hadoop
Rewrite SparkPropagateRelation exploiting Dataframe API
This commit is contained in:
parent
9c8b41475a
commit
0d7b2bf83d
|
@ -1,19 +1,6 @@
|
||||||
|
|
||||||
package eu.dnetlib.dhp.oa.dedup;
|
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.application.ArgumentApplicationParser;
|
||||||
import eu.dnetlib.dhp.schema.common.ModelConstants;
|
import eu.dnetlib.dhp.schema.common.ModelConstants;
|
||||||
import eu.dnetlib.dhp.schema.common.ModelSupport;
|
import eu.dnetlib.dhp.schema.common.ModelSupport;
|
||||||
|
@ -21,16 +8,29 @@ import eu.dnetlib.dhp.schema.oaf.DataInfo;
|
||||||
import eu.dnetlib.dhp.schema.oaf.Relation;
|
import eu.dnetlib.dhp.schema.oaf.Relation;
|
||||||
import eu.dnetlib.dhp.utils.ISLookupClientFactory;
|
import eu.dnetlib.dhp.utils.ISLookupClientFactory;
|
||||||
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService;
|
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService;
|
||||||
|
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.Tuple2;
|
||||||
import scala.Tuple3;
|
import scala.Tuple3;
|
||||||
|
|
||||||
|
import java.util.Objects;
|
||||||
|
|
||||||
|
import static org.apache.spark.sql.functions.col;
|
||||||
|
|
||||||
public class SparkPropagateRelation extends AbstractSparkAction {
|
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 {
|
private static Encoder<Relation> REL_BEAN_ENC = Encoders.bean(Relation.class);
|
||||||
SOURCE, TARGET
|
|
||||||
}
|
private static Encoder<Relation> REL_KRYO_ENC = Encoders.kryo(Relation.class);
|
||||||
|
|
||||||
public SparkPropagateRelation(ArgumentApplicationParser parser, SparkSession spark) {
|
public SparkPropagateRelation(ArgumentApplicationParser parser, SparkSession spark) {
|
||||||
super(parser, spark);
|
super(parser, spark);
|
||||||
|
@ -71,120 +71,42 @@ public class SparkPropagateRelation extends AbstractSparkAction {
|
||||||
Dataset<Relation> mergeRels = spark
|
Dataset<Relation> mergeRels = spark
|
||||||
.read()
|
.read()
|
||||||
.load(DedupUtility.createMergeRelPath(workingPath, "*", "*"))
|
.load(DedupUtility.createMergeRelPath(workingPath, "*", "*"))
|
||||||
.as(Encoders.bean(Relation.class));
|
.as(REL_BEAN_ENC);
|
||||||
|
|
||||||
// <mergedObjectID, dedupID>
|
// <mergedObjectID, dedupID>
|
||||||
Dataset<Tuple2<String, String>> mergedIds = mergeRels
|
Dataset<Row> mergedIds = mergeRels
|
||||||
.where(col("relClass").equalTo(ModelConstants.MERGES))
|
.where(col("relClass").equalTo(ModelConstants.MERGES))
|
||||||
.select(col("source"), col("target"))
|
.select(col("source").as("dedupID"), col("target").as("mergedObjectID"))
|
||||||
.distinct()
|
.distinct()
|
||||||
.map(
|
|
||||||
(MapFunction<Row, Tuple2<String, String>>) r -> new Tuple2<>(r.getString(1), r.getString(0)),
|
|
||||||
Encoders.tuple(Encoders.STRING(), Encoders.STRING()))
|
|
||||||
.cache();
|
.cache();
|
||||||
|
|
||||||
final String relationPath = DedupUtility.createEntityPath(graphBasePath, "relation");
|
final String inputRelationPath = DedupUtility.createEntityPath(graphBasePath, "relation");
|
||||||
|
|
||||||
Dataset<Relation> rels = spark.read().textFile(relationPath).map(patchRelFn(), Encoders.bean(Relation.class));
|
Dataset<Relation> rels = spark.read().schema(REL_BEAN_ENC.schema()).json(inputRelationPath)
|
||||||
|
.as(REL_BEAN_ENC)
|
||||||
|
// .map((MapFunction<Relation, Relation>) rel -> {
|
||||||
|
// if (rel.getDataInfo() == null) {
|
||||||
|
// rel.setDataInfo(new DataInfo());
|
||||||
|
// }
|
||||||
|
// return rel;
|
||||||
|
// }, REL_BEAN_ENC)
|
||||||
|
;
|
||||||
|
|
||||||
Dataset<Relation> newRels = createNewRels(rels, mergedIds, getFixRelFn());
|
Dataset<Tuple3<Relation, String, String>> dedupedRels = rels
|
||||||
|
.joinWith(mergedIds, rels.col("source").equalTo(mergedIds.col("mergedObjectID")), "left_outer")
|
||||||
|
.joinWith(mergedIds, col("_1.target").equalTo(mergedIds.col("mergedObjectID")), "left_outer")
|
||||||
|
.filter("_1._2 IS NOT NULL OR _2 IS NOT NULL")
|
||||||
|
.select("_1._1", "_1._2.dedupID", "_2.dedupID")
|
||||||
|
.as(Encoders.tuple(REL_BEAN_ENC, Encoders.STRING(), Encoders.STRING()))
|
||||||
|
.cache();
|
||||||
|
|
||||||
Dataset<Relation> updated = processDataset(
|
mergedIds.unpersist();
|
||||||
processDataset(rels, mergedIds, FieldType.SOURCE, getDeletedFn()),
|
|
||||||
mergedIds,
|
|
||||||
FieldType.TARGET,
|
|
||||||
getDeletedFn());
|
|
||||||
|
|
||||||
save(
|
Dataset<Relation> newRels = dedupedRels
|
||||||
distinctRelations(
|
.map((MapFunction<Tuple3<Relation, String, String>, Relation>) t -> {
|
||||||
newRels
|
Relation r = t._1();
|
||||||
.union(updated)
|
String newSource = t._2();
|
||||||
.union(mergeRels)
|
String newTarget = t._3();
|
||||||
.map((MapFunction<Relation, Relation>) r -> r, Encoders.kryo(Relation.class)))
|
|
||||||
.filter((FilterFunction<Relation>) r -> !Objects.equals(r.getSource(), r.getTarget())),
|
|
||||||
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));
|
|
||||||
}
|
|
||||||
|
|
||||||
// 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) {
|
|
||||||
|
|
||||||
// <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()));
|
|
||||||
|
|
||||||
// < <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");
|
|
||||||
|
|
||||||
// < <<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");
|
|
||||||
|
|
||||||
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();
|
|
||||||
}
|
|
||||||
|
|
||||||
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));
|
|
||||||
}
|
|
||||||
|
|
||||||
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());
|
|
||||||
}
|
|
||||||
|
|
||||||
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) {
|
if (r.getDataInfo() == null) {
|
||||||
r.setDataInfo(new DataInfo());
|
r.setDataInfo(new DataInfo());
|
||||||
|
@ -198,21 +120,51 @@ public class SparkPropagateRelation extends AbstractSparkAction {
|
||||||
r.setTarget(newTarget);
|
r.setTarget(newTarget);
|
||||||
|
|
||||||
return r;
|
return r;
|
||||||
};
|
}, REL_BEAN_ENC)
|
||||||
}
|
.distinct();
|
||||||
|
|
||||||
private static MapFunction<Tuple2<Tuple2<String, Relation>, Tuple2<String, String>>, Relation> getDeletedFn() {
|
Dataset<Relation> updated = dedupedRels
|
||||||
return value -> {
|
.map((MapFunction<Tuple3<Relation, String, String>, Relation>) t -> {
|
||||||
if (value._2() != null) {
|
Relation r = t._1();
|
||||||
Relation r = value._1()._2();
|
|
||||||
if (r.getDataInfo() == null) {
|
if (r.getDataInfo() == null) {
|
||||||
r.setDataInfo(new DataInfo());
|
r.setDataInfo(new DataInfo());
|
||||||
}
|
}
|
||||||
r.getDataInfo().setDeletedbyinference(true);
|
r.getDataInfo().setDeletedbyinference(true);
|
||||||
return r;
|
return r;
|
||||||
}
|
}, REL_BEAN_ENC);
|
||||||
return value._1()._2();
|
|
||||||
};
|
save(
|
||||||
|
distinctRelations(
|
||||||
|
newRels
|
||||||
|
.union(updated)
|
||||||
|
.union(mergeRels)
|
||||||
|
.map((MapFunction<Relation, Relation>) r -> r, REL_KRYO_ENC)
|
||||||
|
)
|
||||||
|
.filter((FilterFunction<Relation>) r -> !Objects.equals(r.getSource(), r.getTarget())),
|
||||||
|
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())
|
||||||
|
.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