included new stats* workflows in parent pom list of modules, code formatting
This commit is contained in:
parent
bfba71a95c
commit
ef52128c55
|
@ -1,14 +1,14 @@
|
||||||
|
|
||||||
package eu.dnetlib.dhp.actionmanager.promote;
|
package eu.dnetlib.dhp.actionmanager.promote;
|
||||||
|
|
||||||
|
import static eu.dnetlib.dhp.schema.common.ModelSupport.isSubClass;
|
||||||
|
|
||||||
|
import java.util.function.BiFunction;
|
||||||
|
|
||||||
import eu.dnetlib.dhp.common.FunctionalInterfaceSupport.SerializableSupplier;
|
import eu.dnetlib.dhp.common.FunctionalInterfaceSupport.SerializableSupplier;
|
||||||
import eu.dnetlib.dhp.schema.oaf.Oaf;
|
import eu.dnetlib.dhp.schema.oaf.Oaf;
|
||||||
import eu.dnetlib.dhp.schema.oaf.utils.MergeUtils;
|
import eu.dnetlib.dhp.schema.oaf.utils.MergeUtils;
|
||||||
|
|
||||||
import java.util.function.BiFunction;
|
|
||||||
|
|
||||||
import static eu.dnetlib.dhp.schema.common.ModelSupport.isSubClass;
|
|
||||||
|
|
||||||
/** OAF model merging support. */
|
/** OAF model merging support. */
|
||||||
public class MergeAndGet {
|
public class MergeAndGet {
|
||||||
|
|
||||||
|
|
|
@ -100,7 +100,7 @@ public class MergeAndGetTest {
|
||||||
assertTrue(Relation.class.isAssignableFrom(x.getClass()));
|
assertTrue(Relation.class.isAssignableFrom(x.getClass()));
|
||||||
|
|
||||||
// TODO should be reimplemented
|
// TODO should be reimplemented
|
||||||
//verify(a).mergeFrom(b);
|
// verify(a).mergeFrom(b);
|
||||||
assertEquals(a, x);
|
assertEquals(a, x);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -163,7 +163,7 @@ public class MergeAndGetTest {
|
||||||
assertTrue(OafEntity.class.isAssignableFrom(x.getClass()));
|
assertTrue(OafEntity.class.isAssignableFrom(x.getClass()));
|
||||||
|
|
||||||
// TODO should be reimplemented
|
// TODO should be reimplemented
|
||||||
//verify(a).mergeFrom(b);
|
// verify(a).mergeFrom(b);
|
||||||
assertEquals(a, x);
|
assertEquals(a, x);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -95,7 +95,7 @@ public class SparkAtomicActionScoreJob implements Serializable {
|
||||||
|
|
||||||
return projectScores.map((MapFunction<BipProjectModel, Project>) bipProjectScores -> {
|
return projectScores.map((MapFunction<BipProjectModel, Project>) bipProjectScores -> {
|
||||||
Project project = new Project();
|
Project project = new Project();
|
||||||
//project.setId(bipProjectScores.getProjectId());
|
// project.setId(bipProjectScores.getProjectId());
|
||||||
project.setMeasures(bipProjectScores.toMeasures());
|
project.setMeasures(bipProjectScores.toMeasures());
|
||||||
return project;
|
return project;
|
||||||
}, Encoders.bean(Project.class))
|
}, Encoders.bean(Project.class))
|
||||||
|
|
|
@ -38,7 +38,6 @@ public class BipProjectModel {
|
||||||
return projectId;
|
return projectId;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
// each project bip measure has exactly one value, hence one key-value pair
|
// each project bip measure has exactly one value, hence one key-value pair
|
||||||
private Measure createMeasure(String measureId, String measureValue) {
|
private Measure createMeasure(String measureId, String measureValue) {
|
||||||
|
|
||||||
|
|
|
@ -166,6 +166,6 @@ public class MapOCIdsInPids implements Serializable {
|
||||||
.option("compression", "gzip")
|
.option("compression", "gzip")
|
||||||
.json(outputPath);
|
.json(outputPath);
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -1,7 +1,26 @@
|
||||||
|
|
||||||
package eu.dnetlib.dhp.actionmanager.project;
|
package eu.dnetlib.dhp.actionmanager.project;
|
||||||
|
|
||||||
|
import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession;
|
||||||
|
|
||||||
|
import java.util.Arrays;
|
||||||
|
import java.util.Objects;
|
||||||
|
import java.util.Optional;
|
||||||
|
|
||||||
|
import org.apache.commons.io.IOUtils;
|
||||||
|
import org.apache.hadoop.io.Text;
|
||||||
|
import org.apache.hadoop.mapred.SequenceFileOutputFormat;
|
||||||
|
import org.apache.spark.SparkConf;
|
||||||
|
import org.apache.spark.api.java.function.MapFunction;
|
||||||
|
import org.apache.spark.api.java.function.MapGroupsFunction;
|
||||||
|
import org.apache.spark.sql.Dataset;
|
||||||
|
import org.apache.spark.sql.Encoders;
|
||||||
|
import org.apache.spark.sql.SparkSession;
|
||||||
|
import org.slf4j.Logger;
|
||||||
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||||
|
|
||||||
import eu.dnetlib.dhp.actionmanager.project.utils.model.CSVProgramme;
|
import eu.dnetlib.dhp.actionmanager.project.utils.model.CSVProgramme;
|
||||||
import eu.dnetlib.dhp.actionmanager.project.utils.model.CSVProject;
|
import eu.dnetlib.dhp.actionmanager.project.utils.model.CSVProject;
|
||||||
import eu.dnetlib.dhp.actionmanager.project.utils.model.JsonTopic;
|
import eu.dnetlib.dhp.actionmanager.project.utils.model.JsonTopic;
|
||||||
|
@ -15,25 +34,8 @@ import eu.dnetlib.dhp.schema.oaf.OafEntity;
|
||||||
import eu.dnetlib.dhp.schema.oaf.Project;
|
import eu.dnetlib.dhp.schema.oaf.Project;
|
||||||
import eu.dnetlib.dhp.schema.oaf.utils.MergeUtils;
|
import eu.dnetlib.dhp.schema.oaf.utils.MergeUtils;
|
||||||
import eu.dnetlib.dhp.utils.DHPUtils;
|
import eu.dnetlib.dhp.utils.DHPUtils;
|
||||||
import org.apache.commons.io.IOUtils;
|
|
||||||
import org.apache.hadoop.io.Text;
|
|
||||||
import org.apache.hadoop.mapred.SequenceFileOutputFormat;
|
|
||||||
import org.apache.spark.SparkConf;
|
|
||||||
import org.apache.spark.api.java.function.MapFunction;
|
|
||||||
import org.apache.spark.api.java.function.MapGroupsFunction;
|
|
||||||
import org.apache.spark.sql.Dataset;
|
|
||||||
import org.apache.spark.sql.Encoders;
|
|
||||||
import org.apache.spark.sql.SparkSession;
|
|
||||||
import org.slf4j.Logger;
|
|
||||||
import org.slf4j.LoggerFactory;
|
|
||||||
import scala.Tuple2;
|
import scala.Tuple2;
|
||||||
|
|
||||||
import java.util.Arrays;
|
|
||||||
import java.util.Objects;
|
|
||||||
import java.util.Optional;
|
|
||||||
|
|
||||||
import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Class that makes the ActionSet. To prepare the AS two joins are needed
|
* Class that makes the ActionSet. To prepare the AS two joins are needed
|
||||||
*
|
*
|
||||||
|
|
|
@ -79,35 +79,35 @@ public class CreateActionSetSparkJob implements Serializable {
|
||||||
|
|
||||||
private static void createActionSet(SparkSession spark, String inputPath, String outputPath) {
|
private static void createActionSet(SparkSession spark, String inputPath, String outputPath) {
|
||||||
JavaRDD<AtomicAction> relations = spark
|
JavaRDD<AtomicAction> relations = spark
|
||||||
.read()
|
.read()
|
||||||
.textFile(inputPath)
|
.textFile(inputPath)
|
||||||
.map(
|
.map(
|
||||||
(MapFunction<String, TransformativeAgreementModel>) value -> OBJECT_MAPPER
|
(MapFunction<String, TransformativeAgreementModel>) value -> OBJECT_MAPPER
|
||||||
.readValue(value, TransformativeAgreementModel.class),
|
.readValue(value, TransformativeAgreementModel.class),
|
||||||
Encoders.bean(TransformativeAgreementModel.class))
|
Encoders.bean(TransformativeAgreementModel.class))
|
||||||
.flatMap(
|
.flatMap(
|
||||||
(FlatMapFunction<TransformativeAgreementModel, Relation>) value -> createRelation(
|
(FlatMapFunction<TransformativeAgreementModel, Relation>) value -> createRelation(
|
||||||
value)
|
value)
|
||||||
.iterator(),
|
.iterator(),
|
||||||
Encoders.bean(Relation.class))
|
Encoders.bean(Relation.class))
|
||||||
.filter((FilterFunction<Relation>) Objects::nonNull)
|
.filter((FilterFunction<Relation>) Objects::nonNull)
|
||||||
.toJavaRDD()
|
.toJavaRDD()
|
||||||
.map(p -> new AtomicAction(p.getClass(), p));
|
.map(p -> new AtomicAction(p.getClass(), p));
|
||||||
//TODO relations in stand-by waiting to know if we need to create them or not In case we need just make a union before saving the sequence file
|
//TODO relations in stand-by waiting to know if we need to create them or not In case we need just make a union before saving the sequence file
|
||||||
spark
|
spark
|
||||||
.read()
|
.read()
|
||||||
.textFile(inputPath)
|
.textFile(inputPath)
|
||||||
.map(
|
.map(
|
||||||
(MapFunction<String, TransformativeAgreementModel>) value -> OBJECT_MAPPER
|
(MapFunction<String, TransformativeAgreementModel>) value -> OBJECT_MAPPER
|
||||||
.readValue(value, TransformativeAgreementModel.class),
|
.readValue(value, TransformativeAgreementModel.class),
|
||||||
Encoders.bean(TransformativeAgreementModel.class))
|
Encoders.bean(TransformativeAgreementModel.class))
|
||||||
.map(
|
.map(
|
||||||
(MapFunction<TransformativeAgreementModel, Result>) value -> createResult(
|
(MapFunction<TransformativeAgreementModel, Result>) value -> createResult(
|
||||||
value),
|
value),
|
||||||
Encoders.bean(Result.class))
|
Encoders.bean(Result.class))
|
||||||
.filter((FilterFunction<Result>) r -> r != null)
|
.filter((FilterFunction<Result>) r -> r != null)
|
||||||
.toJavaRDD()
|
.toJavaRDD()
|
||||||
.map(p -> new AtomicAction(p.getClass(), p))
|
.map(p -> new AtomicAction(p.getClass(), p))
|
||||||
.mapToPair(
|
.mapToPair(
|
||||||
aa -> new Tuple2<>(new Text(aa.getClazz().getCanonicalName()),
|
aa -> new Tuple2<>(new Text(aa.getClazz().getCanonicalName()),
|
||||||
new Text(OBJECT_MAPPER.writeValueAsString(aa))))
|
new Text(OBJECT_MAPPER.writeValueAsString(aa))))
|
||||||
|
|
|
@ -49,29 +49,29 @@ public class XSLTTransformationFunction implements MapFunction<MetadataRecord, M
|
||||||
public MetadataRecord call(MetadataRecord value) {
|
public MetadataRecord call(MetadataRecord value) {
|
||||||
aggregationCounter.getTotalItems().add(1);
|
aggregationCounter.getTotalItems().add(1);
|
||||||
|
|
||||||
Processor processor = new Processor(false);
|
Processor processor = new Processor(false);
|
||||||
|
|
||||||
processor.registerExtensionFunction(cleanFunction);
|
processor.registerExtensionFunction(cleanFunction);
|
||||||
processor.registerExtensionFunction(new DateCleaner());
|
processor.registerExtensionFunction(new DateCleaner());
|
||||||
processor.registerExtensionFunction(new PersonCleaner());
|
processor.registerExtensionFunction(new PersonCleaner());
|
||||||
|
|
||||||
final XsltCompiler comp = processor.newXsltCompiler();
|
final XsltCompiler comp = processor.newXsltCompiler();
|
||||||
QName datasourceIDParam = new QName(DATASOURCE_ID_PARAM);
|
QName datasourceIDParam = new QName(DATASOURCE_ID_PARAM);
|
||||||
comp.setParameter(datasourceIDParam, new XdmAtomicValue(value.getProvenance().getDatasourceId()));
|
comp.setParameter(datasourceIDParam, new XdmAtomicValue(value.getProvenance().getDatasourceId()));
|
||||||
QName datasourceNameParam = new QName(DATASOURCE_NAME_PARAM);
|
QName datasourceNameParam = new QName(DATASOURCE_NAME_PARAM);
|
||||||
comp.setParameter(datasourceNameParam, new XdmAtomicValue(value.getProvenance().getDatasourceName()));
|
comp.setParameter(datasourceNameParam, new XdmAtomicValue(value.getProvenance().getDatasourceName()));
|
||||||
XsltExecutable xslt;
|
XsltExecutable xslt;
|
||||||
XdmNode source;
|
XdmNode source;
|
||||||
try {
|
try {
|
||||||
xslt = comp
|
xslt = comp
|
||||||
.compile(new StreamSource(IOUtils.toInputStream(transformationRule, StandardCharsets.UTF_8)));
|
.compile(new StreamSource(IOUtils.toInputStream(transformationRule, StandardCharsets.UTF_8)));
|
||||||
source = processor
|
source = processor
|
||||||
.newDocumentBuilder()
|
.newDocumentBuilder()
|
||||||
.build(new StreamSource(IOUtils.toInputStream(value.getBody(), StandardCharsets.UTF_8)));
|
.build(new StreamSource(IOUtils.toInputStream(value.getBody(), StandardCharsets.UTF_8)));
|
||||||
} catch (Throwable e) {
|
} catch (Throwable e) {
|
||||||
throw new RuntimeException("Error on parsing xslt", e);
|
throw new RuntimeException("Error on parsing xslt", e);
|
||||||
}
|
}
|
||||||
try {
|
try {
|
||||||
XsltTransformer trans = xslt.load();
|
XsltTransformer trans = xslt.load();
|
||||||
trans.setInitialContextNode(source);
|
trans.setInitialContextNode(source);
|
||||||
final StringWriter output = new StringWriter();
|
final StringWriter output = new StringWriter();
|
||||||
|
|
|
@ -82,9 +82,9 @@ public class BaseTransfomationTest extends AbstractVocabularyTest {
|
||||||
|
|
||||||
private XSLTTransformationFunction loadTransformationRule(final String path) throws Exception {
|
private XSLTTransformationFunction loadTransformationRule(final String path) throws Exception {
|
||||||
final String xslt = new SAXReader()
|
final String xslt = new SAXReader()
|
||||||
.read(this.getClass().getResourceAsStream(path))
|
.read(this.getClass().getResourceAsStream(path))
|
||||||
.selectSingleNode("//CODE/*")
|
.selectSingleNode("//CODE/*")
|
||||||
.asXML();
|
.asXML();
|
||||||
|
|
||||||
final LongAccumulator la = new LongAccumulator();
|
final LongAccumulator la = new LongAccumulator();
|
||||||
|
|
||||||
|
|
|
@ -281,7 +281,7 @@ class TransformationJobTest extends AbstractVocabularyTest {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testInvalidXSLT() throws Exception{
|
public void testInvalidXSLT() throws Exception {
|
||||||
final MetadataRecord mr = new MetadataRecord();
|
final MetadataRecord mr = new MetadataRecord();
|
||||||
|
|
||||||
mr.setProvenance(new Provenance("openaire____::cnr_explora", "CNR ExploRA", "cnr_________"));
|
mr.setProvenance(new Provenance("openaire____::cnr_explora", "CNR ExploRA", "cnr_________"));
|
||||||
|
@ -289,7 +289,7 @@ class TransformationJobTest extends AbstractVocabularyTest {
|
||||||
// We Load the XSLT transformation Rule from the classpath
|
// We Load the XSLT transformation Rule from the classpath
|
||||||
final XSLTTransformationFunction tr = loadTransformationRule("/eu/dnetlib/dhp/transform/invalid.xslt");
|
final XSLTTransformationFunction tr = loadTransformationRule("/eu/dnetlib/dhp/transform/invalid.xslt");
|
||||||
|
|
||||||
assertThrows(RuntimeException.class,()->tr.call(mr));
|
assertThrows(RuntimeException.class, () -> tr.call(mr));
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -4,7 +4,6 @@ package eu.dnetlib.dhp.oa.dedup;
|
||||||
import java.util.*;
|
import java.util.*;
|
||||||
import java.util.stream.Stream;
|
import java.util.stream.Stream;
|
||||||
|
|
||||||
import eu.dnetlib.dhp.schema.oaf.utils.MergeUtils;
|
|
||||||
import org.apache.commons.beanutils.BeanUtils;
|
import org.apache.commons.beanutils.BeanUtils;
|
||||||
import org.apache.commons.lang3.StringUtils;
|
import org.apache.commons.lang3.StringUtils;
|
||||||
import org.apache.spark.api.java.function.FlatMapFunction;
|
import org.apache.spark.api.java.function.FlatMapFunction;
|
||||||
|
@ -15,200 +14,200 @@ import org.apache.spark.sql.*;
|
||||||
import eu.dnetlib.dhp.oa.dedup.model.Identifier;
|
import eu.dnetlib.dhp.oa.dedup.model.Identifier;
|
||||||
import eu.dnetlib.dhp.oa.merge.AuthorMerger;
|
import eu.dnetlib.dhp.oa.merge.AuthorMerger;
|
||||||
import eu.dnetlib.dhp.schema.common.ModelSupport;
|
import eu.dnetlib.dhp.schema.common.ModelSupport;
|
||||||
import eu.dnetlib.dhp.schema.oaf.utils.MergeUtils;
|
|
||||||
import eu.dnetlib.dhp.schema.oaf.Author;
|
import eu.dnetlib.dhp.schema.oaf.Author;
|
||||||
import eu.dnetlib.dhp.schema.oaf.DataInfo;
|
import eu.dnetlib.dhp.schema.oaf.DataInfo;
|
||||||
import eu.dnetlib.dhp.schema.oaf.OafEntity;
|
import eu.dnetlib.dhp.schema.oaf.OafEntity;
|
||||||
import eu.dnetlib.dhp.schema.oaf.Result;
|
import eu.dnetlib.dhp.schema.oaf.Result;
|
||||||
|
import eu.dnetlib.dhp.schema.oaf.utils.MergeUtils;
|
||||||
import scala.Tuple2;
|
import scala.Tuple2;
|
||||||
import scala.Tuple3;
|
import scala.Tuple3;
|
||||||
import scala.collection.JavaConversions;
|
import scala.collection.JavaConversions;
|
||||||
|
|
||||||
public class DedupRecordFactory {
|
public class DedupRecordFactory {
|
||||||
public static final class DedupRecordReduceState {
|
public static final class DedupRecordReduceState {
|
||||||
public final String dedupId;
|
public final String dedupId;
|
||||||
|
|
||||||
public final ArrayList<String> aliases = new ArrayList<>();
|
public final ArrayList<String> aliases = new ArrayList<>();
|
||||||
|
|
||||||
public final HashSet<String> acceptanceDate = new HashSet<>();
|
public final HashSet<String> acceptanceDate = new HashSet<>();
|
||||||
|
|
||||||
public OafEntity entity;
|
public OafEntity entity;
|
||||||
|
|
||||||
public DedupRecordReduceState(String dedupId, String id, OafEntity entity) {
|
public DedupRecordReduceState(String dedupId, String id, OafEntity entity) {
|
||||||
this.dedupId = dedupId;
|
this.dedupId = dedupId;
|
||||||
this.entity = entity;
|
this.entity = entity;
|
||||||
if (entity == null) {
|
if (entity == null) {
|
||||||
aliases.add(id);
|
aliases.add(id);
|
||||||
} else {
|
} else {
|
||||||
if (Result.class.isAssignableFrom(entity.getClass())) {
|
if (Result.class.isAssignableFrom(entity.getClass())) {
|
||||||
Result result = (Result) entity;
|
Result result = (Result) entity;
|
||||||
if (result.getDateofacceptance() != null
|
if (result.getDateofacceptance() != null
|
||||||
&& StringUtils.isNotBlank(result.getDateofacceptance().getValue())) {
|
&& StringUtils.isNotBlank(result.getDateofacceptance().getValue())) {
|
||||||
acceptanceDate.add(result.getDateofacceptance().getValue());
|
acceptanceDate.add(result.getDateofacceptance().getValue());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
public String getDedupId() {
|
public String getDedupId() {
|
||||||
return dedupId;
|
return dedupId;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private static final int MAX_ACCEPTANCE_DATE = 20;
|
private static final int MAX_ACCEPTANCE_DATE = 20;
|
||||||
|
|
||||||
private DedupRecordFactory() {
|
private DedupRecordFactory() {
|
||||||
}
|
}
|
||||||
|
|
||||||
public static Dataset<OafEntity> createDedupRecord(
|
public static Dataset<OafEntity> createDedupRecord(
|
||||||
final SparkSession spark,
|
final SparkSession spark,
|
||||||
final DataInfo dataInfo,
|
final DataInfo dataInfo,
|
||||||
final String mergeRelsInputPath,
|
final String mergeRelsInputPath,
|
||||||
final String entitiesInputPath,
|
final String entitiesInputPath,
|
||||||
final Class<OafEntity> clazz) {
|
final Class<OafEntity> clazz) {
|
||||||
|
|
||||||
final long ts = System.currentTimeMillis();
|
final long ts = System.currentTimeMillis();
|
||||||
final Encoder<OafEntity> beanEncoder = Encoders.bean(clazz);
|
final Encoder<OafEntity> beanEncoder = Encoders.bean(clazz);
|
||||||
final Encoder<OafEntity> kryoEncoder = Encoders.kryo(clazz);
|
final Encoder<OafEntity> kryoEncoder = Encoders.kryo(clazz);
|
||||||
|
|
||||||
// <id, json_entity>
|
// <id, json_entity>
|
||||||
Dataset<Row> entities = spark
|
Dataset<Row> entities = spark
|
||||||
.read()
|
.read()
|
||||||
.schema(Encoders.bean(clazz).schema())
|
.schema(Encoders.bean(clazz).schema())
|
||||||
.json(entitiesInputPath)
|
.json(entitiesInputPath)
|
||||||
.as(beanEncoder)
|
.as(beanEncoder)
|
||||||
.map(
|
.map(
|
||||||
(MapFunction<OafEntity, Tuple2<String, OafEntity>>) entity -> {
|
(MapFunction<OafEntity, Tuple2<String, OafEntity>>) entity -> {
|
||||||
return new Tuple2<>(entity.getId(), entity);
|
return new Tuple2<>(entity.getId(), entity);
|
||||||
},
|
},
|
||||||
Encoders.tuple(Encoders.STRING(), kryoEncoder))
|
Encoders.tuple(Encoders.STRING(), kryoEncoder))
|
||||||
.selectExpr("_1 AS id", "_2 AS kryoObject");
|
.selectExpr("_1 AS id", "_2 AS kryoObject");
|
||||||
|
|
||||||
// <source, target>: source is the dedup_id, target is the id of the mergedIn
|
// <source, target>: source is the dedup_id, target is the id of the mergedIn
|
||||||
Dataset<Row> mergeRels = spark
|
Dataset<Row> mergeRels = spark
|
||||||
.read()
|
.read()
|
||||||
.load(mergeRelsInputPath)
|
.load(mergeRelsInputPath)
|
||||||
.where("relClass == 'merges'")
|
.where("relClass == 'merges'")
|
||||||
.selectExpr("source as dedupId", "target as id");
|
.selectExpr("source as dedupId", "target as id");
|
||||||
|
|
||||||
return mergeRels
|
return mergeRels
|
||||||
.join(entities, JavaConversions.asScalaBuffer(Collections.singletonList("id")), "left")
|
.join(entities, JavaConversions.asScalaBuffer(Collections.singletonList("id")), "left")
|
||||||
.select("dedupId", "id", "kryoObject")
|
.select("dedupId", "id", "kryoObject")
|
||||||
.as(Encoders.tuple(Encoders.STRING(), Encoders.STRING(), kryoEncoder))
|
.as(Encoders.tuple(Encoders.STRING(), Encoders.STRING(), kryoEncoder))
|
||||||
.map(
|
.map(
|
||||||
(MapFunction<Tuple3<String, String, OafEntity>, DedupRecordReduceState>) t -> new DedupRecordReduceState(
|
(MapFunction<Tuple3<String, String, OafEntity>, DedupRecordReduceState>) t -> new DedupRecordReduceState(
|
||||||
t._1(), t._2(), t._3()),
|
t._1(), t._2(), t._3()),
|
||||||
Encoders.kryo(DedupRecordReduceState.class))
|
Encoders.kryo(DedupRecordReduceState.class))
|
||||||
.groupByKey(
|
.groupByKey(
|
||||||
(MapFunction<DedupRecordReduceState, String>) DedupRecordReduceState::getDedupId, Encoders.STRING())
|
(MapFunction<DedupRecordReduceState, String>) DedupRecordReduceState::getDedupId, Encoders.STRING())
|
||||||
.reduceGroups(
|
.reduceGroups(
|
||||||
(ReduceFunction<DedupRecordReduceState>) (t1, t2) -> {
|
(ReduceFunction<DedupRecordReduceState>) (t1, t2) -> {
|
||||||
if (t1.entity == null) {
|
if (t1.entity == null) {
|
||||||
t2.aliases.addAll(t1.aliases);
|
t2.aliases.addAll(t1.aliases);
|
||||||
return t2;
|
return t2;
|
||||||
}
|
}
|
||||||
if (t1.acceptanceDate.size() < MAX_ACCEPTANCE_DATE) {
|
if (t1.acceptanceDate.size() < MAX_ACCEPTANCE_DATE) {
|
||||||
t1.acceptanceDate.addAll(t2.acceptanceDate);
|
t1.acceptanceDate.addAll(t2.acceptanceDate);
|
||||||
}
|
}
|
||||||
t1.aliases.addAll(t2.aliases);
|
t1.aliases.addAll(t2.aliases);
|
||||||
t1.entity = reduceEntity(t1.entity, t2.entity);
|
t1.entity = reduceEntity(t1.entity, t2.entity);
|
||||||
|
|
||||||
return t1;
|
return t1;
|
||||||
})
|
})
|
||||||
.flatMap((FlatMapFunction<Tuple2<String, DedupRecordReduceState>, OafEntity>) t -> {
|
.flatMap((FlatMapFunction<Tuple2<String, DedupRecordReduceState>, OafEntity>) t -> {
|
||||||
String dedupId = t._1();
|
String dedupId = t._1();
|
||||||
DedupRecordReduceState agg = t._2();
|
DedupRecordReduceState agg = t._2();
|
||||||
|
|
||||||
if (agg.acceptanceDate.size() >= MAX_ACCEPTANCE_DATE) {
|
if (agg.acceptanceDate.size() >= MAX_ACCEPTANCE_DATE) {
|
||||||
return Collections.emptyIterator();
|
return Collections.emptyIterator();
|
||||||
}
|
}
|
||||||
|
|
||||||
return Stream
|
return Stream
|
||||||
.concat(
|
.concat(
|
||||||
Stream
|
Stream
|
||||||
.of(agg.getDedupId())
|
.of(agg.getDedupId())
|
||||||
.map(id -> createDedupOafEntity(id, agg.entity, dataInfo, ts)),
|
.map(id -> createDedupOafEntity(id, agg.entity, dataInfo, ts)),
|
||||||
agg.aliases
|
agg.aliases
|
||||||
.stream()
|
.stream()
|
||||||
.map(id -> createMergedDedupAliasOafEntity(id, agg.entity, dataInfo, ts)))
|
.map(id -> createMergedDedupAliasOafEntity(id, agg.entity, dataInfo, ts)))
|
||||||
.iterator();
|
.iterator();
|
||||||
}, beanEncoder);
|
}, beanEncoder);
|
||||||
}
|
}
|
||||||
|
|
||||||
private static OafEntity createDedupOafEntity(String id, OafEntity base, DataInfo dataInfo, long ts) {
|
private static OafEntity createDedupOafEntity(String id, OafEntity base, DataInfo dataInfo, long ts) {
|
||||||
try {
|
try {
|
||||||
OafEntity res = (OafEntity) BeanUtils.cloneBean(base);
|
OafEntity res = (OafEntity) BeanUtils.cloneBean(base);
|
||||||
res.setId(id);
|
res.setId(id);
|
||||||
res.setDataInfo(dataInfo);
|
res.setDataInfo(dataInfo);
|
||||||
res.setLastupdatetimestamp(ts);
|
res.setLastupdatetimestamp(ts);
|
||||||
return res;
|
return res;
|
||||||
} catch (Exception e) {
|
} catch (Exception e) {
|
||||||
throw new RuntimeException(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private static OafEntity createMergedDedupAliasOafEntity(String id, OafEntity base, DataInfo dataInfo, long ts) {
|
private static OafEntity createMergedDedupAliasOafEntity(String id, OafEntity base, DataInfo dataInfo, long ts) {
|
||||||
try {
|
try {
|
||||||
OafEntity res = createDedupOafEntity(id, base, dataInfo, ts);
|
OafEntity res = createDedupOafEntity(id, base, dataInfo, ts);
|
||||||
DataInfo ds = (DataInfo) BeanUtils.cloneBean(dataInfo);
|
DataInfo ds = (DataInfo) BeanUtils.cloneBean(dataInfo);
|
||||||
ds.setDeletedbyinference(true);
|
ds.setDeletedbyinference(true);
|
||||||
res.setDataInfo(ds);
|
res.setDataInfo(ds);
|
||||||
return res;
|
return res;
|
||||||
} catch (Exception e) {
|
} catch (Exception e) {
|
||||||
throw new RuntimeException(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private static OafEntity reduceEntity(OafEntity entity, OafEntity duplicate) {
|
private static OafEntity reduceEntity(OafEntity entity, OafEntity duplicate) {
|
||||||
|
|
||||||
if (duplicate == null) {
|
if (duplicate == null) {
|
||||||
return entity;
|
return entity;
|
||||||
}
|
}
|
||||||
|
|
||||||
int compare = new IdentifierComparator<>()
|
int compare = new IdentifierComparator<>()
|
||||||
.compare(Identifier.newInstance(entity), Identifier.newInstance(duplicate));
|
.compare(Identifier.newInstance(entity), Identifier.newInstance(duplicate));
|
||||||
|
|
||||||
if (compare > 0) {
|
if (compare > 0) {
|
||||||
OafEntity swap = duplicate;
|
OafEntity swap = duplicate;
|
||||||
duplicate = entity;
|
duplicate = entity;
|
||||||
entity = swap;
|
entity = swap;
|
||||||
}
|
}
|
||||||
|
|
||||||
entity = MergeUtils.checkedMerge(entity, duplicate);
|
entity = MergeUtils.checkedMerge(entity, duplicate);
|
||||||
|
|
||||||
if (ModelSupport.isSubClass(duplicate, Result.class)) {
|
if (ModelSupport.isSubClass(duplicate, Result.class)) {
|
||||||
Result re = (Result) entity;
|
Result re = (Result) entity;
|
||||||
Result rd = (Result) duplicate;
|
Result rd = (Result) duplicate;
|
||||||
|
|
||||||
List<List<Author>> authors = new ArrayList<>();
|
List<List<Author>> authors = new ArrayList<>();
|
||||||
if (re.getAuthor() != null) {
|
if (re.getAuthor() != null) {
|
||||||
authors.add(re.getAuthor());
|
authors.add(re.getAuthor());
|
||||||
}
|
}
|
||||||
if (rd.getAuthor() != null) {
|
if (rd.getAuthor() != null) {
|
||||||
authors.add(rd.getAuthor());
|
authors.add(rd.getAuthor());
|
||||||
}
|
}
|
||||||
|
|
||||||
re.setAuthor(AuthorMerger.merge(authors));
|
re.setAuthor(AuthorMerger.merge(authors));
|
||||||
}
|
}
|
||||||
|
|
||||||
return entity;
|
return entity;
|
||||||
}
|
}
|
||||||
|
|
||||||
public static <T extends OafEntity> T entityMerger(
|
public static <T extends OafEntity> T entityMerger(
|
||||||
String id, Iterator<Tuple2<String, T>> entities, long ts, DataInfo dataInfo, Class<T> clazz) {
|
String id, Iterator<Tuple2<String, T>> entities, long ts, DataInfo dataInfo, Class<T> clazz) {
|
||||||
T base = entities.next()._2();
|
T base = entities.next()._2();
|
||||||
|
|
||||||
while (entities.hasNext()) {
|
while (entities.hasNext()) {
|
||||||
T duplicate = entities.next()._2();
|
T duplicate = entities.next()._2();
|
||||||
if (duplicate != null)
|
if (duplicate != null)
|
||||||
base = (T) reduceEntity(base, duplicate);
|
base = (T) reduceEntity(base, duplicate);
|
||||||
}
|
}
|
||||||
|
|
||||||
base.setId(id);
|
base.setId(id);
|
||||||
base.setDataInfo(dataInfo);
|
base.setDataInfo(dataInfo);
|
||||||
base.setLastupdatetimestamp(ts);
|
base.setLastupdatetimestamp(ts);
|
||||||
|
|
||||||
return base;
|
return base;
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -3,7 +3,6 @@ package eu.dnetlib.dhp.oa.dedup;
|
||||||
|
|
||||||
import static org.apache.spark.sql.functions.col;
|
import static org.apache.spark.sql.functions.col;
|
||||||
|
|
||||||
import eu.dnetlib.dhp.schema.oaf.utils.MergeUtils;
|
|
||||||
import org.apache.commons.io.IOUtils;
|
import org.apache.commons.io.IOUtils;
|
||||||
import org.apache.spark.SparkConf;
|
import org.apache.spark.SparkConf;
|
||||||
import org.apache.spark.api.java.function.MapFunction;
|
import org.apache.spark.api.java.function.MapFunction;
|
||||||
|
@ -21,6 +20,7 @@ import eu.dnetlib.dhp.schema.common.ModelConstants;
|
||||||
import eu.dnetlib.dhp.schema.common.ModelSupport;
|
import eu.dnetlib.dhp.schema.common.ModelSupport;
|
||||||
import eu.dnetlib.dhp.schema.oaf.DataInfo;
|
import eu.dnetlib.dhp.schema.oaf.DataInfo;
|
||||||
import eu.dnetlib.dhp.schema.oaf.Relation;
|
import eu.dnetlib.dhp.schema.oaf.Relation;
|
||||||
|
import eu.dnetlib.dhp.schema.oaf.utils.MergeUtils;
|
||||||
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 scala.Tuple2;
|
import scala.Tuple2;
|
||||||
|
@ -128,8 +128,7 @@ public class SparkPropagateRelation extends AbstractSparkAction {
|
||||||
(MapFunction<Relation, String>) r -> String
|
(MapFunction<Relation, String>) r -> String
|
||||||
.join(" ", r.getSource(), r.getTarget(), r.getRelType(), r.getSubRelType(), r.getRelClass()),
|
.join(" ", r.getSource(), r.getTarget(), r.getRelType(), r.getSubRelType(), r.getRelClass()),
|
||||||
Encoders.STRING())
|
Encoders.STRING())
|
||||||
.reduceGroups((ReduceFunction<Relation>) MergeUtils::mergeRelation
|
.reduceGroups((ReduceFunction<Relation>) MergeUtils::mergeRelation)
|
||||||
)
|
|
||||||
.map((MapFunction<Tuple2<String, Relation>, Relation>) Tuple2::_2, REL_BEAN_ENC);
|
.map((MapFunction<Tuple2<String, Relation>, Relation>) Tuple2::_2, REL_BEAN_ENC);
|
||||||
|
|
||||||
final String outputRelationPath = graphOutputPath + "/relation";
|
final String outputRelationPath = graphOutputPath + "/relation";
|
||||||
|
|
|
@ -13,7 +13,6 @@ import java.util.List;
|
||||||
import java.util.Optional;
|
import java.util.Optional;
|
||||||
import java.util.stream.Collectors;
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
import eu.dnetlib.dhp.schema.oaf.utils.MergeUtils;
|
|
||||||
import org.apache.commons.io.IOUtils;
|
import org.apache.commons.io.IOUtils;
|
||||||
import org.apache.spark.SparkConf;
|
import org.apache.spark.SparkConf;
|
||||||
import org.apache.spark.api.java.function.MapFunction;
|
import org.apache.spark.api.java.function.MapFunction;
|
||||||
|
@ -29,6 +28,7 @@ import eu.dnetlib.dhp.schema.common.ModelConstants;
|
||||||
import eu.dnetlib.dhp.schema.common.ModelSupport;
|
import eu.dnetlib.dhp.schema.common.ModelSupport;
|
||||||
import eu.dnetlib.dhp.schema.oaf.Context;
|
import eu.dnetlib.dhp.schema.oaf.Context;
|
||||||
import eu.dnetlib.dhp.schema.oaf.Result;
|
import eu.dnetlib.dhp.schema.oaf.Result;
|
||||||
|
import eu.dnetlib.dhp.schema.oaf.utils.MergeUtils;
|
||||||
import scala.Tuple2;
|
import scala.Tuple2;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
@ -25,118 +25,118 @@ import scala.Tuple2;
|
||||||
|
|
||||||
public class SparkResultToCommunityThroughSemRelJob {
|
public class SparkResultToCommunityThroughSemRelJob {
|
||||||
|
|
||||||
private static final Logger log = LoggerFactory.getLogger(SparkResultToCommunityThroughSemRelJob.class);
|
private static final Logger log = LoggerFactory.getLogger(SparkResultToCommunityThroughSemRelJob.class);
|
||||||
|
|
||||||
public static void main(String[] args) throws Exception {
|
public static void main(String[] args) throws Exception {
|
||||||
|
|
||||||
String jsonConfiguration = IOUtils
|
String jsonConfiguration = IOUtils
|
||||||
.toString(
|
.toString(
|
||||||
SparkResultToCommunityThroughSemRelJob.class
|
SparkResultToCommunityThroughSemRelJob.class
|
||||||
.getResourceAsStream(
|
.getResourceAsStream(
|
||||||
"/eu/dnetlib/dhp/wf/subworkflows/resulttocommunityfromsemrel/input_communitytoresult_parameters.json"));
|
"/eu/dnetlib/dhp/wf/subworkflows/resulttocommunityfromsemrel/input_communitytoresult_parameters.json"));
|
||||||
|
|
||||||
final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration);
|
final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration);
|
||||||
|
|
||||||
parser.parseArgument(args);
|
parser.parseArgument(args);
|
||||||
|
|
||||||
Boolean isSparkSessionManaged = isSparkSessionManaged(parser);
|
Boolean isSparkSessionManaged = isSparkSessionManaged(parser);
|
||||||
log.info("isSparkSessionManaged: {}", isSparkSessionManaged);
|
log.info("isSparkSessionManaged: {}", isSparkSessionManaged);
|
||||||
|
|
||||||
String inputPath = parser.get("sourcePath");
|
String inputPath = parser.get("sourcePath");
|
||||||
log.info("inputPath: {}", inputPath);
|
log.info("inputPath: {}", inputPath);
|
||||||
|
|
||||||
final String outputPath = parser.get("outputPath");
|
final String outputPath = parser.get("outputPath");
|
||||||
log.info("outputPath: {}", outputPath);
|
log.info("outputPath: {}", outputPath);
|
||||||
|
|
||||||
final String preparedInfoPath = parser.get("preparedInfoPath");
|
final String preparedInfoPath = parser.get("preparedInfoPath");
|
||||||
log.info("preparedInfoPath: {}", preparedInfoPath);
|
log.info("preparedInfoPath: {}", preparedInfoPath);
|
||||||
|
|
||||||
SparkConf conf = new SparkConf();
|
SparkConf conf = new SparkConf();
|
||||||
conf.set("hive.metastore.uris", parser.get("hive_metastore_uris"));
|
conf.set("hive.metastore.uris", parser.get("hive_metastore_uris"));
|
||||||
|
|
||||||
final String resultClassName = parser.get("resultTableName");
|
final String resultClassName = parser.get("resultTableName");
|
||||||
log.info("resultTableName: {}", resultClassName);
|
log.info("resultTableName: {}", resultClassName);
|
||||||
|
|
||||||
final Boolean saveGraph = Optional
|
final Boolean saveGraph = Optional
|
||||||
.ofNullable(parser.get("saveGraph"))
|
.ofNullable(parser.get("saveGraph"))
|
||||||
.map(Boolean::valueOf)
|
.map(Boolean::valueOf)
|
||||||
.orElse(Boolean.TRUE);
|
.orElse(Boolean.TRUE);
|
||||||
log.info("saveGraph: {}", saveGraph);
|
log.info("saveGraph: {}", saveGraph);
|
||||||
|
|
||||||
@SuppressWarnings("unchecked")
|
@SuppressWarnings("unchecked")
|
||||||
Class<? extends Result> resultClazz = (Class<? extends Result>) Class.forName(resultClassName);
|
Class<? extends Result> resultClazz = (Class<? extends Result>) Class.forName(resultClassName);
|
||||||
|
|
||||||
runWithSparkHiveSession(
|
runWithSparkHiveSession(
|
||||||
conf,
|
conf,
|
||||||
isSparkSessionManaged,
|
isSparkSessionManaged,
|
||||||
spark -> {
|
spark -> {
|
||||||
if (isTest(parser)) {
|
if (isTest(parser)) {
|
||||||
removeOutputDir(spark, outputPath);
|
removeOutputDir(spark, outputPath);
|
||||||
}
|
}
|
||||||
if (saveGraph) {
|
if (saveGraph) {
|
||||||
execPropagation(
|
execPropagation(
|
||||||
spark, inputPath, outputPath, preparedInfoPath, resultClazz);
|
spark, inputPath, outputPath, preparedInfoPath, resultClazz);
|
||||||
}
|
}
|
||||||
});
|
});
|
||||||
}
|
}
|
||||||
|
|
||||||
private static <R extends Result> void execPropagation(
|
private static <R extends Result> void execPropagation(
|
||||||
SparkSession spark,
|
SparkSession spark,
|
||||||
String inputPath,
|
String inputPath,
|
||||||
String outputPath,
|
String outputPath,
|
||||||
String preparedInfoPath,
|
String preparedInfoPath,
|
||||||
Class<R> resultClazz) {
|
Class<R> resultClazz) {
|
||||||
|
|
||||||
Dataset<ResultCommunityList> possibleUpdates = readPath(spark, preparedInfoPath, ResultCommunityList.class);
|
Dataset<ResultCommunityList> possibleUpdates = readPath(spark, preparedInfoPath, ResultCommunityList.class);
|
||||||
Dataset<R> result = readPath(spark, inputPath, resultClazz);
|
Dataset<R> result = readPath(spark, inputPath, resultClazz);
|
||||||
|
|
||||||
result
|
result
|
||||||
.joinWith(
|
.joinWith(
|
||||||
possibleUpdates,
|
possibleUpdates,
|
||||||
result.col("id").equalTo(possibleUpdates.col("resultId")),
|
result.col("id").equalTo(possibleUpdates.col("resultId")),
|
||||||
"left_outer")
|
"left_outer")
|
||||||
.map(contextUpdaterFn(), Encoders.bean(resultClazz))
|
.map(contextUpdaterFn(), Encoders.bean(resultClazz))
|
||||||
.write()
|
.write()
|
||||||
.mode(SaveMode.Overwrite)
|
.mode(SaveMode.Overwrite)
|
||||||
.option("compression", "gzip")
|
.option("compression", "gzip")
|
||||||
.json(outputPath);
|
.json(outputPath);
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
private static <R extends Result> MapFunction<Tuple2<R, ResultCommunityList>, R> contextUpdaterFn() {
|
private static <R extends Result> MapFunction<Tuple2<R, ResultCommunityList>, R> contextUpdaterFn() {
|
||||||
return value -> {
|
return value -> {
|
||||||
R ret = value._1();
|
R ret = value._1();
|
||||||
Optional<ResultCommunityList> rcl = Optional.ofNullable(value._2());
|
Optional<ResultCommunityList> rcl = Optional.ofNullable(value._2());
|
||||||
if (rcl.isPresent()) {
|
if (rcl.isPresent()) {
|
||||||
Set<String> contexts = new HashSet<>();
|
Set<String> contexts = new HashSet<>();
|
||||||
ret.getContext().forEach(c -> contexts.add(c.getId()));
|
ret.getContext().forEach(c -> contexts.add(c.getId()));
|
||||||
rcl
|
rcl
|
||||||
.get()
|
.get()
|
||||||
.getCommunityList()
|
.getCommunityList()
|
||||||
.stream()
|
.stream()
|
||||||
.forEach(
|
.forEach(
|
||||||
c -> {
|
c -> {
|
||||||
if (!contexts.contains(c)) {
|
if (!contexts.contains(c)) {
|
||||||
Context newContext = new Context();
|
Context newContext = new Context();
|
||||||
newContext.setId(c);
|
newContext.setId(c);
|
||||||
newContext
|
newContext
|
||||||
.setDataInfo(
|
.setDataInfo(
|
||||||
Arrays
|
Arrays
|
||||||
.asList(
|
.asList(
|
||||||
getDataInfo(
|
getDataInfo(
|
||||||
PROPAGATION_DATA_INFO_TYPE,
|
PROPAGATION_DATA_INFO_TYPE,
|
||||||
PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID,
|
PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID,
|
||||||
PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME,
|
PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME,
|
||||||
ModelConstants.DNET_PROVENANCE_ACTIONS)));
|
ModelConstants.DNET_PROVENANCE_ACTIONS)));
|
||||||
ret.getContext().add(newContext);
|
ret.getContext().add(newContext);
|
||||||
}
|
}
|
||||||
|
|
||||||
});
|
});
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
return ret;
|
return ret;
|
||||||
};
|
};
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -7,7 +7,6 @@ import java.util.*;
|
||||||
import java.util.stream.Collectors;
|
import java.util.stream.Collectors;
|
||||||
import java.util.stream.Stream;
|
import java.util.stream.Stream;
|
||||||
|
|
||||||
import eu.dnetlib.dhp.schema.oaf.utils.MergeUtils;
|
|
||||||
import org.apache.commons.io.IOUtils;
|
import org.apache.commons.io.IOUtils;
|
||||||
import org.apache.spark.SparkConf;
|
import org.apache.spark.SparkConf;
|
||||||
import org.apache.spark.api.java.function.FilterFunction;
|
import org.apache.spark.api.java.function.FilterFunction;
|
||||||
|
@ -25,6 +24,7 @@ import eu.dnetlib.dhp.common.HdfsSupport;
|
||||||
import eu.dnetlib.dhp.schema.common.ModelConstants;
|
import eu.dnetlib.dhp.schema.common.ModelConstants;
|
||||||
import eu.dnetlib.dhp.schema.common.ModelSupport;
|
import eu.dnetlib.dhp.schema.common.ModelSupport;
|
||||||
import eu.dnetlib.dhp.schema.oaf.*;
|
import eu.dnetlib.dhp.schema.oaf.*;
|
||||||
|
import eu.dnetlib.dhp.schema.oaf.utils.MergeUtils;
|
||||||
import scala.Tuple2;
|
import scala.Tuple2;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -251,7 +251,7 @@ public class MergeGraphTableSparkJob {
|
||||||
return (T) MergeUtils.merge(b, a);
|
return (T) MergeUtils.merge(b, a);
|
||||||
}
|
}
|
||||||
if (a instanceof Relation && b instanceof Relation) {
|
if (a instanceof Relation && b instanceof Relation) {
|
||||||
return (T) MergeUtils.mergeRelation((Relation)a, (Relation) b);
|
return (T) MergeUtils.mergeRelation((Relation) a, (Relation) b);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
return Objects.isNull(a) ? b : a;
|
return Objects.isNull(a) ? b : a;
|
||||||
|
|
|
@ -98,8 +98,10 @@ public abstract class AbstractMdRecordToOafMapper {
|
||||||
protected static final String DATACITE_SCHEMA_KERNEL_3 = "http://datacite.org/schema/kernel-3";
|
protected static final String DATACITE_SCHEMA_KERNEL_3 = "http://datacite.org/schema/kernel-3";
|
||||||
protected static final String DATACITE_SCHEMA_KERNEL_3_SLASH = "http://datacite.org/schema/kernel-3/";
|
protected static final String DATACITE_SCHEMA_KERNEL_3_SLASH = "http://datacite.org/schema/kernel-3/";
|
||||||
|
|
||||||
protected static final Qualifier ORCID_PID_TYPE = qualifier(ModelConstants.ORCID_PENDING, ModelConstants.ORCID_CLASSNAME, DNET_PID_TYPES, DNET_PID_TYPES);
|
protected static final Qualifier ORCID_PID_TYPE = qualifier(
|
||||||
protected static final Qualifier MAG_PID_TYPE = qualifier("MAGIdentifier", "Microsoft Academic Graph Identifier", DNET_PID_TYPES, DNET_PID_TYPES);
|
ModelConstants.ORCID_PENDING, ModelConstants.ORCID_CLASSNAME, DNET_PID_TYPES, DNET_PID_TYPES);
|
||||||
|
protected static final Qualifier MAG_PID_TYPE = qualifier(
|
||||||
|
"MAGIdentifier", "Microsoft Academic Graph Identifier", DNET_PID_TYPES, DNET_PID_TYPES);
|
||||||
|
|
||||||
protected static final String DEFAULT_TRUST_FOR_VALIDATED_RELS = "0.999";
|
protected static final String DEFAULT_TRUST_FOR_VALIDATED_RELS = "0.999";
|
||||||
|
|
||||||
|
@ -122,14 +124,14 @@ public abstract class AbstractMdRecordToOafMapper {
|
||||||
|
|
||||||
static {
|
static {
|
||||||
IdentifierFactory.PID_AUTHORITY
|
IdentifierFactory.PID_AUTHORITY
|
||||||
.keySet()
|
.keySet()
|
||||||
.stream()
|
.stream()
|
||||||
.forEach(entry -> pidTypeWithAuthority.put(entry.toString().toLowerCase(), entry.toString()));
|
.forEach(entry -> pidTypeWithAuthority.put(entry.toString().toLowerCase(), entry.toString()));
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
protected AbstractMdRecordToOafMapper(final VocabularyGroup vocs, final boolean invisible,
|
protected AbstractMdRecordToOafMapper(final VocabularyGroup vocs, final boolean invisible,
|
||||||
final boolean shouldHashId, final boolean forceOriginalId) {
|
final boolean shouldHashId, final boolean forceOriginalId) {
|
||||||
this.vocs = vocs;
|
this.vocs = vocs;
|
||||||
this.invisible = invisible;
|
this.invisible = invisible;
|
||||||
this.shouldHashId = shouldHashId;
|
this.shouldHashId = shouldHashId;
|
||||||
|
@ -137,7 +139,7 @@ public abstract class AbstractMdRecordToOafMapper {
|
||||||
}
|
}
|
||||||
|
|
||||||
protected AbstractMdRecordToOafMapper(final VocabularyGroup vocs, final boolean invisible,
|
protected AbstractMdRecordToOafMapper(final VocabularyGroup vocs, final boolean invisible,
|
||||||
final boolean shouldHashId) {
|
final boolean shouldHashId) {
|
||||||
this.vocs = vocs;
|
this.vocs = vocs;
|
||||||
this.invisible = invisible;
|
this.invisible = invisible;
|
||||||
this.shouldHashId = shouldHashId;
|
this.shouldHashId = shouldHashId;
|
||||||
|
@ -149,20 +151,26 @@ public abstract class AbstractMdRecordToOafMapper {
|
||||||
DocumentFactory.getInstance().setXPathNamespaceURIs(nsContext);
|
DocumentFactory.getInstance().setXPathNamespaceURIs(nsContext);
|
||||||
try {
|
try {
|
||||||
final Document doc = DocumentHelper
|
final Document doc = DocumentHelper
|
||||||
.parseText(xml
|
.parseText(
|
||||||
.replaceAll(DATACITE_SCHEMA_KERNEL_4, DATACITE_SCHEMA_KERNEL_3)
|
xml
|
||||||
.replaceAll(DATACITE_SCHEMA_KERNEL_4_SLASH, DATACITE_SCHEMA_KERNEL_3)
|
.replaceAll(DATACITE_SCHEMA_KERNEL_4, DATACITE_SCHEMA_KERNEL_3)
|
||||||
.replaceAll(DATACITE_SCHEMA_KERNEL_3_SLASH, DATACITE_SCHEMA_KERNEL_3));
|
.replaceAll(DATACITE_SCHEMA_KERNEL_4_SLASH, DATACITE_SCHEMA_KERNEL_3)
|
||||||
|
.replaceAll(DATACITE_SCHEMA_KERNEL_3_SLASH, DATACITE_SCHEMA_KERNEL_3));
|
||||||
|
|
||||||
final KeyValue collectedFrom = getProvenanceDatasource(doc, "//oaf:collectedFrom/@id", "//oaf:collectedFrom/@name");
|
final KeyValue collectedFrom = getProvenanceDatasource(
|
||||||
|
doc, "//oaf:collectedFrom/@id", "//oaf:collectedFrom/@name");
|
||||||
|
|
||||||
if (collectedFrom == null) { return Lists.newArrayList(); }
|
if (collectedFrom == null) {
|
||||||
|
return Lists.newArrayList();
|
||||||
|
}
|
||||||
|
|
||||||
final KeyValue hostedBy = StringUtils.isBlank(doc.valueOf("//oaf:hostedBy/@id"))
|
final KeyValue hostedBy = StringUtils.isBlank(doc.valueOf("//oaf:hostedBy/@id"))
|
||||||
? collectedFrom
|
? collectedFrom
|
||||||
: getProvenanceDatasource(doc, "//oaf:hostedBy/@id", "//oaf:hostedBy/@name");
|
: getProvenanceDatasource(doc, "//oaf:hostedBy/@id", "//oaf:hostedBy/@name");
|
||||||
|
|
||||||
if (hostedBy == null) { return Lists.newArrayList(); }
|
if (hostedBy == null) {
|
||||||
|
return Lists.newArrayList();
|
||||||
|
}
|
||||||
|
|
||||||
final DataInfo entityInfo = prepareDataInfo(doc, this.invisible);
|
final DataInfo entityInfo = prepareDataInfo(doc, this.invisible);
|
||||||
final long lastUpdateTimestamp = new Date().getTime();
|
final long lastUpdateTimestamp = new Date().getTime();
|
||||||
|
@ -183,15 +191,15 @@ public abstract class AbstractMdRecordToOafMapper {
|
||||||
|
|
||||||
if (StringUtils.isBlank(type) && this.vocs.vocabularyExists(ModelConstants.DNET_RESULT_TYPOLOGIES)) {
|
if (StringUtils.isBlank(type) && this.vocs.vocabularyExists(ModelConstants.DNET_RESULT_TYPOLOGIES)) {
|
||||||
final String instanceType = instances
|
final String instanceType = instances
|
||||||
.stream()
|
.stream()
|
||||||
.map(i -> i.getInstancetype().getClassid())
|
.map(i -> i.getInstancetype().getClassid())
|
||||||
.findFirst()
|
.findFirst()
|
||||||
.filter(s -> !UNKNOWN.equalsIgnoreCase(s))
|
.filter(s -> !UNKNOWN.equalsIgnoreCase(s))
|
||||||
.orElse("0000"); // Unknown
|
.orElse("0000"); // Unknown
|
||||||
return Optional
|
return Optional
|
||||||
.ofNullable(this.vocs.getSynonymAsQualifier(ModelConstants.DNET_RESULT_TYPOLOGIES, instanceType))
|
.ofNullable(this.vocs.getSynonymAsQualifier(ModelConstants.DNET_RESULT_TYPOLOGIES, instanceType))
|
||||||
.map(Qualifier::getClassid)
|
.map(Qualifier::getClassid)
|
||||||
.orElse("0000");
|
.orElse("0000");
|
||||||
}
|
}
|
||||||
|
|
||||||
return type;
|
return type;
|
||||||
|
@ -201,18 +209,20 @@ public abstract class AbstractMdRecordToOafMapper {
|
||||||
final String dsId = doc.valueOf(xpathId);
|
final String dsId = doc.valueOf(xpathId);
|
||||||
final String dsName = doc.valueOf(xpathName);
|
final String dsName = doc.valueOf(xpathName);
|
||||||
|
|
||||||
if (StringUtils.isBlank(dsId) || StringUtils.isBlank(dsName)) { return null; }
|
if (StringUtils.isBlank(dsId) || StringUtils.isBlank(dsName)) {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
return keyValue(createOpenaireId(10, dsId, true), dsName);
|
return keyValue(createOpenaireId(10, dsId, true), dsName);
|
||||||
}
|
}
|
||||||
|
|
||||||
protected List<Oaf> createOafs(
|
protected List<Oaf> createOafs(
|
||||||
final Document doc,
|
final Document doc,
|
||||||
final String type,
|
final String type,
|
||||||
final List<Instance> instances,
|
final List<Instance> instances,
|
||||||
final KeyValue collectedFrom,
|
final KeyValue collectedFrom,
|
||||||
final DataInfo info,
|
final DataInfo info,
|
||||||
final long lastUpdateTimestamp) {
|
final long lastUpdateTimestamp) {
|
||||||
|
|
||||||
final OafEntity entity = createEntity(doc, type, instances, collectedFrom, info, lastUpdateTimestamp);
|
final OafEntity entity = createEntity(doc, type, instances, collectedFrom, info, lastUpdateTimestamp);
|
||||||
|
|
||||||
|
@ -246,52 +256,52 @@ public abstract class AbstractMdRecordToOafMapper {
|
||||||
}
|
}
|
||||||
|
|
||||||
private OafEntity createEntity(final Document doc,
|
private OafEntity createEntity(final Document doc,
|
||||||
final String type,
|
final String type,
|
||||||
final List<Instance> instances,
|
final List<Instance> instances,
|
||||||
final KeyValue collectedFrom,
|
final KeyValue collectedFrom,
|
||||||
final DataInfo info,
|
final DataInfo info,
|
||||||
final long lastUpdateTimestamp) {
|
final long lastUpdateTimestamp) {
|
||||||
switch (type.toLowerCase()) {
|
switch (type.toLowerCase()) {
|
||||||
case "publication":
|
case "publication":
|
||||||
final Publication p = new Publication();
|
final Publication p = new Publication();
|
||||||
populateResultFields(p, doc, instances, collectedFrom, info, lastUpdateTimestamp);
|
populateResultFields(p, doc, instances, collectedFrom, info, lastUpdateTimestamp);
|
||||||
p.setJournal(prepareJournal(doc, info));
|
p.setJournal(prepareJournal(doc, info));
|
||||||
return p;
|
return p;
|
||||||
case "dataset":
|
case "dataset":
|
||||||
final Dataset d = new Dataset();
|
final Dataset d = new Dataset();
|
||||||
populateResultFields(d, doc, instances, collectedFrom, info, lastUpdateTimestamp);
|
populateResultFields(d, doc, instances, collectedFrom, info, lastUpdateTimestamp);
|
||||||
d.setStoragedate(prepareDatasetStorageDate(doc, info));
|
d.setStoragedate(prepareDatasetStorageDate(doc, info));
|
||||||
d.setDevice(prepareDatasetDevice(doc, info));
|
d.setDevice(prepareDatasetDevice(doc, info));
|
||||||
d.setSize(prepareDatasetSize(doc, info));
|
d.setSize(prepareDatasetSize(doc, info));
|
||||||
d.setVersion(prepareDatasetVersion(doc, info));
|
d.setVersion(prepareDatasetVersion(doc, info));
|
||||||
d.setLastmetadataupdate(prepareDatasetLastMetadataUpdate(doc, info));
|
d.setLastmetadataupdate(prepareDatasetLastMetadataUpdate(doc, info));
|
||||||
d.setMetadataversionnumber(prepareDatasetMetadataVersionNumber(doc, info));
|
d.setMetadataversionnumber(prepareDatasetMetadataVersionNumber(doc, info));
|
||||||
d.setGeolocation(prepareDatasetGeoLocations(doc, info));
|
d.setGeolocation(prepareDatasetGeoLocations(doc, info));
|
||||||
return d;
|
return d;
|
||||||
case "software":
|
case "software":
|
||||||
final Software s = new Software();
|
final Software s = new Software();
|
||||||
populateResultFields(s, doc, instances, collectedFrom, info, lastUpdateTimestamp);
|
populateResultFields(s, doc, instances, collectedFrom, info, lastUpdateTimestamp);
|
||||||
s.setDocumentationUrl(prepareSoftwareDocumentationUrls(doc, info));
|
s.setDocumentationUrl(prepareSoftwareDocumentationUrls(doc, info));
|
||||||
s.setLicense(prepareSoftwareLicenses(doc, info));
|
s.setLicense(prepareSoftwareLicenses(doc, info));
|
||||||
s.setCodeRepositoryUrl(prepareSoftwareCodeRepositoryUrl(doc, info));
|
s.setCodeRepositoryUrl(prepareSoftwareCodeRepositoryUrl(doc, info));
|
||||||
s.setProgrammingLanguage(prepareSoftwareProgrammingLanguage(doc, info));
|
s.setProgrammingLanguage(prepareSoftwareProgrammingLanguage(doc, info));
|
||||||
return s;
|
return s;
|
||||||
case "":
|
case "":
|
||||||
case "otherresearchproducts":
|
case "otherresearchproducts":
|
||||||
default:
|
default:
|
||||||
final OtherResearchProduct o = new OtherResearchProduct();
|
final OtherResearchProduct o = new OtherResearchProduct();
|
||||||
populateResultFields(o, doc, instances, collectedFrom, info, lastUpdateTimestamp);
|
populateResultFields(o, doc, instances, collectedFrom, info, lastUpdateTimestamp);
|
||||||
o.setContactperson(prepareOtherResearchProductContactPersons(doc, info));
|
o.setContactperson(prepareOtherResearchProductContactPersons(doc, info));
|
||||||
o.setContactgroup(prepareOtherResearchProductContactGroups(doc, info));
|
o.setContactgroup(prepareOtherResearchProductContactGroups(doc, info));
|
||||||
o.setTool(prepareOtherResearchProductTools(doc, info));
|
o.setTool(prepareOtherResearchProductTools(doc, info));
|
||||||
return o;
|
return o;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private List<Oaf> addProjectRels(
|
private List<Oaf> addProjectRels(
|
||||||
final Document doc,
|
final Document doc,
|
||||||
final OafEntity entity,
|
final OafEntity entity,
|
||||||
final DataInfo info) {
|
final DataInfo info) {
|
||||||
|
|
||||||
final List<Oaf> res = new ArrayList<>();
|
final List<Oaf> res = new ArrayList<>();
|
||||||
|
|
||||||
|
@ -307,13 +317,21 @@ public abstract class AbstractMdRecordToOafMapper {
|
||||||
final String projectId = createOpenaireId(40, originalId, true);
|
final String projectId = createOpenaireId(40, originalId, true);
|
||||||
|
|
||||||
res
|
res
|
||||||
.add(OafMapperUtils
|
.add(
|
||||||
.getRelation(docId, projectId, RESULT_PROJECT, OUTCOME, IS_PRODUCED_BY, entity.getCollectedfrom(), info, entity
|
OafMapperUtils
|
||||||
.getLastupdatetimestamp(), validationdDate, null));
|
.getRelation(
|
||||||
|
docId, projectId, RESULT_PROJECT, OUTCOME, IS_PRODUCED_BY, entity.getCollectedfrom(),
|
||||||
|
info, entity
|
||||||
|
.getLastupdatetimestamp(),
|
||||||
|
validationdDate, null));
|
||||||
res
|
res
|
||||||
.add(OafMapperUtils
|
.add(
|
||||||
.getRelation(projectId, docId, RESULT_PROJECT, OUTCOME, PRODUCES, entity.getCollectedfrom(), info, entity
|
OafMapperUtils
|
||||||
.getLastupdatetimestamp(), validationdDate, null));
|
.getRelation(
|
||||||
|
projectId, docId, RESULT_PROJECT, OUTCOME, PRODUCES, entity.getCollectedfrom(), info,
|
||||||
|
entity
|
||||||
|
.getLastupdatetimestamp(),
|
||||||
|
validationdDate, null));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -333,11 +351,11 @@ public abstract class AbstractMdRecordToOafMapper {
|
||||||
final String relClass = element.attributeValue("relClass");
|
final String relClass = element.attributeValue("relClass");
|
||||||
|
|
||||||
if (StringUtils.isNotBlank(target) && StringUtils.isNotBlank(relType) && StringUtils.isNotBlank(subRelType)
|
if (StringUtils.isNotBlank(target) && StringUtils.isNotBlank(relType) && StringUtils.isNotBlank(subRelType)
|
||||||
&& StringUtils.isNotBlank(relClass)) {
|
&& StringUtils.isNotBlank(relClass)) {
|
||||||
|
|
||||||
final String relClassInverse = ModelSupport
|
final String relClassInverse = ModelSupport
|
||||||
.findInverse(ModelSupport.rel(relType, subRelType, relClass))
|
.findInverse(ModelSupport.rel(relType, subRelType, relClass))
|
||||||
.getInverseRelClass();
|
.getInverseRelClass();
|
||||||
final String validationDate = ((Node) o).valueOf("@validationDate");
|
final String validationDate = ((Node) o).valueOf("@validationDate");
|
||||||
|
|
||||||
if (StringUtils.isNotBlank(target)) {
|
if (StringUtils.isNotBlank(target)) {
|
||||||
|
@ -345,13 +363,21 @@ public abstract class AbstractMdRecordToOafMapper {
|
||||||
if (StringUtils.isNotBlank(targetType)) {
|
if (StringUtils.isNotBlank(targetType)) {
|
||||||
final String targetId = createOpenaireId(targetType, target, true);
|
final String targetId = createOpenaireId(targetType, target, true);
|
||||||
rels
|
rels
|
||||||
.add(OafMapperUtils
|
.add(
|
||||||
.getRelation(entity.getId(), targetId, relType, subRelType, relClass, entity.getCollectedfrom(), info, entity
|
OafMapperUtils
|
||||||
.getLastupdatetimestamp(), validationDate, null));
|
.getRelation(
|
||||||
|
entity.getId(), targetId, relType, subRelType, relClass,
|
||||||
|
entity.getCollectedfrom(), info, entity
|
||||||
|
.getLastupdatetimestamp(),
|
||||||
|
validationDate, null));
|
||||||
rels
|
rels
|
||||||
.add(OafMapperUtils
|
.add(
|
||||||
.getRelation(targetId, entity.getId(), relType, subRelType, relClassInverse, entity.getCollectedfrom(), info, entity
|
OafMapperUtils
|
||||||
.getLastupdatetimestamp(), validationDate, null));
|
.getRelation(
|
||||||
|
targetId, entity.getId(), relType, subRelType, relClassInverse,
|
||||||
|
entity.getCollectedfrom(), info, entity
|
||||||
|
.getLastupdatetimestamp(),
|
||||||
|
validationDate, null));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -384,30 +410,37 @@ public abstract class AbstractMdRecordToOafMapper {
|
||||||
}
|
}
|
||||||
|
|
||||||
rels
|
rels
|
||||||
.add(OafMapperUtils
|
.add(
|
||||||
.getRelation(resultId, orgId, RESULT_ORGANIZATION, AFFILIATION, HAS_AUTHOR_INSTITUTION, entity.getCollectedfrom(), info, entity
|
OafMapperUtils
|
||||||
.getLastupdatetimestamp(), null, properties));
|
.getRelation(
|
||||||
|
resultId, orgId, RESULT_ORGANIZATION, AFFILIATION, HAS_AUTHOR_INSTITUTION,
|
||||||
|
entity.getCollectedfrom(), info, entity
|
||||||
|
.getLastupdatetimestamp(),
|
||||||
|
null, properties));
|
||||||
rels
|
rels
|
||||||
.add(OafMapperUtils
|
.add(
|
||||||
.getRelation(orgId, resultId, RESULT_ORGANIZATION, AFFILIATION, IS_AUTHOR_INSTITUTION_OF, entity
|
OafMapperUtils
|
||||||
.getCollectedfrom(), info, entity.getLastupdatetimestamp(), null, properties));
|
.getRelation(
|
||||||
|
orgId, resultId, RESULT_ORGANIZATION, AFFILIATION, IS_AUTHOR_INSTITUTION_OF, entity
|
||||||
|
.getCollectedfrom(),
|
||||||
|
info, entity.getLastupdatetimestamp(), null, properties));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
return rels;
|
return rels;
|
||||||
}
|
}
|
||||||
|
|
||||||
protected abstract List<Oaf> addOtherResultRels(
|
protected abstract List<Oaf> addOtherResultRels(
|
||||||
final Document doc,
|
final Document doc,
|
||||||
final OafEntity entity,
|
final OafEntity entity,
|
||||||
DataInfo info);
|
DataInfo info);
|
||||||
|
|
||||||
private void populateResultFields(
|
private void populateResultFields(
|
||||||
final Result r,
|
final Result r,
|
||||||
final Document doc,
|
final Document doc,
|
||||||
final List<Instance> instances,
|
final List<Instance> instances,
|
||||||
final KeyValue collectedFrom,
|
final KeyValue collectedFrom,
|
||||||
final DataInfo info,
|
final DataInfo info,
|
||||||
final long lastUpdateTimestamp) {
|
final long lastUpdateTimestamp) {
|
||||||
r.setDataInfo(info);
|
r.setDataInfo(info);
|
||||||
r.setLastupdatetimestamp(lastUpdateTimestamp);
|
r.setLastupdatetimestamp(lastUpdateTimestamp);
|
||||||
r.setId(createOpenaireId(50, doc.valueOf("//dri:objIdentifier"), false));
|
r.setId(createOpenaireId(50, doc.valueOf("//dri:objIdentifier"), false));
|
||||||
|
@ -499,10 +532,10 @@ public abstract class AbstractMdRecordToOafMapper {
|
||||||
protected abstract Qualifier prepareResourceType(Document doc, DataInfo info);
|
protected abstract Qualifier prepareResourceType(Document doc, DataInfo info);
|
||||||
|
|
||||||
protected abstract List<Instance> prepareInstances(
|
protected abstract List<Instance> prepareInstances(
|
||||||
Document doc,
|
Document doc,
|
||||||
DataInfo info,
|
DataInfo info,
|
||||||
KeyValue collectedfrom,
|
KeyValue collectedfrom,
|
||||||
KeyValue hostedby);
|
KeyValue hostedby);
|
||||||
|
|
||||||
protected abstract List<Field<String>> prepareSources(Document doc, DataInfo info);
|
protected abstract List<Field<String>> prepareSources(Document doc, DataInfo info);
|
||||||
|
|
||||||
|
@ -527,16 +560,16 @@ public abstract class AbstractMdRecordToOafMapper {
|
||||||
protected abstract List<Author> prepareAuthors(Document doc, DataInfo info);
|
protected abstract List<Author> prepareAuthors(Document doc, DataInfo info);
|
||||||
|
|
||||||
protected abstract List<Field<String>> prepareOtherResearchProductTools(
|
protected abstract List<Field<String>> prepareOtherResearchProductTools(
|
||||||
Document doc,
|
Document doc,
|
||||||
DataInfo info);
|
DataInfo info);
|
||||||
|
|
||||||
protected abstract List<Field<String>> prepareOtherResearchProductContactGroups(
|
protected abstract List<Field<String>> prepareOtherResearchProductContactGroups(
|
||||||
Document doc,
|
Document doc,
|
||||||
DataInfo info);
|
DataInfo info);
|
||||||
|
|
||||||
protected abstract List<Field<String>> prepareOtherResearchProductContactPersons(
|
protected abstract List<Field<String>> prepareOtherResearchProductContactPersons(
|
||||||
Document doc,
|
Document doc,
|
||||||
DataInfo info);
|
DataInfo info);
|
||||||
|
|
||||||
protected abstract Qualifier prepareSoftwareProgrammingLanguage(Document doc, DataInfo info);
|
protected abstract Qualifier prepareSoftwareProgrammingLanguage(Document doc, DataInfo info);
|
||||||
|
|
||||||
|
@ -545,8 +578,8 @@ public abstract class AbstractMdRecordToOafMapper {
|
||||||
protected abstract List<StructuredProperty> prepareSoftwareLicenses(Document doc, DataInfo info);
|
protected abstract List<StructuredProperty> prepareSoftwareLicenses(Document doc, DataInfo info);
|
||||||
|
|
||||||
protected abstract List<Field<String>> prepareSoftwareDocumentationUrls(
|
protected abstract List<Field<String>> prepareSoftwareDocumentationUrls(
|
||||||
Document doc,
|
Document doc,
|
||||||
DataInfo info);
|
DataInfo info);
|
||||||
|
|
||||||
protected abstract List<GeoLocation> prepareDatasetGeoLocations(Document doc, DataInfo info);
|
protected abstract List<GeoLocation> prepareDatasetGeoLocations(Document doc, DataInfo info);
|
||||||
|
|
||||||
|
@ -566,13 +599,13 @@ public abstract class AbstractMdRecordToOafMapper {
|
||||||
|
|
||||||
protected List<InstanceTypeMapping> prepareInstanceTypeMapping(final Document doc) {
|
protected List<InstanceTypeMapping> prepareInstanceTypeMapping(final Document doc) {
|
||||||
return Optional
|
return Optional
|
||||||
.ofNullable(findOriginalType(doc))
|
.ofNullable(findOriginalType(doc))
|
||||||
.map(originalType -> {
|
.map(originalType -> {
|
||||||
final List<InstanceTypeMapping> mappings = Lists.newArrayList();
|
final List<InstanceTypeMapping> mappings = Lists.newArrayList();
|
||||||
mappings.add(OafMapperUtils.instanceTypeMapping(originalType, OPENAIRE_COAR_RESOURCE_TYPES_3_1));
|
mappings.add(OafMapperUtils.instanceTypeMapping(originalType, OPENAIRE_COAR_RESOURCE_TYPES_3_1));
|
||||||
return mappings;
|
return mappings;
|
||||||
})
|
})
|
||||||
.orElse(new ArrayList<>());
|
.orElse(new ArrayList<>());
|
||||||
}
|
}
|
||||||
|
|
||||||
private Journal prepareJournal(final Document doc, final DataInfo info) {
|
private Journal prepareJournal(final Document doc, final DataInfo info) {
|
||||||
|
@ -587,7 +620,9 @@ public abstract class AbstractMdRecordToOafMapper {
|
||||||
final String sp = n.valueOf("@sp");
|
final String sp = n.valueOf("@sp");
|
||||||
final String vol = n.valueOf("@vol");
|
final String vol = n.valueOf("@vol");
|
||||||
final String edition = n.valueOf("@edition");
|
final String edition = n.valueOf("@edition");
|
||||||
if (StringUtils.isNotBlank(name)) { return journal(name, issnPrinted, issnOnline, issnLinking, ep, iss, sp, vol, edition, null, null, info); }
|
if (StringUtils.isNotBlank(name)) {
|
||||||
|
return journal(name, issnPrinted, issnOnline, issnLinking, ep, iss, sp, vol, edition, null, null, info);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
@ -596,13 +631,18 @@ public abstract class AbstractMdRecordToOafMapper {
|
||||||
final Node n = doc.selectSingleNode("//*[local-name()='provenance']/*[local-name()='originDescription']");
|
final Node n = doc.selectSingleNode("//*[local-name()='provenance']/*[local-name()='originDescription']");
|
||||||
if (n != null) {
|
if (n != null) {
|
||||||
final String id = n.valueOf("./*[local-name()='identifier']");
|
final String id = n.valueOf("./*[local-name()='identifier']");
|
||||||
if (StringUtils.isNotBlank(id)) { return Lists.newArrayList(id); }
|
if (StringUtils.isNotBlank(id)) {
|
||||||
|
return Lists.newArrayList(id);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
final List<String> idList = doc
|
final List<String> idList = doc
|
||||||
.selectNodes("normalize-space(//*[local-name()='header']/*[local-name()='identifier' or local-name()='recordIdentifier']/text())");
|
.selectNodes(
|
||||||
|
"normalize-space(//*[local-name()='header']/*[local-name()='identifier' or local-name()='recordIdentifier']/text())");
|
||||||
final Set<String> originalIds = Sets.newHashSet(idList);
|
final Set<String> originalIds = Sets.newHashSet(idList);
|
||||||
|
|
||||||
if (originalIds.isEmpty()) { throw new IllegalStateException("missing originalID on " + doc.asXML()); }
|
if (originalIds.isEmpty()) {
|
||||||
|
throw new IllegalStateException("missing originalID on " + doc.asXML());
|
||||||
|
}
|
||||||
return Lists.newArrayList(originalIds);
|
return Lists.newArrayList(originalIds);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -628,11 +668,11 @@ public abstract class AbstractMdRecordToOafMapper {
|
||||||
}
|
}
|
||||||
|
|
||||||
protected List<StructuredProperty> prepareListStructPropsWithValidQualifier(
|
protected List<StructuredProperty> prepareListStructPropsWithValidQualifier(
|
||||||
final Node node,
|
final Node node,
|
||||||
final String xpath,
|
final String xpath,
|
||||||
final String xpathClassId,
|
final String xpathClassId,
|
||||||
final String schemeId,
|
final String schemeId,
|
||||||
final DataInfo info) {
|
final DataInfo info) {
|
||||||
final List<StructuredProperty> res = new ArrayList<>();
|
final List<StructuredProperty> res = new ArrayList<>();
|
||||||
|
|
||||||
for (final Object o : node.selectNodes(xpath)) {
|
for (final Object o : node.selectNodes(xpath)) {
|
||||||
|
@ -646,10 +686,10 @@ public abstract class AbstractMdRecordToOafMapper {
|
||||||
}
|
}
|
||||||
|
|
||||||
protected List<StructuredProperty> prepareListStructProps(
|
protected List<StructuredProperty> prepareListStructProps(
|
||||||
final Node node,
|
final Node node,
|
||||||
final String xpath,
|
final String xpath,
|
||||||
final Qualifier qualifier,
|
final Qualifier qualifier,
|
||||||
final DataInfo info) {
|
final DataInfo info) {
|
||||||
final List<StructuredProperty> res = new ArrayList<>();
|
final List<StructuredProperty> res = new ArrayList<>();
|
||||||
for (final Object o : node.selectNodes(xpath)) {
|
for (final Object o : node.selectNodes(xpath)) {
|
||||||
final Node n = (Node) o;
|
final Node n = (Node) o;
|
||||||
|
@ -659,28 +699,34 @@ public abstract class AbstractMdRecordToOafMapper {
|
||||||
}
|
}
|
||||||
|
|
||||||
protected List<StructuredProperty> prepareListStructProps(
|
protected List<StructuredProperty> prepareListStructProps(
|
||||||
final Node node,
|
final Node node,
|
||||||
final String xpath,
|
final String xpath,
|
||||||
final DataInfo info) {
|
final DataInfo info) {
|
||||||
final List<StructuredProperty> res = new ArrayList<>();
|
final List<StructuredProperty> res = new ArrayList<>();
|
||||||
for (final Object o : node.selectNodes(xpath)) {
|
for (final Object o : node.selectNodes(xpath)) {
|
||||||
final Node n = (Node) o;
|
final Node n = (Node) o;
|
||||||
res
|
res
|
||||||
.add(structuredProperty(n.getText(), n.valueOf("@classid"), n.valueOf("@classname"), n.valueOf("@schemeid"), n
|
.add(
|
||||||
.valueOf("@schemename"), info));
|
structuredProperty(
|
||||||
|
n.getText(), n.valueOf("@classid"), n.valueOf("@classname"), n.valueOf("@schemeid"), n
|
||||||
|
.valueOf("@schemename"),
|
||||||
|
info));
|
||||||
}
|
}
|
||||||
return res;
|
return res;
|
||||||
}
|
}
|
||||||
|
|
||||||
protected List<Subject> prepareSubjectList(
|
protected List<Subject> prepareSubjectList(
|
||||||
final Node node,
|
final Node node,
|
||||||
final String xpath,
|
final String xpath,
|
||||||
final DataInfo info) {
|
final DataInfo info) {
|
||||||
final List<Subject> res = new ArrayList<>();
|
final List<Subject> res = new ArrayList<>();
|
||||||
for (final Object o : node.selectNodes(xpath)) {
|
for (final Object o : node.selectNodes(xpath)) {
|
||||||
final Node n = (Node) o;
|
final Node n = (Node) o;
|
||||||
res
|
res
|
||||||
.add(subject(n.getText(), n.valueOf("@classid"), n.valueOf("@classname"), n.valueOf("@schemeid"), n.valueOf("@schemename"), info));
|
.add(
|
||||||
|
subject(
|
||||||
|
n.getText(), n.valueOf("@classid"), n.valueOf("@classname"), n.valueOf("@schemeid"),
|
||||||
|
n.valueOf("@schemename"), info));
|
||||||
}
|
}
|
||||||
return res;
|
return res;
|
||||||
}
|
}
|
||||||
|
@ -688,7 +734,9 @@ public abstract class AbstractMdRecordToOafMapper {
|
||||||
protected OAIProvenance prepareOAIprovenance(final Document doc) {
|
protected OAIProvenance prepareOAIprovenance(final Document doc) {
|
||||||
final Node n = doc.selectSingleNode("//*[local-name()='provenance']/*[local-name()='originDescription']");
|
final Node n = doc.selectSingleNode("//*[local-name()='provenance']/*[local-name()='originDescription']");
|
||||||
|
|
||||||
if (n == null) { return null; }
|
if (n == null) {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
final String identifier = n.valueOf("./*[local-name()='identifier']");
|
final String identifier = n.valueOf("./*[local-name()='identifier']");
|
||||||
final String baseURL = n.valueOf("./*[local-name()='baseURL']");
|
final String baseURL = n.valueOf("./*[local-name()='baseURL']");
|
||||||
|
@ -703,7 +751,9 @@ public abstract class AbstractMdRecordToOafMapper {
|
||||||
protected DataInfo prepareDataInfo(final Document doc, final boolean invisible) {
|
protected DataInfo prepareDataInfo(final Document doc, final boolean invisible) {
|
||||||
final Node n = doc.selectSingleNode("//oaf:datainfo");
|
final Node n = doc.selectSingleNode("//oaf:datainfo");
|
||||||
|
|
||||||
if (n == null) { return dataInfo(false, null, false, invisible, REPOSITORY_PROVENANCE_ACTIONS, "0.9"); }
|
if (n == null) {
|
||||||
|
return dataInfo(false, null, false, invisible, REPOSITORY_PROVENANCE_ACTIONS, "0.9");
|
||||||
|
}
|
||||||
|
|
||||||
final String paClassId = n.valueOf("./oaf:provenanceaction/@classid");
|
final String paClassId = n.valueOf("./oaf:provenanceaction/@classid");
|
||||||
final String paClassName = n.valueOf("./oaf:provenanceaction/@classname");
|
final String paClassName = n.valueOf("./oaf:provenanceaction/@classname");
|
||||||
|
@ -715,11 +765,14 @@ public abstract class AbstractMdRecordToOafMapper {
|
||||||
final Boolean inferred = Boolean.parseBoolean(n.valueOf("./oaf:inferred"));
|
final Boolean inferred = Boolean.parseBoolean(n.valueOf("./oaf:inferred"));
|
||||||
final String trust = n.valueOf("./oaf:trust");
|
final String trust = n.valueOf("./oaf:trust");
|
||||||
|
|
||||||
return dataInfo(deletedbyinference, inferenceprovenance, inferred, invisible, qualifier(paClassId, paClassName, paSchemeId, paSchemeName), trust);
|
return dataInfo(
|
||||||
|
deletedbyinference, inferenceprovenance, inferred, invisible,
|
||||||
|
qualifier(paClassId, paClassName, paSchemeId, paSchemeName), trust);
|
||||||
}
|
}
|
||||||
|
|
||||||
protected List<Field<String>> prepareListURL(final Node node, final String xpath, final DataInfo info) {
|
protected List<Field<String>> prepareListURL(final Node node, final String xpath, final DataInfo info) {
|
||||||
return listFields(info, prepareListString(node, xpath)
|
return listFields(
|
||||||
|
info, prepareListString(node, xpath)
|
||||||
.stream()
|
.stream()
|
||||||
.filter(URL_VALIDATOR::isValid)
|
.filter(URL_VALIDATOR::isValid)
|
||||||
.collect(Collectors.toList()));
|
.collect(Collectors.toList()));
|
||||||
|
@ -730,9 +783,9 @@ public abstract class AbstractMdRecordToOafMapper {
|
||||||
}
|
}
|
||||||
|
|
||||||
protected List<Field<String>> prepareListFields(
|
protected List<Field<String>> prepareListFields(
|
||||||
final Node node,
|
final Node node,
|
||||||
final String xpath,
|
final String xpath,
|
||||||
final DataInfo info) {
|
final DataInfo info) {
|
||||||
return listFields(info, prepareListString(node, xpath));
|
return listFields(info, prepareListString(node, xpath));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -749,11 +802,13 @@ public abstract class AbstractMdRecordToOafMapper {
|
||||||
|
|
||||||
protected Set<String> validateUrl(final Collection<String> url) {
|
protected Set<String> validateUrl(final Collection<String> url) {
|
||||||
|
|
||||||
if (Objects.isNull(url)) { return new HashSet<>(); }
|
if (Objects.isNull(url)) {
|
||||||
|
return new HashSet<>();
|
||||||
|
}
|
||||||
return url
|
return url
|
||||||
.stream()
|
.stream()
|
||||||
.filter(URL_VALIDATOR::isValid)
|
.filter(URL_VALIDATOR::isValid)
|
||||||
.collect(Collectors.toCollection(HashSet::new));
|
.collect(Collectors.toCollection(HashSet::new));
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -1,16 +1,13 @@
|
||||||
|
|
||||||
package eu.dnetlib.dhp.oa.graph.raw;
|
package eu.dnetlib.dhp.oa.graph.raw;
|
||||||
|
|
||||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession;
|
||||||
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
|
|
||||||
import eu.dnetlib.dhp.common.HdfsSupport;
|
import java.util.Arrays;
|
||||||
import eu.dnetlib.dhp.common.vocabulary.VocabularyGroup;
|
import java.util.List;
|
||||||
import eu.dnetlib.dhp.oa.graph.raw.common.AbstractMigrationApplication;
|
import java.util.Objects;
|
||||||
import eu.dnetlib.dhp.schema.common.ModelSupport;
|
import java.util.Optional;
|
||||||
import eu.dnetlib.dhp.schema.oaf.*;
|
|
||||||
import eu.dnetlib.dhp.schema.oaf.utils.MergeUtils;
|
|
||||||
import eu.dnetlib.dhp.utils.ISLookupClientFactory;
|
|
||||||
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService;
|
|
||||||
import org.apache.commons.io.IOUtils;
|
import org.apache.commons.io.IOUtils;
|
||||||
import org.apache.commons.lang3.StringUtils;
|
import org.apache.commons.lang3.StringUtils;
|
||||||
import org.apache.hadoop.io.Text;
|
import org.apache.hadoop.io.Text;
|
||||||
|
@ -21,15 +18,20 @@ import org.apache.spark.api.java.JavaSparkContext;
|
||||||
import org.apache.spark.sql.SparkSession;
|
import org.apache.spark.sql.SparkSession;
|
||||||
import org.slf4j.Logger;
|
import org.slf4j.Logger;
|
||||||
import org.slf4j.LoggerFactory;
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
|
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||||
|
|
||||||
|
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
|
||||||
|
import eu.dnetlib.dhp.common.HdfsSupport;
|
||||||
|
import eu.dnetlib.dhp.common.vocabulary.VocabularyGroup;
|
||||||
|
import eu.dnetlib.dhp.oa.graph.raw.common.AbstractMigrationApplication;
|
||||||
|
import eu.dnetlib.dhp.schema.common.ModelSupport;
|
||||||
|
import eu.dnetlib.dhp.schema.oaf.*;
|
||||||
|
import eu.dnetlib.dhp.schema.oaf.utils.MergeUtils;
|
||||||
|
import eu.dnetlib.dhp.utils.ISLookupClientFactory;
|
||||||
|
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService;
|
||||||
import scala.Tuple2;
|
import scala.Tuple2;
|
||||||
|
|
||||||
import java.util.Arrays;
|
|
||||||
import java.util.List;
|
|
||||||
import java.util.Objects;
|
|
||||||
import java.util.Optional;
|
|
||||||
|
|
||||||
import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession;
|
|
||||||
|
|
||||||
public class GenerateEntitiesApplication extends AbstractMigrationApplication {
|
public class GenerateEntitiesApplication extends AbstractMigrationApplication {
|
||||||
|
|
||||||
private static final Logger log = LoggerFactory.getLogger(GenerateEntitiesApplication.class);
|
private static final Logger log = LoggerFactory.getLogger(GenerateEntitiesApplication.class);
|
||||||
|
|
|
@ -1,12 +1,13 @@
|
||||||
|
|
||||||
package eu.dnetlib.dhp.oa.graph.raw;
|
package eu.dnetlib.dhp.oa.graph.raw;
|
||||||
|
|
||||||
import eu.dnetlib.dhp.common.vocabulary.VocabularyGroup;
|
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||||
import eu.dnetlib.dhp.schema.common.ModelConstants;
|
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||||
import eu.dnetlib.dhp.schema.oaf.*;
|
import static org.mockito.Mockito.lenient;
|
||||||
import eu.dnetlib.dhp.schema.oaf.utils.MergeUtils;
|
|
||||||
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException;
|
import java.io.IOException;
|
||||||
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService;
|
import java.util.List;
|
||||||
|
|
||||||
import org.apache.commons.io.IOUtils;
|
import org.apache.commons.io.IOUtils;
|
||||||
import org.dom4j.DocumentException;
|
import org.dom4j.DocumentException;
|
||||||
import org.junit.jupiter.api.BeforeEach;
|
import org.junit.jupiter.api.BeforeEach;
|
||||||
|
@ -15,12 +16,12 @@ import org.junit.jupiter.api.extension.ExtendWith;
|
||||||
import org.mockito.Mock;
|
import org.mockito.Mock;
|
||||||
import org.mockito.junit.jupiter.MockitoExtension;
|
import org.mockito.junit.jupiter.MockitoExtension;
|
||||||
|
|
||||||
import java.io.IOException;
|
import eu.dnetlib.dhp.common.vocabulary.VocabularyGroup;
|
||||||
import java.util.List;
|
import eu.dnetlib.dhp.schema.common.ModelConstants;
|
||||||
|
import eu.dnetlib.dhp.schema.oaf.*;
|
||||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
import eu.dnetlib.dhp.schema.oaf.utils.MergeUtils;
|
||||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException;
|
||||||
import static org.mockito.Mockito.lenient;
|
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService;
|
||||||
|
|
||||||
@ExtendWith(MockitoExtension.class)
|
@ExtendWith(MockitoExtension.class)
|
||||||
class GenerateEntitiesApplicationTest {
|
class GenerateEntitiesApplicationTest {
|
||||||
|
|
|
@ -1137,7 +1137,8 @@ public class XmlRecordFactory implements Serializable {
|
||||||
XmlSerializationUtils
|
XmlSerializationUtils
|
||||||
.asXmlElement("dateofacceptance", i.getDateofacceptance().getValue()));
|
.asXmlElement("dateofacceptance", i.getDateofacceptance().getValue()));
|
||||||
}
|
}
|
||||||
if (i.getInstancetype() != null && StringUtils.isNotBlank(i.getInstancetype().getClassid())) {
|
if (i.getInstancetype() != null
|
||||||
|
&& StringUtils.isNotBlank(i.getInstancetype().getClassid())) {
|
||||||
instanceFields
|
instanceFields
|
||||||
.add(XmlSerializationUtils.mapQualifier("instancetype", i.getInstancetype()));
|
.add(XmlSerializationUtils.mapQualifier("instancetype", i.getInstancetype()));
|
||||||
}
|
}
|
||||||
|
@ -1178,7 +1179,8 @@ public class XmlRecordFactory implements Serializable {
|
||||||
if (re.getDatasourcetypeui() != null && StringUtils.isNotBlank(re.getDatasourcetypeui().getClassid())) {
|
if (re.getDatasourcetypeui() != null && StringUtils.isNotBlank(re.getDatasourcetypeui().getClassid())) {
|
||||||
metadata.add(XmlSerializationUtils.mapQualifier("datasourcetypeui", re.getDatasourcetypeui()));
|
metadata.add(XmlSerializationUtils.mapQualifier("datasourcetypeui", re.getDatasourcetypeui()));
|
||||||
}
|
}
|
||||||
if (re.getOpenairecompatibility() != null && StringUtils.isNotBlank(re.getOpenairecompatibility().getClassid())) {
|
if (re.getOpenairecompatibility() != null
|
||||||
|
&& StringUtils.isNotBlank(re.getOpenairecompatibility().getClassid())) {
|
||||||
metadata
|
metadata
|
||||||
.add(
|
.add(
|
||||||
XmlSerializationUtils
|
XmlSerializationUtils
|
||||||
|
@ -1285,7 +1287,8 @@ public class XmlRecordFactory implements Serializable {
|
||||||
groupInstancesByUrl(((Result) entity).getInstance()).forEach(instance -> {
|
groupInstancesByUrl(((Result) entity).getInstance()).forEach(instance -> {
|
||||||
final List<String> fields = Lists.newArrayList();
|
final List<String> fields = Lists.newArrayList();
|
||||||
|
|
||||||
if (instance.getAccessright() != null && StringUtils.isNotBlank(instance.getAccessright().getClassid())) {
|
if (instance.getAccessright() != null
|
||||||
|
&& StringUtils.isNotBlank(instance.getAccessright().getClassid())) {
|
||||||
fields
|
fields
|
||||||
.add(XmlSerializationUtils.mapQualifier("accessright", instance.getAccessright()));
|
.add(XmlSerializationUtils.mapQualifier("accessright", instance.getAccessright()));
|
||||||
}
|
}
|
||||||
|
|
|
@ -7,10 +7,11 @@ import static org.apache.commons.lang3.StringUtils.isNotBlank;
|
||||||
|
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
|
||||||
|
import org.apache.commons.lang3.StringUtils;
|
||||||
|
|
||||||
import com.google.common.collect.Lists;
|
import com.google.common.collect.Lists;
|
||||||
|
|
||||||
import eu.dnetlib.dhp.schema.oaf.*;
|
import eu.dnetlib.dhp.schema.oaf.*;
|
||||||
import org.apache.commons.lang3.StringUtils;
|
|
||||||
import scala.Tuple2;
|
import scala.Tuple2;
|
||||||
|
|
||||||
public class XmlSerializationUtils {
|
public class XmlSerializationUtils {
|
||||||
|
|
|
@ -3,7 +3,7 @@
|
||||||
<parent>
|
<parent>
|
||||||
<artifactId>dhp-workflows</artifactId>
|
<artifactId>dhp-workflows</artifactId>
|
||||||
<groupId>eu.dnetlib.dhp</groupId>
|
<groupId>eu.dnetlib.dhp</groupId>
|
||||||
<version>1.2.4-SNAPSHOT</version>
|
<version>1.2.5-SNAPSHOT</version>
|
||||||
</parent>
|
</parent>
|
||||||
<modelVersion>4.0.0</modelVersion>
|
<modelVersion>4.0.0</modelVersion>
|
||||||
<artifactId>dhp-stats-hist-snaps</artifactId>
|
<artifactId>dhp-stats-hist-snaps</artifactId>
|
||||||
|
|
|
@ -3,7 +3,7 @@
|
||||||
<parent>
|
<parent>
|
||||||
<artifactId>dhp-workflows</artifactId>
|
<artifactId>dhp-workflows</artifactId>
|
||||||
<groupId>eu.dnetlib.dhp</groupId>
|
<groupId>eu.dnetlib.dhp</groupId>
|
||||||
<version>1.2.4-SNAPSHOT</version>
|
<version>1.2.5-SNAPSHOT</version>
|
||||||
</parent>
|
</parent>
|
||||||
<modelVersion>4.0.0</modelVersion>
|
<modelVersion>4.0.0</modelVersion>
|
||||||
<artifactId>dhp-stats-monitor-irish</artifactId>
|
<artifactId>dhp-stats-monitor-irish</artifactId>
|
||||||
|
|
|
@ -3,7 +3,7 @@
|
||||||
<parent>
|
<parent>
|
||||||
<artifactId>dhp-workflows</artifactId>
|
<artifactId>dhp-workflows</artifactId>
|
||||||
<groupId>eu.dnetlib.dhp</groupId>
|
<groupId>eu.dnetlib.dhp</groupId>
|
||||||
<version>1.2.4-SNAPSHOT</version>
|
<version>1.2.5-SNAPSHOT</version>
|
||||||
</parent>
|
</parent>
|
||||||
<modelVersion>4.0.0</modelVersion>
|
<modelVersion>4.0.0</modelVersion>
|
||||||
<artifactId>dhp-stats-monitor-update</artifactId>
|
<artifactId>dhp-stats-monitor-update</artifactId>
|
||||||
|
|
|
@ -31,6 +31,10 @@
|
||||||
<module>dhp-enrichment</module>
|
<module>dhp-enrichment</module>
|
||||||
<module>dhp-graph-provision</module>
|
<module>dhp-graph-provision</module>
|
||||||
<module>dhp-blacklist</module>
|
<module>dhp-blacklist</module>
|
||||||
|
<module>dhp-stats-actionsets</module>
|
||||||
|
<module>dhp-stats-hist-snaps</module>
|
||||||
|
<module>dhp-stats-monitor-irish</module>
|
||||||
|
<module>dhp-stats-monitor-update</module>
|
||||||
<module>dhp-stats-update</module>
|
<module>dhp-stats-update</module>
|
||||||
<module>dhp-stats-promote</module>
|
<module>dhp-stats-promote</module>
|
||||||
<module>dhp-usage-stats-build</module>
|
<module>dhp-usage-stats-build</module>
|
||||||
|
|
Loading…
Reference in New Issue