forked from D-Net/dnet-hadoop
trying to avoid OOM in SparkPropagateRelation
This commit is contained in:
parent
8eedfefc98
commit
069ef5eaed
|
@ -12,6 +12,9 @@ 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.Log;
|
||||||
import org.apache.commons.logging.LogFactory;
|
import org.apache.commons.logging.LogFactory;
|
||||||
|
import org.apache.hadoop.fs.FileStatus;
|
||||||
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
|
import org.apache.hadoop.fs.Path;
|
||||||
import org.apache.hadoop.io.compress.GzipCodec;
|
import org.apache.hadoop.io.compress.GzipCodec;
|
||||||
import org.apache.spark.SparkConf;
|
import org.apache.spark.SparkConf;
|
||||||
import org.apache.spark.api.java.JavaPairRDD;
|
import org.apache.spark.api.java.JavaPairRDD;
|
||||||
|
@ -30,6 +33,11 @@ import java.io.IOException;
|
||||||
|
|
||||||
public class SparkPropagateRelation extends AbstractSparkAction {
|
public class SparkPropagateRelation extends AbstractSparkAction {
|
||||||
|
|
||||||
|
private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper()
|
||||||
|
.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);;
|
||||||
|
|
||||||
|
public static final int NUM_PARTITIONS = 3000;
|
||||||
|
|
||||||
enum FieldType {
|
enum FieldType {
|
||||||
SOURCE,
|
SOURCE,
|
||||||
TARGET
|
TARGET
|
||||||
|
@ -64,8 +72,18 @@ public class SparkPropagateRelation extends AbstractSparkAction {
|
||||||
System.out.println(String.format("workingPath: '%s'", workingPath));
|
System.out.println(String.format("workingPath: '%s'", workingPath));
|
||||||
System.out.println(String.format("dedupGraphPath:'%s'", dedupGraphPath));
|
System.out.println(String.format("dedupGraphPath:'%s'", dedupGraphPath));
|
||||||
|
|
||||||
|
final String relationsPath = DedupUtility.createEntityPath(dedupGraphPath, "relation");
|
||||||
|
final String newRelsPath = DedupUtility.createEntityPath(workingPath, "newRels");
|
||||||
|
final String fixedSourceId = DedupUtility.createEntityPath(workingPath, "fixedSourceId");
|
||||||
|
final String deletedSourceId = DedupUtility.createEntityPath(workingPath, "deletedSourceId");
|
||||||
|
|
||||||
final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext());
|
final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext());
|
||||||
|
|
||||||
|
deletePath(relationsPath);
|
||||||
|
deletePath(newRelsPath);
|
||||||
|
deletePath(fixedSourceId);
|
||||||
|
deletePath(deletedSourceId);
|
||||||
|
|
||||||
final Dataset<Relation> mergeRels = spark.read().load(DedupUtility.createMergeRelPath(workingPath, "*", "*")).as(Encoders.bean(Relation.class));
|
final Dataset<Relation> mergeRels = spark.read().load(DedupUtility.createMergeRelPath(workingPath, "*", "*")).as(Encoders.bean(Relation.class));
|
||||||
|
|
||||||
final JavaPairRDD<String, String> mergedIds = mergeRels
|
final JavaPairRDD<String, String> mergedIds = mergeRels
|
||||||
|
@ -75,9 +93,9 @@ public class SparkPropagateRelation extends AbstractSparkAction {
|
||||||
.toJavaRDD()
|
.toJavaRDD()
|
||||||
.mapToPair((PairFunction<Row, String, String>) r -> new Tuple2<String, String>(r.getString(1), r.getString(0)));
|
.mapToPair((PairFunction<Row, String, String>) r -> new Tuple2<String, String>(r.getString(1), r.getString(0)));
|
||||||
|
|
||||||
JavaRDD<String> relations = sc.textFile(DedupUtility.createEntityPath(graphBasePath, "relation"));
|
sc.textFile(DedupUtility.createEntityPath(graphBasePath, "relation"))
|
||||||
|
.repartition(NUM_PARTITIONS)
|
||||||
relations.mapToPair(
|
.mapToPair(
|
||||||
(PairFunction<String, String, String>) s ->
|
(PairFunction<String, String, String>) s ->
|
||||||
new Tuple2<String, String>(MapDocumentUtil.getJPathString(SOURCEJSONPATH, s), s))
|
new Tuple2<String, String>(MapDocumentUtil.getJPathString(SOURCEJSONPATH, s), s))
|
||||||
.leftOuterJoin(mergedIds)
|
.leftOuterJoin(mergedIds)
|
||||||
|
@ -87,6 +105,9 @@ public class SparkPropagateRelation extends AbstractSparkAction {
|
||||||
}
|
}
|
||||||
return v1._2()._1();
|
return v1._2()._1();
|
||||||
})
|
})
|
||||||
|
.saveAsTextFile(fixedSourceId, GzipCodec.class);
|
||||||
|
|
||||||
|
sc.textFile(fixedSourceId)
|
||||||
.mapToPair(
|
.mapToPair(
|
||||||
(PairFunction<String, String, String>) s ->
|
(PairFunction<String, String, String>) s ->
|
||||||
new Tuple2<String, String>(MapDocumentUtil.getJPathString(TARGETJSONPATH, s), s))
|
new Tuple2<String, String>(MapDocumentUtil.getJPathString(TARGETJSONPATH, s), s))
|
||||||
|
@ -97,12 +118,13 @@ public class SparkPropagateRelation extends AbstractSparkAction {
|
||||||
}
|
}
|
||||||
return v1._2()._1();
|
return v1._2()._1();
|
||||||
}).filter(SparkPropagateRelation::containsDedup)
|
}).filter(SparkPropagateRelation::containsDedup)
|
||||||
.repartition(500)
|
.repartition(NUM_PARTITIONS)
|
||||||
.saveAsTextFile(DedupUtility.createEntityPath(workingPath, "newRels"), GzipCodec.class);
|
.saveAsTextFile(newRelsPath, GzipCodec.class);
|
||||||
|
|
||||||
//update deleted by inference
|
//update deleted by inference
|
||||||
relations.mapToPair(
|
sc.textFile(DedupUtility.createEntityPath(graphBasePath, "relation"))
|
||||||
(PairFunction<String, String, String>) s ->
|
.repartition(NUM_PARTITIONS)
|
||||||
|
.mapToPair((PairFunction<String, String, String>) s ->
|
||||||
new Tuple2<String, String>(MapDocumentUtil.getJPathString(SOURCEJSONPATH, s), s))
|
new Tuple2<String, String>(MapDocumentUtil.getJPathString(SOURCEJSONPATH, s), s))
|
||||||
.leftOuterJoin(mergedIds)
|
.leftOuterJoin(mergedIds)
|
||||||
.map((Function<Tuple2<String, Tuple2<String, Optional<String>>>, String>) v1 -> {
|
.map((Function<Tuple2<String, Tuple2<String, Optional<String>>>, String>) v1 -> {
|
||||||
|
@ -111,6 +133,10 @@ public class SparkPropagateRelation extends AbstractSparkAction {
|
||||||
}
|
}
|
||||||
return v1._2()._1();
|
return v1._2()._1();
|
||||||
})
|
})
|
||||||
|
.saveAsTextFile(deletedSourceId, GzipCodec.class);
|
||||||
|
|
||||||
|
sc.textFile(deletedSourceId)
|
||||||
|
.repartition(NUM_PARTITIONS)
|
||||||
.mapToPair(
|
.mapToPair(
|
||||||
(PairFunction<String, String, String>) s ->
|
(PairFunction<String, String, String>) s ->
|
||||||
new Tuple2<String, String>(MapDocumentUtil.getJPathString(TARGETJSONPATH, s), s))
|
new Tuple2<String, String>(MapDocumentUtil.getJPathString(TARGETJSONPATH, s), s))
|
||||||
|
@ -121,17 +147,30 @@ public class SparkPropagateRelation extends AbstractSparkAction {
|
||||||
}
|
}
|
||||||
return v1._2()._1();
|
return v1._2()._1();
|
||||||
})
|
})
|
||||||
.repartition(500)
|
.repartition(NUM_PARTITIONS)
|
||||||
.saveAsTextFile(DedupUtility.createEntityPath(workingPath, "updated"), GzipCodec.class);
|
.saveAsTextFile(DedupUtility.createEntityPath(workingPath, "updated"), GzipCodec.class);
|
||||||
|
|
||||||
JavaRDD<String> newRels = sc
|
JavaRDD<String> newRels = sc
|
||||||
.textFile(DedupUtility.createEntityPath(workingPath, "newRels"));
|
.textFile(newRelsPath);
|
||||||
|
|
||||||
sc
|
sc
|
||||||
.textFile(DedupUtility.createEntityPath(workingPath, "updated"))
|
.textFile(DedupUtility.createEntityPath(workingPath, "updated"))
|
||||||
.union(newRels)
|
.union(newRels)
|
||||||
.repartition(1000)
|
.repartition(NUM_PARTITIONS)
|
||||||
.saveAsTextFile(DedupUtility.createEntityPath(dedupGraphPath, "relation"), GzipCodec.class);
|
.saveAsTextFile(relationsPath, GzipCodec.class);
|
||||||
|
}
|
||||||
|
|
||||||
|
private void deletePath(String path) {
|
||||||
|
try {
|
||||||
|
Path p = new Path(path);
|
||||||
|
FileSystem fs = FileSystem.get(spark.sparkContext().hadoopConfiguration());
|
||||||
|
|
||||||
|
if (fs.exists(p)) {
|
||||||
|
fs.delete(p, true);
|
||||||
|
}
|
||||||
|
} catch (IOException e) {
|
||||||
|
throw new RuntimeException(e);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private static boolean containsDedup(final String json) {
|
private static boolean containsDedup(final String json) {
|
||||||
|
@ -142,20 +181,19 @@ public class SparkPropagateRelation extends AbstractSparkAction {
|
||||||
}
|
}
|
||||||
|
|
||||||
private static String replaceField(final String json, final String id, final FieldType type) {
|
private static String replaceField(final String json, final String id, final FieldType type) {
|
||||||
ObjectMapper mapper = new ObjectMapper();
|
|
||||||
mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
|
|
||||||
try {
|
try {
|
||||||
Relation relation = mapper.readValue(json, Relation.class);
|
Relation relation = OBJECT_MAPPER.readValue(json, Relation.class);
|
||||||
if (relation.getDataInfo() == null)
|
if (relation.getDataInfo() == null) {
|
||||||
relation.setDataInfo(new DataInfo());
|
relation.setDataInfo(new DataInfo());
|
||||||
|
}
|
||||||
relation.getDataInfo().setDeletedbyinference(false);
|
relation.getDataInfo().setDeletedbyinference(false);
|
||||||
switch (type) {
|
switch (type) {
|
||||||
case SOURCE:
|
case SOURCE:
|
||||||
relation.setSource(id);
|
relation.setSource(id);
|
||||||
return mapper.writeValueAsString(relation);
|
return OBJECT_MAPPER.writeValueAsString(relation);
|
||||||
case TARGET:
|
case TARGET:
|
||||||
relation.setTarget(id);
|
relation.setTarget(id);
|
||||||
return mapper.writeValueAsString(relation);
|
return OBJECT_MAPPER.writeValueAsString(relation);
|
||||||
default:
|
default:
|
||||||
throw new IllegalArgumentException("");
|
throw new IllegalArgumentException("");
|
||||||
}
|
}
|
||||||
|
@ -165,14 +203,13 @@ public class SparkPropagateRelation extends AbstractSparkAction {
|
||||||
}
|
}
|
||||||
|
|
||||||
private static <T extends Oaf> String updateDeletedByInference(final String json, final Class<T> clazz) {
|
private static <T extends Oaf> String updateDeletedByInference(final String json, final Class<T> clazz) {
|
||||||
final ObjectMapper mapper = new ObjectMapper();
|
|
||||||
mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
|
|
||||||
try {
|
try {
|
||||||
Oaf entity = mapper.readValue(json, clazz);
|
Oaf entity = OBJECT_MAPPER.readValue(json, clazz);
|
||||||
if (entity.getDataInfo()== null)
|
if (entity.getDataInfo() == null) {
|
||||||
entity.setDataInfo(new DataInfo());
|
entity.setDataInfo(new DataInfo());
|
||||||
|
}
|
||||||
entity.getDataInfo().setDeletedbyinference(true);
|
entity.getDataInfo().setDeletedbyinference(true);
|
||||||
return mapper.writeValueAsString(entity);
|
return OBJECT_MAPPER.writeValueAsString(entity);
|
||||||
} catch (IOException e) {
|
} catch (IOException e) {
|
||||||
throw new RuntimeException("Unable to convert json", e);
|
throw new RuntimeException("Unable to convert json", e);
|
||||||
}
|
}
|
||||||
|
|
|
@ -24,6 +24,29 @@
|
||||||
<name>sparkExecutorCores</name>
|
<name>sparkExecutorCores</name>
|
||||||
<description>number of cores used by single executor</description>
|
<description>number of cores used by single executor</description>
|
||||||
</property>
|
</property>
|
||||||
|
|
||||||
|
<property>
|
||||||
|
<name>oozieActionShareLibForSpark2</name>
|
||||||
|
<description>oozie action sharelib for spark 2.*</description>
|
||||||
|
</property>
|
||||||
|
<property>
|
||||||
|
<name>spark2ExtraListeners</name>
|
||||||
|
<value>com.cloudera.spark.lineage.NavigatorAppListener</value>
|
||||||
|
<description>spark 2.* extra listeners classname</description>
|
||||||
|
</property>
|
||||||
|
<property>
|
||||||
|
<name>spark2SqlQueryExecutionListeners</name>
|
||||||
|
<value>com.cloudera.spark.lineage.NavigatorQueryListener</value>
|
||||||
|
<description>spark 2.* sql query execution listeners classname</description>
|
||||||
|
</property>
|
||||||
|
<property>
|
||||||
|
<name>spark2YarnHistoryServerAddress</name>
|
||||||
|
<description>spark 2.* yarn history server address</description>
|
||||||
|
</property>
|
||||||
|
<property>
|
||||||
|
<name>spark2EventLogDir</name>
|
||||||
|
<description>spark 2.* event log dir location</description>
|
||||||
|
</property>
|
||||||
</parameters>
|
</parameters>
|
||||||
|
|
||||||
<global>
|
<global>
|
||||||
|
@ -38,6 +61,10 @@
|
||||||
<name>oozie.launcher.mapred.job.queue.name</name>
|
<name>oozie.launcher.mapred.job.queue.name</name>
|
||||||
<value>${oozieLauncherQueueName}</value>
|
<value>${oozieLauncherQueueName}</value>
|
||||||
</property>
|
</property>
|
||||||
|
<property>
|
||||||
|
<name>oozie.action.sharelib.for.spark</name>
|
||||||
|
<value>${oozieActionShareLibForSpark2}</value>
|
||||||
|
</property>
|
||||||
</configuration>
|
</configuration>
|
||||||
</global>
|
</global>
|
||||||
|
|
||||||
|
@ -50,7 +77,7 @@
|
||||||
<action name="UpdateEntity">
|
<action name="UpdateEntity">
|
||||||
<spark xmlns="uri:oozie:spark-action:0.2">
|
<spark xmlns="uri:oozie:spark-action:0.2">
|
||||||
<prepare>
|
<prepare>
|
||||||
<delete path='${dedupGraphPath}'/>
|
<delete path="${dedupGraphPath}"/>
|
||||||
</prepare>
|
</prepare>
|
||||||
<master>yarn</master>
|
<master>yarn</master>
|
||||||
<mode>cluster</mode>
|
<mode>cluster</mode>
|
||||||
|
@ -58,12 +85,14 @@
|
||||||
<class>eu.dnetlib.dhp.oa.dedup.SparkUpdateEntity</class>
|
<class>eu.dnetlib.dhp.oa.dedup.SparkUpdateEntity</class>
|
||||||
<jar>dhp-dedup-openaire-${projectVersion}.jar</jar>
|
<jar>dhp-dedup-openaire-${projectVersion}.jar</jar>
|
||||||
<spark-opts>
|
<spark-opts>
|
||||||
--executor-memory ${sparkExecutorMemory}
|
--executor-memory=${sparkExecutorMemory}
|
||||||
--executor-cores ${sparkExecutorCores}
|
--executor-cores=${sparkExecutorCores}
|
||||||
--driver-memory=${sparkDriverMemory}
|
--driver-memory=${sparkDriverMemory}
|
||||||
--conf spark.extraListeners="com.cloudera.spark.lineage.NavigatorAppListener"
|
--conf spark.extraListeners=${spark2ExtraListeners}
|
||||||
--conf spark.sql.queryExecutionListeners="com.cloudera.spark.lineage.NavigatorQueryListener"
|
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
|
||||||
--conf spark.sql.warehouse.dir="/user/hive/warehouse"
|
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
|
||||||
|
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
|
||||||
|
--conf spark.sql.shuffle.partitions=3840
|
||||||
</spark-opts>
|
</spark-opts>
|
||||||
<arg>-mt</arg><arg>yarn</arg>
|
<arg>-mt</arg><arg>yarn</arg>
|
||||||
<arg>--i</arg><arg>${graphBasePath}</arg>
|
<arg>--i</arg><arg>${graphBasePath}</arg>
|
||||||
|
@ -77,7 +106,9 @@
|
||||||
<action name="PropagateRelation">
|
<action name="PropagateRelation">
|
||||||
<spark xmlns="uri:oozie:spark-action:0.2">
|
<spark xmlns="uri:oozie:spark-action:0.2">
|
||||||
<prepare>
|
<prepare>
|
||||||
<delete path='${dedupGraphPath}/relation'/>
|
<delete path="${dedupGraphPath}/relation"/>
|
||||||
|
<delete path="${workingPath}/newRels"/>
|
||||||
|
<delete path="${workingPath}/updated"/>
|
||||||
</prepare>
|
</prepare>
|
||||||
<master>yarn</master>
|
<master>yarn</master>
|
||||||
<mode>cluster</mode>
|
<mode>cluster</mode>
|
||||||
|
@ -85,12 +116,14 @@
|
||||||
<class>eu.dnetlib.dhp.oa.dedup.SparkPropagateRelation</class>
|
<class>eu.dnetlib.dhp.oa.dedup.SparkPropagateRelation</class>
|
||||||
<jar>dhp-dedup-openaire-${projectVersion}.jar</jar>
|
<jar>dhp-dedup-openaire-${projectVersion}.jar</jar>
|
||||||
<spark-opts>
|
<spark-opts>
|
||||||
--executor-memory ${sparkExecutorMemory}
|
--executor-memory=${sparkExecutorMemory}
|
||||||
--executor-cores ${sparkExecutorCores}
|
--executor-cores=${sparkExecutorCores}
|
||||||
--driver-memory=${sparkDriverMemory}
|
--driver-memory=${sparkDriverMemory}
|
||||||
--conf spark.extraListeners="com.cloudera.spark.lineage.NavigatorAppListener"
|
--conf spark.extraListeners=${spark2ExtraListeners}
|
||||||
--conf spark.sql.queryExecutionListeners="com.cloudera.spark.lineage.NavigatorQueryListener"
|
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
|
||||||
--conf spark.sql.warehouse.dir="/user/hive/warehouse"
|
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
|
||||||
|
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
|
||||||
|
--conf spark.sql.shuffle.partitions=7680
|
||||||
</spark-opts>
|
</spark-opts>
|
||||||
<arg>-mt</arg><arg>yarn</arg>
|
<arg>-mt</arg><arg>yarn</arg>
|
||||||
<arg>--i</arg><arg>${graphBasePath}</arg>
|
<arg>--i</arg><arg>${graphBasePath}</arg>
|
||||||
|
|
Loading…
Reference in New Issue