2020-04-27 14:52:31 +02:00
|
|
|
|
2020-04-04 14:03:43 +02:00
|
|
|
package eu.dnetlib.dhp.oa.provision;
|
|
|
|
|
2020-04-18 12:42:58 +02:00
|
|
|
import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession;
|
|
|
|
|
|
|
|
import java.util.List;
|
2020-05-28 13:52:30 +02:00
|
|
|
import java.util.Objects;
|
2020-04-18 12:42:58 +02:00
|
|
|
import java.util.Optional;
|
2020-05-27 12:37:33 +02:00
|
|
|
import java.util.stream.Collectors;
|
2020-04-28 11:23:29 +02:00
|
|
|
|
2020-04-04 14:03:43 +02:00
|
|
|
import org.apache.commons.io.IOUtils;
|
2020-04-06 08:59:58 +02:00
|
|
|
import org.apache.commons.lang3.StringUtils;
|
2020-04-04 14:03:43 +02:00
|
|
|
import org.apache.spark.SparkConf;
|
2020-04-06 08:59:58 +02:00
|
|
|
import org.apache.spark.api.java.function.FilterFunction;
|
2020-04-04 14:03:43 +02:00
|
|
|
import org.apache.spark.api.java.function.MapFunction;
|
|
|
|
import org.apache.spark.sql.Dataset;
|
|
|
|
import org.apache.spark.sql.Encoders;
|
|
|
|
import org.apache.spark.sql.SaveMode;
|
|
|
|
import org.apache.spark.sql.SparkSession;
|
|
|
|
import org.slf4j.Logger;
|
|
|
|
import org.slf4j.LoggerFactory;
|
2020-04-28 11:23:29 +02:00
|
|
|
|
|
|
|
import com.fasterxml.jackson.core.JsonProcessingException;
|
|
|
|
import com.fasterxml.jackson.databind.ObjectMapper;
|
2020-05-28 13:52:30 +02:00
|
|
|
import com.google.common.collect.Lists;
|
2020-04-28 11:23:29 +02:00
|
|
|
|
|
|
|
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
|
|
|
|
import eu.dnetlib.dhp.common.HdfsSupport;
|
|
|
|
import eu.dnetlib.dhp.oa.provision.model.EntityRelEntity;
|
2020-05-29 10:58:15 +02:00
|
|
|
import eu.dnetlib.dhp.oa.provision.model.ProvisionModelSupport;
|
2020-04-28 11:23:29 +02:00
|
|
|
import eu.dnetlib.dhp.oa.provision.model.TypedRow;
|
|
|
|
import eu.dnetlib.dhp.schema.common.ModelSupport;
|
|
|
|
import eu.dnetlib.dhp.schema.oaf.*;
|
2020-04-04 14:03:43 +02:00
|
|
|
import scala.Tuple2;
|
2020-04-06 08:59:58 +02:00
|
|
|
import scala.collection.JavaConverters;
|
|
|
|
import scala.collection.Seq;
|
2020-04-04 14:03:43 +02:00
|
|
|
|
|
|
|
/**
|
2020-04-27 14:52:31 +02:00
|
|
|
* Joins the graph nodes by resolving the links of distance = 1 to create an adjacency list of linked objects. The
|
|
|
|
* operation considers all the entity types (publication, dataset, software, ORP, project, datasource, organization, and
|
|
|
|
* all the possible relationships (similarity links produced by the Dedup process are excluded).
|
|
|
|
* <p>
|
|
|
|
* The operation is implemented by sequentially joining one entity type at time (E) with the relationships (R), and
|
|
|
|
* again by E, finally grouped by E.id;
|
|
|
|
* <p>
|
|
|
|
* The workflow is organized in different parts aimed to to reduce the complexity of the operation 1)
|
|
|
|
* PrepareRelationsJob: only consider relationships that are not virtually deleted ($.dataInfo.deletedbyinference ==
|
|
|
|
* false), each entity can be linked at most to 100 other objects
|
|
|
|
* <p>
|
|
|
|
* 2) JoinRelationEntityByTargetJob: (phase 1): prepare tuples [relation - target entity] (R - T): for each entity type
|
|
|
|
* E_i map E_i as RelatedEntity T_i to simplify the model and extracting only the necessary information join (R.target =
|
|
|
|
* T_i.id) save the tuples (R_i, T_i) (phase 2): create the union of all the entity types E, hash by id read the tuples
|
|
|
|
* (R, T), hash by R.source join E.id = (R, T).source, where E becomes the Source Entity S save the tuples (S, R, T)
|
|
|
|
* <p>
|
|
|
|
* 3) AdjacencyListBuilderJob: given the tuple (S - R - T) we need to group by S.id -> List [ R - T ], mapping the
|
|
|
|
* result as JoinedEntity
|
|
|
|
* <p>
|
|
|
|
* 4) XmlConverterJob: convert the JoinedEntities as XML records
|
2020-04-04 14:03:43 +02:00
|
|
|
*/
|
|
|
|
public class CreateRelatedEntitiesJob_phase2 {
|
|
|
|
|
2020-04-27 14:52:31 +02:00
|
|
|
private static final Logger log = LoggerFactory.getLogger(CreateRelatedEntitiesJob_phase2.class);
|
|
|
|
|
|
|
|
private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
|
|
|
|
|
2020-05-27 12:37:33 +02:00
|
|
|
private static final int MAX_EXTERNAL_ENTITIES = 50;
|
2020-05-28 13:52:30 +02:00
|
|
|
private static final int MAX_AUTHORS = 200;
|
2020-05-28 17:36:37 +02:00
|
|
|
private static final int MAX_AUTHOR_FULLNAME_LENGTH = 1000;
|
|
|
|
private static final int MAX_TITLE_LENGTH = 5000;
|
|
|
|
private static final int MAX_ABSTRACT_LENGTH = 100000;
|
2020-05-28 13:52:30 +02:00
|
|
|
|
2020-04-27 14:52:31 +02:00
|
|
|
public static void main(String[] args) throws Exception {
|
|
|
|
|
|
|
|
String jsonConfiguration = IOUtils
|
|
|
|
.toString(
|
|
|
|
PrepareRelationsJob.class
|
|
|
|
.getResourceAsStream(
|
2020-05-29 10:58:15 +02:00
|
|
|
"/eu/dnetlib/dhp/oa/provision/input_params_related_entities_pahase2.json"));
|
2020-04-27 14:52:31 +02:00
|
|
|
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 inputRelatedEntitiesPath = parser.get("inputRelatedEntitiesPath");
|
|
|
|
log.info("inputRelatedEntitiesPath: {}", inputRelatedEntitiesPath);
|
|
|
|
|
2020-05-29 10:58:15 +02:00
|
|
|
String inputEntityPath = parser.get("inputEntityPath");
|
|
|
|
log.info("inputEntityPath: {}", inputEntityPath);
|
2020-04-27 14:52:31 +02:00
|
|
|
|
|
|
|
String outputPath = parser.get("outputPath");
|
|
|
|
log.info("outputPath: {}", outputPath);
|
|
|
|
|
|
|
|
int numPartitions = Integer.parseInt(parser.get("numPartitions"));
|
|
|
|
log.info("numPartitions: {}", numPartitions);
|
|
|
|
|
2020-05-29 10:58:15 +02:00
|
|
|
String graphTableClassName = parser.get("graphTableClassName");
|
|
|
|
log.info("graphTableClassName: {}", graphTableClassName);
|
|
|
|
|
|
|
|
Class<? extends OafEntity> entityClazz = (Class<? extends OafEntity>) Class.forName(graphTableClassName);
|
|
|
|
|
2020-04-27 14:52:31 +02:00
|
|
|
SparkConf conf = new SparkConf();
|
|
|
|
conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer");
|
2020-05-29 10:58:15 +02:00
|
|
|
conf.registerKryoClasses(ProvisionModelSupport.getModelClasses());
|
2020-04-27 14:52:31 +02:00
|
|
|
|
|
|
|
runWithSparkSession(
|
|
|
|
conf,
|
|
|
|
isSparkSessionManaged,
|
|
|
|
spark -> {
|
|
|
|
removeOutputDir(spark, outputPath);
|
2020-05-29 10:58:15 +02:00
|
|
|
joinEntityWithRelatedEntities(
|
|
|
|
spark, inputRelatedEntitiesPath, inputEntityPath, outputPath, numPartitions, entityClazz);
|
2020-04-27 14:52:31 +02:00
|
|
|
});
|
|
|
|
}
|
|
|
|
|
2020-05-29 10:58:15 +02:00
|
|
|
private static <E extends OafEntity> void joinEntityWithRelatedEntities(
|
2020-04-27 14:52:31 +02:00
|
|
|
SparkSession spark,
|
2020-05-29 10:58:15 +02:00
|
|
|
String relatedEntitiesPath,
|
|
|
|
String entityPath,
|
2020-04-27 14:52:31 +02:00
|
|
|
String outputPath,
|
2020-05-29 10:58:15 +02:00
|
|
|
int numPartitions,
|
|
|
|
Class<E> entityClazz) {
|
|
|
|
|
|
|
|
Dataset<Tuple2<String, E>> entity = readPathEntity(spark, entityPath, entityClazz);
|
|
|
|
Dataset<Tuple2<String, EntityRelEntity>> relatedEntities = readRelatedEntities(
|
|
|
|
spark, relatedEntitiesPath, entityClazz);
|
|
|
|
|
|
|
|
entity
|
|
|
|
.joinWith(relatedEntities, entity.col("_1").equalTo(relatedEntities.col("_1")), "left_outer")
|
|
|
|
.map((MapFunction<Tuple2<Tuple2<String, E>, Tuple2<String, EntityRelEntity>>, EntityRelEntity>) value -> {
|
|
|
|
EntityRelEntity re = new EntityRelEntity();
|
|
|
|
re.setEntity(getTypedRow(entityClazz.getCanonicalName().toLowerCase(), value._1()._2()));
|
|
|
|
Optional<EntityRelEntity> related = Optional.ofNullable(value._2()).map(Tuple2::_2);
|
|
|
|
if (related.isPresent()) {
|
|
|
|
re.setRelation(related.get().getRelation());
|
|
|
|
re.setTarget(related.get().getTarget());
|
|
|
|
}
|
|
|
|
return re;
|
|
|
|
}, Encoders.bean(EntityRelEntity.class))
|
2020-04-27 14:52:31 +02:00
|
|
|
.repartition(numPartitions)
|
|
|
|
.filter(
|
|
|
|
(FilterFunction<EntityRelEntity>) value -> value.getEntity() != null
|
|
|
|
&& StringUtils.isNotBlank(value.getEntity().getId()))
|
|
|
|
.write()
|
|
|
|
.mode(SaveMode.Overwrite)
|
|
|
|
.parquet(outputPath);
|
|
|
|
}
|
|
|
|
|
2020-05-29 10:58:15 +02:00
|
|
|
private static <E extends OafEntity> Dataset<Tuple2<String, EntityRelEntity>> readRelatedEntities(
|
|
|
|
SparkSession spark, String inputRelatedEntitiesPath, Class<E> entityClazz) {
|
2020-04-27 14:52:31 +02:00
|
|
|
|
|
|
|
log.info("Reading related entities from: {}", inputRelatedEntitiesPath);
|
|
|
|
|
|
|
|
final List<String> paths = HdfsSupport
|
|
|
|
.listFiles(inputRelatedEntitiesPath, spark.sparkContext().hadoopConfiguration());
|
|
|
|
|
|
|
|
log.info("Found paths: {}", String.join(",", paths));
|
|
|
|
|
2020-05-29 10:58:15 +02:00
|
|
|
final String idPrefix = ModelSupport.getIdPrefix(entityClazz);
|
|
|
|
|
2020-04-27 14:52:31 +02:00
|
|
|
return spark
|
|
|
|
.read()
|
|
|
|
.load(toSeq(paths))
|
|
|
|
.as(Encoders.bean(EntityRelEntity.class))
|
2020-05-29 10:58:15 +02:00
|
|
|
.filter((FilterFunction<EntityRelEntity>) e -> e.getRelation().getSource().startsWith(idPrefix))
|
2020-04-27 14:52:31 +02:00
|
|
|
.map(
|
|
|
|
(MapFunction<EntityRelEntity, Tuple2<String, EntityRelEntity>>) value -> new Tuple2<>(
|
|
|
|
value.getRelation().getSource(), value),
|
|
|
|
Encoders.tuple(Encoders.STRING(), Encoders.kryo(EntityRelEntity.class)));
|
|
|
|
}
|
|
|
|
|
2020-05-29 10:58:15 +02:00
|
|
|
private static <E extends OafEntity> Dataset<Tuple2<String, E>> readPathEntity(
|
2020-04-27 14:52:31 +02:00
|
|
|
SparkSession spark, String inputEntityPath, Class<E> entityClazz) {
|
|
|
|
|
|
|
|
log.info("Reading Graph table from: {}", inputEntityPath);
|
|
|
|
return spark
|
|
|
|
.read()
|
|
|
|
.textFile(inputEntityPath)
|
|
|
|
.map(
|
|
|
|
(MapFunction<String, E>) value -> OBJECT_MAPPER.readValue(value, entityClazz),
|
|
|
|
Encoders.bean(entityClazz))
|
|
|
|
.filter("dataInfo.invisible == false")
|
2020-05-28 17:36:37 +02:00
|
|
|
.map((MapFunction<E, E>) e -> pruneOutliers(entityClazz, e), Encoders.bean(entityClazz))
|
2020-04-27 14:52:31 +02:00
|
|
|
.map(
|
2020-05-29 10:58:15 +02:00
|
|
|
(MapFunction<E, Tuple2<String, E>>) e -> new Tuple2<>(e.getId(), e),
|
|
|
|
Encoders.tuple(Encoders.STRING(), Encoders.kryo(entityClazz)));
|
2020-04-27 14:52:31 +02:00
|
|
|
}
|
|
|
|
|
2020-05-28 17:36:37 +02:00
|
|
|
private static <E extends OafEntity> E pruneOutliers(Class<E> entityClazz, E e) {
|
|
|
|
if (ModelSupport.isSubClass(entityClazz, Result.class)) {
|
|
|
|
Result r = (Result) e;
|
|
|
|
if (r.getExternalReference() != null) {
|
|
|
|
List<ExternalReference> refs = r
|
|
|
|
.getExternalReference()
|
|
|
|
.stream()
|
|
|
|
.limit(MAX_EXTERNAL_ENTITIES)
|
|
|
|
.collect(Collectors.toList());
|
|
|
|
r.setExternalReference(refs);
|
|
|
|
}
|
|
|
|
if (r.getAuthor() != null) {
|
|
|
|
List<Author> authors = Lists.newArrayList();
|
|
|
|
for (Author a : r.getAuthor()) {
|
|
|
|
a.setFullname(StringUtils.left(a.getFullname(), MAX_AUTHOR_FULLNAME_LENGTH));
|
|
|
|
if (authors.size() < MAX_AUTHORS || hasORCID(a)) {
|
|
|
|
authors.add(a);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
r.setAuthor(authors);
|
|
|
|
}
|
|
|
|
if (r.getDescription() != null) {
|
|
|
|
List<Field<String>> desc = r
|
|
|
|
.getDescription()
|
|
|
|
.stream()
|
|
|
|
.filter(Objects::nonNull)
|
|
|
|
.map(d -> {
|
|
|
|
d.setValue(StringUtils.left(d.getValue(), MAX_ABSTRACT_LENGTH));
|
|
|
|
return d;
|
|
|
|
})
|
|
|
|
.collect(Collectors.toList());
|
|
|
|
r.setDescription(desc);
|
|
|
|
}
|
|
|
|
if (r.getTitle() != null) {
|
|
|
|
List<StructuredProperty> titles = r
|
|
|
|
.getTitle()
|
|
|
|
.stream()
|
|
|
|
.filter(Objects::nonNull)
|
|
|
|
.map(t -> {
|
|
|
|
t.setValue(StringUtils.left(t.getValue(), MAX_TITLE_LENGTH));
|
|
|
|
return t;
|
|
|
|
})
|
|
|
|
.collect(Collectors.toList());
|
|
|
|
r.setTitle(titles);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
return e;
|
|
|
|
}
|
|
|
|
|
2020-05-28 13:52:30 +02:00
|
|
|
private static boolean hasORCID(Author a) {
|
|
|
|
return a.getPid() != null && a
|
|
|
|
.getPid()
|
|
|
|
.stream()
|
|
|
|
.filter(Objects::nonNull)
|
|
|
|
.map(StructuredProperty::getQualifier)
|
|
|
|
.filter(Objects::nonNull)
|
|
|
|
.map(Qualifier::getClassid)
|
|
|
|
.filter(StringUtils::isNotBlank)
|
|
|
|
.anyMatch(c -> "orcid".equals(c.toLowerCase()));
|
|
|
|
}
|
|
|
|
|
2020-04-27 14:52:31 +02:00
|
|
|
private static TypedRow getTypedRow(String type, OafEntity entity)
|
|
|
|
throws JsonProcessingException {
|
|
|
|
TypedRow t = new TypedRow();
|
|
|
|
t.setType(type);
|
|
|
|
t.setDeleted(entity.getDataInfo().getDeletedbyinference());
|
|
|
|
t.setId(entity.getId());
|
|
|
|
t.setOaf(OBJECT_MAPPER.writeValueAsString(entity));
|
|
|
|
return t;
|
|
|
|
}
|
|
|
|
|
|
|
|
private static void removeOutputDir(SparkSession spark, String path) {
|
|
|
|
HdfsSupport.remove(path, spark.sparkContext().hadoopConfiguration());
|
|
|
|
}
|
|
|
|
|
|
|
|
private static Seq<String> toSeq(List<String> list) {
|
|
|
|
return JavaConverters.asScalaIteratorConverter(list.iterator()).asScala().toSeq();
|
|
|
|
}
|
2020-04-04 14:03:43 +02:00
|
|
|
}
|