trying to avoid OOM in SparkPropagateRelation

This commit is contained in:
Claudio Atzori 2020-04-16 11:13:51 +02:00
parent 069ef5eaed
commit 011b342bc9
3 changed files with 149 additions and 146 deletions

View File

@ -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())

View File

@ -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();
sc.textFile(DedupUtility.createEntityPath(graphBasePath, "relation")) final String relationPath = 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) Dataset<Relation> rels = spark.read()
.mapToPair( .textFile(relationPath)
(PairFunction<String, String, String>) s -> .map(patchRelFn(), Encoders.bean(Relation.class));
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 replaceField(v1._2()._1(), v1._2()._2().get(), FieldType.TARGET);
}
return v1._2()._1();
}).filter(SparkPropagateRelation::containsDedup)
.repartition(NUM_PARTITIONS)
.saveAsTextFile(newRelsPath, GzipCodec.class);
//update deleted by inference Dataset<Relation> newRels =
sc.textFile(DedupUtility.createEntityPath(graphBasePath, "relation")) processDataset(
.repartition(NUM_PARTITIONS) processDataset(rels, mergedIds, FieldType.SOURCE, getFixRelFn(FieldType.SOURCE)),
.mapToPair((PairFunction<String, String, String>) s -> mergedIds, FieldType.TARGET, getFixRelFn(FieldType.TARGET))
new Tuple2<String, String>(MapDocumentUtil.getJPathString(SOURCEJSONPATH, s), s)) .filter(SparkPropagateRelation::containsDedup);
.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();
})
.saveAsTextFile(deletedSourceId, GzipCodec.class);
sc.textFile(deletedSourceId) Dataset<Relation> updated = processDataset(
.repartition(NUM_PARTITIONS) processDataset(rels, mergedIds, FieldType.SOURCE, getDeletedFn()),
.mapToPair( mergedIds, FieldType.TARGET, getDeletedFn());
(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 save(newRels.union(updated), outputRelationPath);
.textFile(newRelsPath);
sc }
.textFile(DedupUtility.createEntityPath(workingPath, "updated"))
.union(newRels) private static Dataset<Relation> processDataset(Dataset<Relation> rels, Dataset<Tuple2<String, String>> mergedIds, FieldType type,
.repartition(NUM_PARTITIONS) MapFunction<Tuple2<Tuple2<String, Relation>, Tuple2<String, String>>, Relation> mapFn) {
.saveAsTextFile(relationsPath, GzipCodec.class); 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 static MapFunction<String, Relation> patchRelFn() {
return value -> {
final Relation rel = OBJECT_MAPPER.readValue(value, Relation.class);
if (rel.getDataInfo() == null) {
rel.setDataInfo(new DataInfo());
}
return rel;
};
}
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<String, Relation>, Tuple2<String, String>>, Relation> getFixRelFn(FieldType type) {
return value -> {
if (value._2() != null) {
Relation r = value._1()._2();
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);
}
}
} }

View File

@ -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>