forked from D-Net/dnet-hadoop
trying to avoid OOM in SparkPropagateRelation
This commit is contained in:
parent
069ef5eaed
commit
011b342bc9
|
@ -2,6 +2,7 @@ package eu.dnetlib.dhp.oa.dedup;
|
||||||
|
|
||||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||||
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
|
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
|
||||||
|
import eu.dnetlib.dhp.schema.oaf.*;
|
||||||
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException;
|
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException;
|
||||||
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService;
|
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService;
|
||||||
import eu.dnetlib.pace.config.DedupConfig;
|
import eu.dnetlib.pace.config.DedupConfig;
|
||||||
|
@ -70,6 +71,36 @@ abstract class AbstractSparkAction implements Serializable {
|
||||||
protected static SparkSession getSparkSession(ArgumentApplicationParser parser) {
|
protected static SparkSession getSparkSession(ArgumentApplicationParser parser) {
|
||||||
SparkConf conf = new SparkConf();
|
SparkConf conf = new SparkConf();
|
||||||
|
|
||||||
|
conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer");
|
||||||
|
conf.registerKryoClasses(new Class[] {
|
||||||
|
Author.class,
|
||||||
|
Context.class,
|
||||||
|
Country.class,
|
||||||
|
DataInfo.class,
|
||||||
|
Dataset.class,
|
||||||
|
Datasource.class,
|
||||||
|
ExternalReference.class,
|
||||||
|
ExtraInfo.class,
|
||||||
|
Field.class,
|
||||||
|
GeoLocation.class,
|
||||||
|
Instance.class,
|
||||||
|
Journal.class,
|
||||||
|
KeyValue.class,
|
||||||
|
Oaf.class,
|
||||||
|
OafEntity.class,
|
||||||
|
OAIProvenance.class,
|
||||||
|
Organization.class,
|
||||||
|
OriginDescription.class,
|
||||||
|
OtherResearchProduct.class,
|
||||||
|
Project.class,
|
||||||
|
Publication.class,
|
||||||
|
Qualifier.class,
|
||||||
|
Relation.class,
|
||||||
|
Result.class,
|
||||||
|
Software.class,
|
||||||
|
StructuredProperty.class
|
||||||
|
});
|
||||||
|
|
||||||
return SparkSession
|
return SparkSession
|
||||||
.builder()
|
.builder()
|
||||||
.appName(SparkCreateSimRels.class.getSimpleName())
|
.appName(SparkCreateSimRels.class.getSimpleName())
|
||||||
|
|
|
@ -4,61 +4,50 @@ import com.fasterxml.jackson.databind.DeserializationFeature;
|
||||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||||
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
|
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
|
||||||
import eu.dnetlib.dhp.schema.oaf.DataInfo;
|
import eu.dnetlib.dhp.schema.oaf.DataInfo;
|
||||||
import eu.dnetlib.dhp.schema.oaf.Oaf;
|
|
||||||
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 eu.dnetlib.pace.util.MapDocumentUtil;
|
|
||||||
import org.apache.commons.io.IOUtils;
|
import org.apache.commons.io.IOUtils;
|
||||||
import org.apache.commons.logging.Log;
|
|
||||||
import org.apache.commons.logging.LogFactory;
|
|
||||||
import org.apache.hadoop.fs.FileStatus;
|
|
||||||
import org.apache.hadoop.fs.FileSystem;
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
import org.apache.hadoop.fs.Path;
|
import org.apache.hadoop.fs.Path;
|
||||||
import org.apache.hadoop.io.compress.GzipCodec;
|
import org.apache.spark.api.java.function.MapFunction;
|
||||||
import org.apache.spark.SparkConf;
|
|
||||||
import org.apache.spark.api.java.JavaPairRDD;
|
|
||||||
import org.apache.spark.api.java.JavaRDD;
|
|
||||||
import org.apache.spark.api.java.JavaSparkContext;
|
|
||||||
import org.apache.spark.api.java.Optional;
|
|
||||||
import org.apache.spark.api.java.function.Function;
|
|
||||||
import org.apache.spark.api.java.function.PairFunction;
|
|
||||||
import org.apache.spark.sql.Dataset;
|
import org.apache.spark.sql.Dataset;
|
||||||
import org.apache.spark.sql.Encoders;
|
import org.apache.spark.sql.Encoders;
|
||||||
import org.apache.spark.sql.Row;
|
import org.apache.spark.sql.Row;
|
||||||
import org.apache.spark.sql.SparkSession;
|
import org.apache.spark.sql.SparkSession;
|
||||||
|
import org.slf4j.Logger;
|
||||||
|
import org.slf4j.LoggerFactory;
|
||||||
import scala.Tuple2;
|
import scala.Tuple2;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
|
||||||
|
import static org.apache.spark.sql.functions.col;
|
||||||
|
|
||||||
public class SparkPropagateRelation extends AbstractSparkAction {
|
public class SparkPropagateRelation extends AbstractSparkAction {
|
||||||
|
|
||||||
private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper()
|
private static final Logger log = LoggerFactory.getLogger(SparkPropagateRelation.class);
|
||||||
.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);;
|
|
||||||
|
|
||||||
public static final int NUM_PARTITIONS = 3000;
|
private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper()
|
||||||
|
.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
|
||||||
|
|
||||||
enum FieldType {
|
enum FieldType {
|
||||||
SOURCE,
|
SOURCE,
|
||||||
TARGET
|
TARGET
|
||||||
}
|
}
|
||||||
|
|
||||||
final static String SOURCEJSONPATH = "$.source";
|
|
||||||
final static String TARGETJSONPATH = "$.target";
|
|
||||||
|
|
||||||
private static final Log log = LogFactory.getLog(SparkPropagateRelation.class);
|
|
||||||
|
|
||||||
public SparkPropagateRelation(ArgumentApplicationParser parser, SparkSession spark) throws Exception {
|
public SparkPropagateRelation(ArgumentApplicationParser parser, SparkSession spark) throws Exception {
|
||||||
super(parser, spark);
|
super(parser, spark);
|
||||||
}
|
}
|
||||||
|
|
||||||
public static void main(String[] args) throws Exception {
|
public static void main(String[] args) throws Exception {
|
||||||
ArgumentApplicationParser parser = new ArgumentApplicationParser(
|
ArgumentApplicationParser parser = new ArgumentApplicationParser(
|
||||||
IOUtils.toString(
|
IOUtils.toString(SparkCreateSimRels.class.getResourceAsStream(
|
||||||
SparkCreateSimRels.class.getResourceAsStream("/eu/dnetlib/dhp/oa/dedup/propagateRelation_parameters.json")));
|
"/eu/dnetlib/dhp/oa/dedup/propagateRelation_parameters.json")));
|
||||||
|
|
||||||
parser.parseArgument(args);
|
parser.parseArgument(args);
|
||||||
|
|
||||||
new SparkPropagateRelation(parser, getSparkSession(parser)).run(ISLookupClientFactory.getLookUpService(parser.get("isLookUpUrl")));
|
new SparkPropagateRelation(parser, getSparkSession(parser))
|
||||||
|
.run(ISLookupClientFactory.getLookUpService(parser.get("isLookUpUrl")));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -68,96 +57,114 @@ public class SparkPropagateRelation extends AbstractSparkAction {
|
||||||
final String workingPath = parser.get("workingPath");
|
final String workingPath = parser.get("workingPath");
|
||||||
final String dedupGraphPath = parser.get("dedupGraphPath");
|
final String dedupGraphPath = parser.get("dedupGraphPath");
|
||||||
|
|
||||||
System.out.println(String.format("graphBasePath: '%s'", graphBasePath));
|
log.info("graphBasePath: '{}'", graphBasePath);
|
||||||
System.out.println(String.format("workingPath: '%s'", workingPath));
|
log.info("workingPath: '{}'", workingPath);
|
||||||
System.out.println(String.format("dedupGraphPath:'%s'", dedupGraphPath));
|
log.info("dedupGraphPath: '{}'", dedupGraphPath);
|
||||||
|
|
||||||
final String relationsPath = DedupUtility.createEntityPath(dedupGraphPath, "relation");
|
final String outputRelationPath = DedupUtility.createEntityPath(dedupGraphPath, "relation");
|
||||||
final String newRelsPath = DedupUtility.createEntityPath(workingPath, "newRels");
|
deletePath(outputRelationPath);
|
||||||
final String fixedSourceId = DedupUtility.createEntityPath(workingPath, "fixedSourceId");
|
|
||||||
final String deletedSourceId = DedupUtility.createEntityPath(workingPath, "deletedSourceId");
|
|
||||||
|
|
||||||
final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext());
|
Dataset<Relation> mergeRels = spark.read()
|
||||||
|
.load(DedupUtility.createMergeRelPath(workingPath, "*", "*"))
|
||||||
|
.as(Encoders.bean(Relation.class));
|
||||||
|
|
||||||
deletePath(relationsPath);
|
Dataset<Tuple2<String, String>> mergedIds = mergeRels
|
||||||
deletePath(newRelsPath);
|
.where(col("relClass").equalTo("merges"))
|
||||||
deletePath(fixedSourceId);
|
.select(col("source"), col("target"))
|
||||||
deletePath(deletedSourceId);
|
|
||||||
|
|
||||||
final Dataset<Relation> mergeRels = spark.read().load(DedupUtility.createMergeRelPath(workingPath, "*", "*")).as(Encoders.bean(Relation.class));
|
|
||||||
|
|
||||||
final JavaPairRDD<String, String> mergedIds = mergeRels
|
|
||||||
.where("relClass == 'merges'")
|
|
||||||
.select(mergeRels.col("source"), mergeRels.col("target"))
|
|
||||||
.distinct()
|
.distinct()
|
||||||
.toJavaRDD()
|
.map((MapFunction<Row, Tuple2<String, String>>)
|
||||||
.mapToPair((PairFunction<Row, String, String>) r -> new Tuple2<String, String>(r.getString(1), r.getString(0)));
|
r -> new Tuple2<>(r.getString(1), r.getString(0)),
|
||||||
|
Encoders.tuple(Encoders.STRING(), Encoders.STRING()))
|
||||||
|
.cache();
|
||||||
|
|
||||||
|
final String relationPath = DedupUtility.createEntityPath(graphBasePath, "relation");
|
||||||
|
|
||||||
|
Dataset<Relation> rels = spark.read()
|
||||||
|
.textFile(relationPath)
|
||||||
|
.map(patchRelFn(), Encoders.bean(Relation.class));
|
||||||
|
|
||||||
|
Dataset<Relation> newRels =
|
||||||
|
processDataset(
|
||||||
|
processDataset(rels, mergedIds, FieldType.SOURCE, getFixRelFn(FieldType.SOURCE)),
|
||||||
|
mergedIds, FieldType.TARGET, getFixRelFn(FieldType.TARGET))
|
||||||
|
.filter(SparkPropagateRelation::containsDedup);
|
||||||
|
|
||||||
|
Dataset<Relation> updated = processDataset(
|
||||||
|
processDataset(rels, mergedIds, FieldType.SOURCE, getDeletedFn()),
|
||||||
|
mergedIds, FieldType.TARGET, getDeletedFn());
|
||||||
|
|
||||||
|
save(newRels.union(updated), outputRelationPath);
|
||||||
|
|
||||||
sc.textFile(DedupUtility.createEntityPath(graphBasePath, "relation"))
|
|
||||||
.repartition(NUM_PARTITIONS)
|
|
||||||
.mapToPair(
|
|
||||||
(PairFunction<String, String, String>) s ->
|
|
||||||
new Tuple2<String, String>(MapDocumentUtil.getJPathString(SOURCEJSONPATH, s), s))
|
|
||||||
.leftOuterJoin(mergedIds)
|
|
||||||
.map((Function<Tuple2<String, Tuple2<String, Optional<String>>>, String>) v1 -> {
|
|
||||||
if (v1._2()._2().isPresent()) {
|
|
||||||
return replaceField(v1._2()._1(), v1._2()._2().get(), FieldType.SOURCE);
|
|
||||||
}
|
}
|
||||||
return v1._2()._1();
|
|
||||||
})
|
|
||||||
.saveAsTextFile(fixedSourceId, GzipCodec.class);
|
|
||||||
|
|
||||||
sc.textFile(fixedSourceId)
|
private static Dataset<Relation> processDataset(Dataset<Relation> rels, Dataset<Tuple2<String, String>> mergedIds, FieldType type,
|
||||||
.mapToPair(
|
MapFunction<Tuple2<Tuple2<String, Relation>, Tuple2<String, String>>, Relation> mapFn) {
|
||||||
(PairFunction<String, String, String>) s ->
|
final Dataset<Tuple2<String, Relation>> mapped = rels
|
||||||
new Tuple2<String, String>(MapDocumentUtil.getJPathString(TARGETJSONPATH, s), s))
|
.map((MapFunction<Relation, Tuple2<String, Relation>>)
|
||||||
.leftOuterJoin(mergedIds)
|
r -> new Tuple2<>(getId(r, type), r),
|
||||||
.map((Function<Tuple2<String, Tuple2<String, Optional<String>>>, String>) v1 -> {
|
Encoders.tuple(Encoders.STRING(), Encoders.kryo(Relation.class)));
|
||||||
if (v1._2()._2().isPresent()) {
|
return mapped
|
||||||
return replaceField(v1._2()._1(), v1._2()._2().get(), FieldType.TARGET);
|
.joinWith(mergedIds, mapped.col("_1").equalTo(mergedIds.col("_1")), "left_outer")
|
||||||
|
.map(mapFn, Encoders.bean(Relation.class));
|
||||||
}
|
}
|
||||||
return v1._2()._1();
|
|
||||||
}).filter(SparkPropagateRelation::containsDedup)
|
|
||||||
.repartition(NUM_PARTITIONS)
|
|
||||||
.saveAsTextFile(newRelsPath, GzipCodec.class);
|
|
||||||
|
|
||||||
//update deleted by inference
|
private static MapFunction<String, Relation> patchRelFn() {
|
||||||
sc.textFile(DedupUtility.createEntityPath(graphBasePath, "relation"))
|
return value -> {
|
||||||
.repartition(NUM_PARTITIONS)
|
final Relation rel = OBJECT_MAPPER.readValue(value, Relation.class);
|
||||||
.mapToPair((PairFunction<String, String, String>) s ->
|
if (rel.getDataInfo() == null) {
|
||||||
new Tuple2<String, String>(MapDocumentUtil.getJPathString(SOURCEJSONPATH, s), s))
|
rel.setDataInfo(new DataInfo());
|
||||||
.leftOuterJoin(mergedIds)
|
|
||||||
.map((Function<Tuple2<String, Tuple2<String, Optional<String>>>, String>) v1 -> {
|
|
||||||
if (v1._2()._2().isPresent()) {
|
|
||||||
return updateDeletedByInference(v1._2()._1(), Relation.class);
|
|
||||||
}
|
}
|
||||||
return v1._2()._1();
|
return rel;
|
||||||
})
|
};
|
||||||
.saveAsTextFile(deletedSourceId, GzipCodec.class);
|
|
||||||
|
|
||||||
sc.textFile(deletedSourceId)
|
|
||||||
.repartition(NUM_PARTITIONS)
|
|
||||||
.mapToPair(
|
|
||||||
(PairFunction<String, String, String>) s ->
|
|
||||||
new Tuple2<String, String>(MapDocumentUtil.getJPathString(TARGETJSONPATH, s), s))
|
|
||||||
.leftOuterJoin(mergedIds)
|
|
||||||
.map((Function<Tuple2<String, Tuple2<String, Optional<String>>>, String>) v1 -> {
|
|
||||||
if (v1._2()._2().isPresent()) {
|
|
||||||
return updateDeletedByInference(v1._2()._1(), Relation.class);
|
|
||||||
}
|
}
|
||||||
return v1._2()._1();
|
|
||||||
})
|
|
||||||
.repartition(NUM_PARTITIONS)
|
|
||||||
.saveAsTextFile(DedupUtility.createEntityPath(workingPath, "updated"), GzipCodec.class);
|
|
||||||
|
|
||||||
JavaRDD<String> newRels = sc
|
private static String getId(Relation r, FieldType type) {
|
||||||
.textFile(newRelsPath);
|
switch (type) {
|
||||||
|
case SOURCE:
|
||||||
|
return r.getSource();
|
||||||
|
case TARGET:
|
||||||
|
return r.getTarget();
|
||||||
|
default:
|
||||||
|
throw new IllegalArgumentException("");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
sc
|
private static MapFunction<Tuple2<Tuple2<String, Relation>, Tuple2<String, String>>, Relation> getFixRelFn(FieldType type) {
|
||||||
.textFile(DedupUtility.createEntityPath(workingPath, "updated"))
|
return value -> {
|
||||||
.union(newRels)
|
if (value._2() != null) {
|
||||||
.repartition(NUM_PARTITIONS)
|
Relation r = value._1()._2();
|
||||||
.saveAsTextFile(relationsPath, GzipCodec.class);
|
String id = value._2()._2();
|
||||||
|
if (r.getDataInfo() == null) {
|
||||||
|
r.setDataInfo(new DataInfo());
|
||||||
|
}
|
||||||
|
r.getDataInfo().setDeletedbyinference(false);
|
||||||
|
switch (type) {
|
||||||
|
case SOURCE:
|
||||||
|
r.setSource(id);
|
||||||
|
return r;
|
||||||
|
case TARGET:
|
||||||
|
r.setTarget(id);
|
||||||
|
return r;
|
||||||
|
default:
|
||||||
|
throw new IllegalArgumentException("");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return value._1()._2();
|
||||||
|
};
|
||||||
|
}
|
||||||
|
|
||||||
|
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 void deletePath(String path) {
|
private void deletePath(String path) {
|
||||||
|
@ -173,45 +180,15 @@ public class SparkPropagateRelation extends AbstractSparkAction {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private static boolean containsDedup(final String json) {
|
private static void save(Dataset<Relation> dataset, String outPath) {
|
||||||
final String source = MapDocumentUtil.getJPathString(SOURCEJSONPATH, json);
|
dataset
|
||||||
final String target = MapDocumentUtil.getJPathString(TARGETJSONPATH, json);
|
.write()
|
||||||
|
.option("compression", "gzip")
|
||||||
return source.toLowerCase().contains("dedup") || target.toLowerCase().contains("dedup");
|
.json(outPath);
|
||||||
}
|
}
|
||||||
|
|
||||||
private static String replaceField(final String json, final String id, final FieldType type) {
|
private static boolean containsDedup(final Relation r) {
|
||||||
try {
|
return r.getSource().toLowerCase().contains("dedup") || r.getTarget().toLowerCase().contains("dedup");
|
||||||
Relation relation = OBJECT_MAPPER.readValue(json, Relation.class);
|
|
||||||
if (relation.getDataInfo() == null) {
|
|
||||||
relation.setDataInfo(new DataInfo());
|
|
||||||
}
|
|
||||||
relation.getDataInfo().setDeletedbyinference(false);
|
|
||||||
switch (type) {
|
|
||||||
case SOURCE:
|
|
||||||
relation.setSource(id);
|
|
||||||
return OBJECT_MAPPER.writeValueAsString(relation);
|
|
||||||
case TARGET:
|
|
||||||
relation.setTarget(id);
|
|
||||||
return OBJECT_MAPPER.writeValueAsString(relation);
|
|
||||||
default:
|
|
||||||
throw new IllegalArgumentException("");
|
|
||||||
}
|
|
||||||
} catch (IOException e) {
|
|
||||||
throw new RuntimeException("unable to deserialize json relation: " + json, e);
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
private static <T extends Oaf> String updateDeletedByInference(final String json, final Class<T> clazz) {
|
|
||||||
try {
|
|
||||||
Oaf entity = OBJECT_MAPPER.readValue(json, clazz);
|
|
||||||
if (entity.getDataInfo() == null) {
|
|
||||||
entity.setDataInfo(new DataInfo());
|
|
||||||
}
|
|
||||||
entity.getDataInfo().setDeletedbyinference(true);
|
|
||||||
return OBJECT_MAPPER.writeValueAsString(entity);
|
|
||||||
} catch (IOException e) {
|
|
||||||
throw new RuntimeException("Unable to convert json", e);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
}
|
|
@ -105,11 +105,6 @@
|
||||||
|
|
||||||
<action name="PropagateRelation">
|
<action name="PropagateRelation">
|
||||||
<spark xmlns="uri:oozie:spark-action:0.2">
|
<spark xmlns="uri:oozie:spark-action:0.2">
|
||||||
<prepare>
|
|
||||||
<delete path="${dedupGraphPath}/relation"/>
|
|
||||||
<delete path="${workingPath}/newRels"/>
|
|
||||||
<delete path="${workingPath}/updated"/>
|
|
||||||
</prepare>
|
|
||||||
<master>yarn</master>
|
<master>yarn</master>
|
||||||
<mode>cluster</mode>
|
<mode>cluster</mode>
|
||||||
<name>Update Relations</name>
|
<name>Update Relations</name>
|
||||||
|
|
Loading…
Reference in New Issue