2020-04-27 14:52:31 +02:00
|
|
|
|
2020-04-10 17:53:07 +02:00
|
|
|
package eu.dnetlib.dhp.oa.graph.raw;
|
2020-03-02 16:12:14 +01:00
|
|
|
|
2020-04-18 12:42:58 +02:00
|
|
|
import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession;
|
|
|
|
|
|
|
|
import java.io.IOException;
|
2020-05-26 13:06:55 +02:00
|
|
|
import java.util.Arrays;
|
|
|
|
import java.util.List;
|
|
|
|
import java.util.Objects;
|
|
|
|
import java.util.Optional;
|
2020-04-18 12:42:58 +02:00
|
|
|
import java.util.stream.Collectors;
|
2020-04-28 11:23:29 +02:00
|
|
|
|
2020-03-02 16:12:14 +01:00
|
|
|
import org.apache.commons.io.IOUtils;
|
|
|
|
import org.apache.commons.lang3.StringUtils;
|
|
|
|
import org.apache.hadoop.fs.FileSystem;
|
|
|
|
import org.apache.hadoop.fs.Path;
|
|
|
|
import org.apache.hadoop.io.Text;
|
2020-03-18 09:34:57 +01:00
|
|
|
import org.apache.hadoop.io.compress.GzipCodec;
|
2020-04-10 17:53:07 +02:00
|
|
|
import org.apache.spark.SparkConf;
|
2020-03-02 16:12:14 +01:00
|
|
|
import org.apache.spark.api.java.JavaRDD;
|
|
|
|
import org.apache.spark.api.java.JavaSparkContext;
|
|
|
|
import org.apache.spark.sql.SparkSession;
|
2020-04-10 17:53:07 +02:00
|
|
|
import org.slf4j.Logger;
|
|
|
|
import org.slf4j.LoggerFactory;
|
2020-04-28 11:23:29 +02:00
|
|
|
|
|
|
|
import com.fasterxml.jackson.databind.ObjectMapper;
|
|
|
|
|
|
|
|
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
|
|
|
|
import eu.dnetlib.dhp.common.HdfsSupport;
|
2020-05-27 11:34:13 +02:00
|
|
|
import eu.dnetlib.dhp.oa.graph.raw.common.VocabularyGroup;
|
2020-04-28 11:23:29 +02:00
|
|
|
import eu.dnetlib.dhp.schema.common.ModelSupport;
|
2020-05-26 13:06:55 +02:00
|
|
|
import eu.dnetlib.dhp.schema.oaf.Dataset;
|
|
|
|
import eu.dnetlib.dhp.schema.oaf.Datasource;
|
|
|
|
import eu.dnetlib.dhp.schema.oaf.Oaf;
|
|
|
|
import eu.dnetlib.dhp.schema.oaf.OafEntity;
|
|
|
|
import eu.dnetlib.dhp.schema.oaf.Organization;
|
|
|
|
import eu.dnetlib.dhp.schema.oaf.OtherResearchProduct;
|
|
|
|
import eu.dnetlib.dhp.schema.oaf.Project;
|
|
|
|
import eu.dnetlib.dhp.schema.oaf.Publication;
|
|
|
|
import eu.dnetlib.dhp.schema.oaf.Relation;
|
|
|
|
import eu.dnetlib.dhp.schema.oaf.Software;
|
2020-04-10 17:53:07 +02:00
|
|
|
import scala.Tuple2;
|
2020-03-02 16:12:14 +01:00
|
|
|
|
|
|
|
public class GenerateEntitiesApplication {
|
|
|
|
|
2020-04-27 14:52:31 +02:00
|
|
|
private static final Logger log = LoggerFactory.getLogger(GenerateEntitiesApplication.class);
|
|
|
|
|
|
|
|
private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
|
|
|
|
|
|
|
|
public static void main(final String[] args) throws Exception {
|
|
|
|
final ArgumentApplicationParser parser = new ArgumentApplicationParser(
|
|
|
|
IOUtils
|
2020-05-26 13:11:09 +02:00
|
|
|
.toString(
|
|
|
|
GenerateEntitiesApplication.class
|
|
|
|
.getResourceAsStream("/eu/dnetlib/dhp/oa/graph/generate_entities_parameters.json")));
|
2020-04-27 14:52:31 +02:00
|
|
|
|
|
|
|
parser.parseArgument(args);
|
|
|
|
|
2020-05-26 13:06:55 +02:00
|
|
|
final Boolean isSparkSessionManaged = Optional
|
2020-04-27 14:52:31 +02:00
|
|
|
.ofNullable(parser.get("isSparkSessionManaged"))
|
|
|
|
.map(Boolean::valueOf)
|
|
|
|
.orElse(Boolean.TRUE);
|
|
|
|
log.info("isSparkSessionManaged: {}", isSparkSessionManaged);
|
|
|
|
|
|
|
|
final String sourcePaths = parser.get("sourcePaths");
|
2020-05-29 18:17:30 +02:00
|
|
|
log.info("sourcePaths: {}", sourcePaths);
|
|
|
|
|
2020-04-27 14:52:31 +02:00
|
|
|
final String targetPath = parser.get("targetPath");
|
2020-05-29 18:17:30 +02:00
|
|
|
log.info("targetPath: {}", targetPath);
|
2020-04-27 14:52:31 +02:00
|
|
|
|
2020-05-29 18:17:30 +02:00
|
|
|
final String isLookupUrl = parser.get("isLookupUrl");
|
|
|
|
log.info("isLookupUrl: {}", isLookupUrl);
|
2020-04-27 14:52:31 +02:00
|
|
|
|
2020-05-29 12:03:51 +02:00
|
|
|
final VocabularyGroup vocs = VocabularyGroup.loadVocsFromIS(isLookupUrl);
|
2020-05-26 13:06:55 +02:00
|
|
|
|
|
|
|
final SparkConf conf = new SparkConf();
|
|
|
|
runWithSparkSession(conf, isSparkSessionManaged, spark -> {
|
|
|
|
removeOutputDir(spark, targetPath);
|
2020-05-27 11:34:13 +02:00
|
|
|
generateEntities(spark, vocs, sourcePaths, targetPath);
|
2020-05-26 13:06:55 +02:00
|
|
|
});
|
2020-04-27 14:52:31 +02:00
|
|
|
}
|
|
|
|
|
|
|
|
private static void generateEntities(
|
|
|
|
final SparkSession spark,
|
2020-05-27 11:34:13 +02:00
|
|
|
final VocabularyGroup vocs,
|
2020-04-27 14:52:31 +02:00
|
|
|
final String sourcePaths,
|
|
|
|
final String targetPath) {
|
|
|
|
|
2020-05-26 13:06:55 +02:00
|
|
|
final JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext());
|
2020-04-27 14:52:31 +02:00
|
|
|
final List<String> existingSourcePaths = Arrays
|
|
|
|
.stream(sourcePaths.split(","))
|
|
|
|
.filter(p -> exists(sc, p))
|
|
|
|
.collect(Collectors.toList());
|
|
|
|
|
|
|
|
log.info("Generate entities from files:");
|
|
|
|
existingSourcePaths.forEach(log::info);
|
|
|
|
|
|
|
|
JavaRDD<Oaf> inputRdd = sc.emptyRDD();
|
|
|
|
|
|
|
|
for (final String sp : existingSourcePaths) {
|
|
|
|
inputRdd = inputRdd
|
2020-05-26 13:11:09 +02:00
|
|
|
.union(
|
|
|
|
sc
|
|
|
|
.sequenceFile(sp, Text.class, Text.class)
|
|
|
|
.map(k -> new Tuple2<>(k._1().toString(), k._2().toString()))
|
2020-05-27 11:34:13 +02:00
|
|
|
.map(k -> convertToListOaf(k._1(), k._2(), vocs))
|
2020-05-26 13:11:09 +02:00
|
|
|
.filter(Objects::nonNull)
|
|
|
|
.flatMap(list -> list.iterator()));
|
2020-04-27 14:52:31 +02:00
|
|
|
}
|
|
|
|
|
|
|
|
inputRdd
|
|
|
|
.mapToPair(oaf -> new Tuple2<>(ModelSupport.idFn().apply(oaf), oaf))
|
|
|
|
.reduceByKey((o1, o2) -> merge(o1, o2))
|
|
|
|
.map(Tuple2::_2)
|
2020-05-26 13:11:09 +02:00
|
|
|
.map(
|
|
|
|
oaf -> oaf.getClass().getSimpleName().toLowerCase()
|
|
|
|
+ "|"
|
|
|
|
+ OBJECT_MAPPER.writeValueAsString(oaf))
|
2020-04-27 14:52:31 +02:00
|
|
|
.saveAsTextFile(targetPath, GzipCodec.class);
|
|
|
|
}
|
|
|
|
|
2020-05-26 13:06:55 +02:00
|
|
|
private static Oaf merge(final Oaf o1, final Oaf o2) {
|
2020-04-27 14:52:31 +02:00
|
|
|
if (ModelSupport.isSubClass(o1, OafEntity.class)) {
|
|
|
|
((OafEntity) o1).mergeFrom((OafEntity) o2);
|
|
|
|
} else if (ModelSupport.isSubClass(o1, Relation.class)) {
|
|
|
|
((Relation) o1).mergeFrom((Relation) o2);
|
|
|
|
} else {
|
|
|
|
throw new RuntimeException("invalid Oaf type:" + o1.getClass().getCanonicalName());
|
|
|
|
}
|
|
|
|
return o1;
|
|
|
|
}
|
|
|
|
|
|
|
|
private static List<Oaf> convertToListOaf(
|
2020-05-26 13:06:55 +02:00
|
|
|
final String id,
|
|
|
|
final String s,
|
2020-05-27 11:34:13 +02:00
|
|
|
final VocabularyGroup vocs) {
|
2020-04-27 14:52:31 +02:00
|
|
|
final String type = StringUtils.substringAfter(id, ":");
|
|
|
|
|
|
|
|
switch (type.toLowerCase()) {
|
2020-06-10 10:04:00 +02:00
|
|
|
case "oaf-store-cleaned":
|
|
|
|
return new OafToOafMapper(vocs, false).processMdRecord(s);
|
|
|
|
case "odf-store-cleaned":
|
|
|
|
return new OdfToOafMapper(vocs, false).processMdRecord(s);
|
|
|
|
case "oaf-store-intersection":
|
|
|
|
return new OafToOafMapper(vocs, true).processMdRecord(s);
|
|
|
|
case "odf-store-intersection":
|
|
|
|
return new OdfToOafMapper(vocs, true).processMdRecord(s);
|
2020-05-26 13:11:09 +02:00
|
|
|
case "datasource":
|
|
|
|
return Arrays.asList(convertFromJson(s, Datasource.class));
|
|
|
|
case "organization":
|
|
|
|
return Arrays.asList(convertFromJson(s, Organization.class));
|
|
|
|
case "project":
|
|
|
|
return Arrays.asList(convertFromJson(s, Project.class));
|
|
|
|
case "relation":
|
|
|
|
return Arrays.asList(convertFromJson(s, Relation.class));
|
|
|
|
case "publication":
|
|
|
|
return Arrays.asList(convertFromJson(s, Publication.class));
|
|
|
|
case "dataset":
|
|
|
|
return Arrays.asList(convertFromJson(s, Dataset.class));
|
|
|
|
case "software":
|
|
|
|
return Arrays.asList(convertFromJson(s, Software.class));
|
|
|
|
case "otherresearchproduct":
|
|
|
|
return Arrays.asList(convertFromJson(s, OtherResearchProduct.class));
|
|
|
|
default:
|
|
|
|
throw new RuntimeException("type not managed: " + type.toLowerCase());
|
2020-04-27 14:52:31 +02:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
private static Oaf convertFromJson(final String s, final Class<? extends Oaf> clazz) {
|
|
|
|
try {
|
|
|
|
return OBJECT_MAPPER.readValue(s, clazz);
|
|
|
|
} catch (final Exception e) {
|
|
|
|
log.error("Error parsing object of class: " + clazz);
|
|
|
|
log.error(s);
|
|
|
|
throw new RuntimeException(e);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
private static boolean exists(final JavaSparkContext context, final String pathToFile) {
|
|
|
|
try {
|
|
|
|
final FileSystem hdfs = FileSystem.get(context.hadoopConfiguration());
|
|
|
|
final Path path = new Path(pathToFile);
|
|
|
|
return hdfs.exists(path);
|
|
|
|
} catch (final IOException e) {
|
|
|
|
throw new RuntimeException(e);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2020-05-26 13:06:55 +02:00
|
|
|
private static void removeOutputDir(final SparkSession spark, final String path) {
|
2020-04-27 14:52:31 +02:00
|
|
|
HdfsSupport.remove(path, spark.sparkContext().hadoopConfiguration());
|
|
|
|
}
|
2020-03-02 16:12:14 +01:00
|
|
|
}
|