forked from D-Net/dnet-hadoop
reverted behaviour of the cleaning workflow: grouping entities by ID will be managed differently
This commit is contained in:
parent
e43ab07af6
commit
33bae02451
|
@ -1,206 +0,0 @@
|
||||||
|
|
||||||
package eu.dnetlib.dhp.oa.graph.clean;
|
|
||||||
|
|
||||||
import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession;
|
|
||||||
import static eu.dnetlib.dhp.utils.DHPUtils.toSeq;
|
|
||||||
|
|
||||||
import java.io.IOException;
|
|
||||||
import java.util.List;
|
|
||||||
import java.util.Objects;
|
|
||||||
import java.util.Optional;
|
|
||||||
import java.util.stream.Collectors;
|
|
||||||
|
|
||||||
import org.apache.commons.io.IOUtils;
|
|
||||||
import org.apache.commons.lang3.StringUtils;
|
|
||||||
import org.apache.spark.SparkConf;
|
|
||||||
import org.apache.spark.api.java.JavaSparkContext;
|
|
||||||
import org.apache.spark.api.java.function.FilterFunction;
|
|
||||||
import org.apache.spark.api.java.function.MapFunction;
|
|
||||||
import org.apache.spark.sql.*;
|
|
||||||
import org.apache.spark.sql.expressions.Aggregator;
|
|
||||||
import org.slf4j.Logger;
|
|
||||||
import org.slf4j.LoggerFactory;
|
|
||||||
|
|
||||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
|
||||||
import com.jayway.jsonpath.Configuration;
|
|
||||||
import com.jayway.jsonpath.DocumentContext;
|
|
||||||
import com.jayway.jsonpath.JsonPath;
|
|
||||||
import com.jayway.jsonpath.Option;
|
|
||||||
|
|
||||||
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
|
|
||||||
import eu.dnetlib.dhp.common.HdfsSupport;
|
|
||||||
import eu.dnetlib.dhp.schema.common.ModelSupport;
|
|
||||||
import eu.dnetlib.dhp.schema.oaf.*;
|
|
||||||
import scala.Tuple2;
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Groups the graph content by entity identifier to ensure ID uniqueness
|
|
||||||
*/
|
|
||||||
public class GroupEntitiesAndRelationsSparkJob {
|
|
||||||
|
|
||||||
private static final Logger log = LoggerFactory.getLogger(GroupEntitiesAndRelationsSparkJob.class);
|
|
||||||
|
|
||||||
private final static String ID_JPATH = "$.id";
|
|
||||||
|
|
||||||
private final static String SOURCE_JPATH = "$.source";
|
|
||||||
|
|
||||||
private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
|
|
||||||
|
|
||||||
public static void main(String[] args) throws Exception {
|
|
||||||
|
|
||||||
String jsonConfiguration = IOUtils
|
|
||||||
.toString(
|
|
||||||
GroupEntitiesAndRelationsSparkJob.class
|
|
||||||
.getResourceAsStream(
|
|
||||||
"/eu/dnetlib/dhp/oa/graph/group_graph_entities_parameters.json"));
|
|
||||||
final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration);
|
|
||||||
parser.parseArgument(args);
|
|
||||||
|
|
||||||
Boolean isSparkSessionManaged = Optional
|
|
||||||
.ofNullable(parser.get("isSparkSessionManaged"))
|
|
||||||
.map(Boolean::valueOf)
|
|
||||||
.orElse(Boolean.TRUE);
|
|
||||||
log.info("isSparkSessionManaged: {}", isSparkSessionManaged);
|
|
||||||
|
|
||||||
String graphInputPath = parser.get("graphInputPath");
|
|
||||||
log.info("graphInputPath: {}", graphInputPath);
|
|
||||||
|
|
||||||
String outputPath = parser.get("outputPath");
|
|
||||||
log.info("outputPath: {}", outputPath);
|
|
||||||
|
|
||||||
SparkConf conf = new SparkConf();
|
|
||||||
conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer");
|
|
||||||
conf.registerKryoClasses(ModelSupport.getOafModelClasses());
|
|
||||||
|
|
||||||
runWithSparkSession(
|
|
||||||
conf,
|
|
||||||
isSparkSessionManaged,
|
|
||||||
spark -> {
|
|
||||||
HdfsSupport.remove(outputPath, spark.sparkContext().hadoopConfiguration());
|
|
||||||
groupEntitiesAndRelations(spark, graphInputPath, outputPath);
|
|
||||||
});
|
|
||||||
}
|
|
||||||
|
|
||||||
private static void groupEntitiesAndRelations(
|
|
||||||
SparkSession spark,
|
|
||||||
String inputPath,
|
|
||||||
String outputPath) {
|
|
||||||
|
|
||||||
TypedColumn<Oaf, Oaf> aggregator = new GroupingAggregator().toColumn();
|
|
||||||
final JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext());
|
|
||||||
spark
|
|
||||||
.read()
|
|
||||||
.textFile(toSeq(listPaths(inputPath, sc)))
|
|
||||||
.map((MapFunction<String, Oaf>) s -> parseOaf(s), Encoders.kryo(Oaf.class))
|
|
||||||
.filter((FilterFunction<Oaf>) oaf -> StringUtils.isNotBlank(ModelSupport.idFn().apply(oaf)))
|
|
||||||
.groupByKey((MapFunction<Oaf, String>) oaf -> ModelSupport.idFn().apply(oaf), Encoders.STRING())
|
|
||||||
.agg(aggregator)
|
|
||||||
.map(
|
|
||||||
(MapFunction<Tuple2<String, Oaf>, String>) t -> t._2().getClass().getName() +
|
|
||||||
"|" + OBJECT_MAPPER.writeValueAsString(t._2()),
|
|
||||||
Encoders.STRING())
|
|
||||||
.write()
|
|
||||||
.option("compression", "gzip")
|
|
||||||
.mode(SaveMode.Overwrite)
|
|
||||||
.text(outputPath);
|
|
||||||
}
|
|
||||||
|
|
||||||
public static class GroupingAggregator extends Aggregator<Oaf, Oaf, Oaf> {
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public Oaf zero() {
|
|
||||||
return null;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public Oaf reduce(Oaf b, Oaf a) {
|
|
||||||
return mergeAndGet(b, a);
|
|
||||||
}
|
|
||||||
|
|
||||||
private Oaf mergeAndGet(Oaf b, Oaf a) {
|
|
||||||
if (Objects.nonNull(a) && Objects.nonNull(b)) {
|
|
||||||
return OafMapperUtils.merge(b, a);
|
|
||||||
}
|
|
||||||
return Objects.isNull(a) ? b : a;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public Oaf merge(Oaf b, Oaf a) {
|
|
||||||
return mergeAndGet(b, a);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public Oaf finish(Oaf j) {
|
|
||||||
return j;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public Encoder<Oaf> bufferEncoder() {
|
|
||||||
return Encoders.kryo(Oaf.class);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public Encoder<Oaf> outputEncoder() {
|
|
||||||
return Encoders.kryo(Oaf.class);
|
|
||||||
}
|
|
||||||
|
|
||||||
}
|
|
||||||
|
|
||||||
private static Oaf parseOaf(String s) {
|
|
||||||
|
|
||||||
DocumentContext dc = JsonPath
|
|
||||||
.parse(s, Configuration.defaultConfiguration().addOptions(Option.SUPPRESS_EXCEPTIONS));
|
|
||||||
final String id = dc.read(ID_JPATH);
|
|
||||||
if (StringUtils.isNotBlank(id)) {
|
|
||||||
|
|
||||||
String prefix = StringUtils.substringBefore(id, "|");
|
|
||||||
switch (prefix) {
|
|
||||||
case "10":
|
|
||||||
return parse(s, Datasource.class);
|
|
||||||
case "20":
|
|
||||||
return parse(s, Organization.class);
|
|
||||||
case "40":
|
|
||||||
return parse(s, Project.class);
|
|
||||||
case "50":
|
|
||||||
String resultType = dc.read("$.resulttype.classid");
|
|
||||||
switch (resultType) {
|
|
||||||
case "publication":
|
|
||||||
return parse(s, Publication.class);
|
|
||||||
case "dataset":
|
|
||||||
return parse(s, eu.dnetlib.dhp.schema.oaf.Dataset.class);
|
|
||||||
case "software":
|
|
||||||
return parse(s, Software.class);
|
|
||||||
case "other":
|
|
||||||
return parse(s, OtherResearchProduct.class);
|
|
||||||
default:
|
|
||||||
throw new IllegalArgumentException(String.format("invalid resultType: '%s'", resultType));
|
|
||||||
}
|
|
||||||
default:
|
|
||||||
throw new IllegalArgumentException(String.format("invalid id prefix: '%s'", prefix));
|
|
||||||
}
|
|
||||||
} else {
|
|
||||||
String source = dc.read(SOURCE_JPATH);
|
|
||||||
if (StringUtils.isNotBlank(source)) {
|
|
||||||
return parse(s, Relation.class);
|
|
||||||
} else {
|
|
||||||
throw new IllegalArgumentException(String.format("invalid oaf: '%s'", s));
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
private static <T extends Oaf> Oaf parse(String s, Class<T> clazz) {
|
|
||||||
try {
|
|
||||||
return OBJECT_MAPPER.readValue(s, clazz);
|
|
||||||
} catch (IOException e) {
|
|
||||||
throw new RuntimeException(e);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
private static List<String> listPaths(String inputPath, JavaSparkContext sc) {
|
|
||||||
return HdfsSupport
|
|
||||||
.listFiles(inputPath, sc.hadoopConfiguration())
|
|
||||||
.stream()
|
|
||||||
.collect(Collectors.toList());
|
|
||||||
}
|
|
||||||
|
|
||||||
}
|
|
|
@ -50,36 +50,12 @@
|
||||||
</property>
|
</property>
|
||||||
</parameters>
|
</parameters>
|
||||||
|
|
||||||
<start to="group_entities"/>
|
<start to="fork_clean_graph"/>
|
||||||
|
|
||||||
<kill name="Kill">
|
<kill name="Kill">
|
||||||
<message>Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}]</message>
|
<message>Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}]</message>
|
||||||
</kill>
|
</kill>
|
||||||
|
|
||||||
<action name="group_entities">
|
|
||||||
<spark xmlns="uri:oozie:spark-action:0.2">
|
|
||||||
<master>yarn</master>
|
|
||||||
<mode>cluster</mode>
|
|
||||||
<name>group graph entities and relations</name>
|
|
||||||
<class>eu.dnetlib.dhp.oa.graph.clean.GroupEntitiesAndRelationsSparkJob</class>
|
|
||||||
<jar>dhp-graph-mapper-${projectVersion}.jar</jar>
|
|
||||||
<spark-opts>
|
|
||||||
--executor-cores=${sparkExecutorCores}
|
|
||||||
--executor-memory=${sparkExecutorMemory}
|
|
||||||
--driver-memory=${sparkDriverMemory}
|
|
||||||
--conf spark.extraListeners=${spark2ExtraListeners}
|
|
||||||
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
|
|
||||||
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
|
|
||||||
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
|
|
||||||
--conf spark.sql.shuffle.partitions=7680
|
|
||||||
</spark-opts>
|
|
||||||
<arg>--graphInputPath</arg><arg>${graphInputPath}</arg>
|
|
||||||
<arg>--outputPath</arg><arg>${workingDir}/grouped_entities</arg>
|
|
||||||
</spark>
|
|
||||||
<ok to="fork_clean_graph"/>
|
|
||||||
<error to="Kill"/>
|
|
||||||
</action>
|
|
||||||
|
|
||||||
<fork name="fork_clean_graph">
|
<fork name="fork_clean_graph">
|
||||||
<path start="clean_publication"/>
|
<path start="clean_publication"/>
|
||||||
<path start="clean_dataset"/>
|
<path start="clean_dataset"/>
|
||||||
|
@ -108,7 +84,7 @@
|
||||||
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
|
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
|
||||||
--conf spark.sql.shuffle.partitions=7680
|
--conf spark.sql.shuffle.partitions=7680
|
||||||
</spark-opts>
|
</spark-opts>
|
||||||
<arg>--inputPath</arg><arg>${workingDir}/grouped_entities</arg>
|
<arg>--inputPath</arg><arg>${graphInputPath}/publication</arg>
|
||||||
<arg>--outputPath</arg><arg>${graphOutputPath}/publication</arg>
|
<arg>--outputPath</arg><arg>${graphOutputPath}/publication</arg>
|
||||||
<arg>--graphTableClassName</arg><arg>eu.dnetlib.dhp.schema.oaf.Publication</arg>
|
<arg>--graphTableClassName</arg><arg>eu.dnetlib.dhp.schema.oaf.Publication</arg>
|
||||||
<arg>--isLookupUrl</arg><arg>${isLookupUrl}</arg>
|
<arg>--isLookupUrl</arg><arg>${isLookupUrl}</arg>
|
||||||
|
@ -134,7 +110,7 @@
|
||||||
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
|
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
|
||||||
--conf spark.sql.shuffle.partitions=7680
|
--conf spark.sql.shuffle.partitions=7680
|
||||||
</spark-opts>
|
</spark-opts>
|
||||||
<arg>--inputPath</arg><arg>${workingDir}/grouped_entities</arg>
|
<arg>--inputPath</arg><arg>${graphInputPath}/dataset</arg>
|
||||||
<arg>--outputPath</arg><arg>${graphOutputPath}/dataset</arg>
|
<arg>--outputPath</arg><arg>${graphOutputPath}/dataset</arg>
|
||||||
<arg>--graphTableClassName</arg><arg>eu.dnetlib.dhp.schema.oaf.Dataset</arg>
|
<arg>--graphTableClassName</arg><arg>eu.dnetlib.dhp.schema.oaf.Dataset</arg>
|
||||||
<arg>--isLookupUrl</arg><arg>${isLookupUrl}</arg>
|
<arg>--isLookupUrl</arg><arg>${isLookupUrl}</arg>
|
||||||
|
@ -160,7 +136,7 @@
|
||||||
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
|
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
|
||||||
--conf spark.sql.shuffle.partitions=7680
|
--conf spark.sql.shuffle.partitions=7680
|
||||||
</spark-opts>
|
</spark-opts>
|
||||||
<arg>--inputPath</arg><arg>${workingDir}/grouped_entities</arg>
|
<arg>--inputPath</arg><arg>${graphInputPath}/otherresearchproduct</arg>
|
||||||
<arg>--outputPath</arg><arg>${graphOutputPath}/otherresearchproduct</arg>
|
<arg>--outputPath</arg><arg>${graphOutputPath}/otherresearchproduct</arg>
|
||||||
<arg>--graphTableClassName</arg><arg>eu.dnetlib.dhp.schema.oaf.OtherResearchProduct</arg>
|
<arg>--graphTableClassName</arg><arg>eu.dnetlib.dhp.schema.oaf.OtherResearchProduct</arg>
|
||||||
<arg>--isLookupUrl</arg><arg>${isLookupUrl}</arg>
|
<arg>--isLookupUrl</arg><arg>${isLookupUrl}</arg>
|
||||||
|
@ -186,7 +162,7 @@
|
||||||
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
|
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
|
||||||
--conf spark.sql.shuffle.partitions=7680
|
--conf spark.sql.shuffle.partitions=7680
|
||||||
</spark-opts>
|
</spark-opts>
|
||||||
<arg>--inputPath</arg><arg>${workingDir}/grouped_entities</arg>
|
<arg>--inputPath</arg><arg>${graphInputPath}/software</arg>
|
||||||
<arg>--outputPath</arg><arg>${graphOutputPath}/software</arg>
|
<arg>--outputPath</arg><arg>${graphOutputPath}/software</arg>
|
||||||
<arg>--graphTableClassName</arg><arg>eu.dnetlib.dhp.schema.oaf.Software</arg>
|
<arg>--graphTableClassName</arg><arg>eu.dnetlib.dhp.schema.oaf.Software</arg>
|
||||||
<arg>--isLookupUrl</arg><arg>${isLookupUrl}</arg>
|
<arg>--isLookupUrl</arg><arg>${isLookupUrl}</arg>
|
||||||
|
@ -212,7 +188,7 @@
|
||||||
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
|
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
|
||||||
--conf spark.sql.shuffle.partitions=7680
|
--conf spark.sql.shuffle.partitions=7680
|
||||||
</spark-opts>
|
</spark-opts>
|
||||||
<arg>--inputPath</arg><arg>${workingDir}/grouped_entities</arg>
|
<arg>--inputPath</arg><arg>${graphInputPath}/datasource</arg>
|
||||||
<arg>--outputPath</arg><arg>${graphOutputPath}/datasource</arg>
|
<arg>--outputPath</arg><arg>${graphOutputPath}/datasource</arg>
|
||||||
<arg>--graphTableClassName</arg><arg>eu.dnetlib.dhp.schema.oaf.Datasource</arg>
|
<arg>--graphTableClassName</arg><arg>eu.dnetlib.dhp.schema.oaf.Datasource</arg>
|
||||||
<arg>--isLookupUrl</arg><arg>${isLookupUrl}</arg>
|
<arg>--isLookupUrl</arg><arg>${isLookupUrl}</arg>
|
||||||
|
@ -238,7 +214,7 @@
|
||||||
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
|
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
|
||||||
--conf spark.sql.shuffle.partitions=7680
|
--conf spark.sql.shuffle.partitions=7680
|
||||||
</spark-opts>
|
</spark-opts>
|
||||||
<arg>--inputPath</arg><arg>${workingDir}/grouped_entities</arg>
|
<arg>--inputPath</arg><arg>${graphInputPath}/organization</arg>
|
||||||
<arg>--outputPath</arg><arg>${graphOutputPath}/organization</arg>
|
<arg>--outputPath</arg><arg>${graphOutputPath}/organization</arg>
|
||||||
<arg>--graphTableClassName</arg><arg>eu.dnetlib.dhp.schema.oaf.Organization</arg>
|
<arg>--graphTableClassName</arg><arg>eu.dnetlib.dhp.schema.oaf.Organization</arg>
|
||||||
<arg>--isLookupUrl</arg><arg>${isLookupUrl}</arg>
|
<arg>--isLookupUrl</arg><arg>${isLookupUrl}</arg>
|
||||||
|
@ -264,7 +240,7 @@
|
||||||
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
|
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
|
||||||
--conf spark.sql.shuffle.partitions=7680
|
--conf spark.sql.shuffle.partitions=7680
|
||||||
</spark-opts>
|
</spark-opts>
|
||||||
<arg>--inputPath</arg><arg>${workingDir}/grouped_entities</arg>
|
<arg>--inputPath</arg><arg>${graphInputPath}/project</arg>
|
||||||
<arg>--outputPath</arg><arg>${graphOutputPath}/project</arg>
|
<arg>--outputPath</arg><arg>${graphOutputPath}/project</arg>
|
||||||
<arg>--graphTableClassName</arg><arg>eu.dnetlib.dhp.schema.oaf.Project</arg>
|
<arg>--graphTableClassName</arg><arg>eu.dnetlib.dhp.schema.oaf.Project</arg>
|
||||||
<arg>--isLookupUrl</arg><arg>${isLookupUrl}</arg>
|
<arg>--isLookupUrl</arg><arg>${isLookupUrl}</arg>
|
||||||
|
@ -290,7 +266,7 @@
|
||||||
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
|
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
|
||||||
--conf spark.sql.shuffle.partitions=7680
|
--conf spark.sql.shuffle.partitions=7680
|
||||||
</spark-opts>
|
</spark-opts>
|
||||||
<arg>--inputPath</arg><arg>${workingDir}/grouped_entities</arg>
|
<arg>--inputPath</arg><arg>${graphInputPath}/relation</arg>
|
||||||
<arg>--outputPath</arg><arg>${graphOutputPath}/relation</arg>
|
<arg>--outputPath</arg><arg>${graphOutputPath}/relation</arg>
|
||||||
<arg>--graphTableClassName</arg><arg>eu.dnetlib.dhp.schema.oaf.Relation</arg>
|
<arg>--graphTableClassName</arg><arg>eu.dnetlib.dhp.schema.oaf.Relation</arg>
|
||||||
<arg>--isLookupUrl</arg><arg>${isLookupUrl}</arg>
|
<arg>--isLookupUrl</arg><arg>${isLookupUrl}</arg>
|
||||||
|
|
Loading…
Reference in New Issue