forked from D-Net/dnet-hadoop
WIP: extended provision workflow to create the JSON based payload
This commit is contained in:
parent
6fcf872daa
commit
d4871b31e8
|
@ -62,8 +62,16 @@ public class SolrAdminApplication implements Closeable {
|
|||
final String collection = ProvisionConstants.getCollectionName(format);
|
||||
log.info("collection: {}", collection);
|
||||
|
||||
try (SolrAdminApplication app = new SolrAdminApplication(zkHost)) {
|
||||
app.execute(action, collection, query, commit);
|
||||
final boolean shouldIndex = Optional
|
||||
.ofNullable(parser.get("shouldIndex"))
|
||||
.map(Boolean::valueOf)
|
||||
.orElse(false);
|
||||
log.info("shouldIndex: {}", shouldIndex);
|
||||
|
||||
if (shouldIndex) {
|
||||
try (SolrAdminApplication app = new SolrAdminApplication(zkHost)) {
|
||||
app.execute(action, collection, query, commit);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -3,6 +3,7 @@ package eu.dnetlib.dhp.oa.provision;
|
|||
|
||||
import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession;
|
||||
import static eu.dnetlib.dhp.utils.DHPUtils.toSeq;
|
||||
import static org.apache.spark.sql.functions.*;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
@ -14,22 +15,31 @@ import org.apache.hadoop.io.compress.GzipCodec;
|
|||
import org.apache.hadoop.mapred.SequenceFileOutputFormat;
|
||||
import org.apache.spark.SparkConf;
|
||||
import org.apache.spark.SparkContext;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.api.java.function.MapFunction;
|
||||
import org.apache.spark.api.java.function.PairFunction;
|
||||
import org.apache.spark.sql.Encoders;
|
||||
import org.apache.spark.sql.SparkSession;
|
||||
import org.apache.spark.sql.*;
|
||||
import org.apache.spark.sql.expressions.UserDefinedFunction;
|
||||
import org.apache.spark.sql.types.DataTypes;
|
||||
import org.apache.spark.util.LongAccumulator;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonInclude;
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.google.common.collect.Maps;
|
||||
|
||||
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
|
||||
import eu.dnetlib.dhp.common.HdfsSupport;
|
||||
import eu.dnetlib.dhp.common.vocabulary.VocabularyGroup;
|
||||
import eu.dnetlib.dhp.oa.provision.model.JoinedEntity;
|
||||
import eu.dnetlib.dhp.oa.provision.model.ProvisionModelSupport;
|
||||
import eu.dnetlib.dhp.oa.provision.model.TupleWrapper;
|
||||
import eu.dnetlib.dhp.oa.provision.utils.ContextMapper;
|
||||
import eu.dnetlib.dhp.oa.provision.utils.XmlRecordFactory;
|
||||
import eu.dnetlib.dhp.schema.solr.SolrRecord;
|
||||
import eu.dnetlib.dhp.utils.ISLookupClientFactory;
|
||||
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService;
|
||||
import scala.Tuple2;
|
||||
|
||||
/**
|
||||
|
@ -65,13 +75,20 @@ public class XmlConverterJob {
|
|||
final String contextApiBaseUrl = parser.get("contextApiBaseUrl");
|
||||
log.info("contextApiBaseUrl: {}", contextApiBaseUrl);
|
||||
|
||||
String isLookupUrl = parser.get("isLookupUrl");
|
||||
log.info("isLookupUrl: {}", isLookupUrl);
|
||||
|
||||
ISLookUpService isLookup = ISLookupClientFactory.getLookUpService(isLookupUrl);
|
||||
|
||||
final SparkConf conf = new SparkConf();
|
||||
conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer");
|
||||
conf.registerKryoClasses(ProvisionModelSupport.getModelClasses());
|
||||
|
||||
runWithSparkSession(conf, isSparkSessionManaged, spark -> {
|
||||
removeOutputDir(spark, outputPath);
|
||||
convertToXml(spark, inputPath, outputPath, ContextMapper.fromAPI(contextApiBaseUrl));
|
||||
convertToXml(
|
||||
spark, inputPath, outputPath, ContextMapper.fromAPI(contextApiBaseUrl),
|
||||
VocabularyGroup.loadVocsFromIS(isLookup));
|
||||
});
|
||||
}
|
||||
|
||||
|
@ -79,7 +96,8 @@ public class XmlConverterJob {
|
|||
final SparkSession spark,
|
||||
final String inputPath,
|
||||
final String outputPath,
|
||||
final ContextMapper contextMapper) {
|
||||
final ContextMapper contextMapper,
|
||||
final VocabularyGroup vocabularies) {
|
||||
|
||||
final XmlRecordFactory recordFactory = new XmlRecordFactory(
|
||||
prepareAccumulators(spark.sparkContext()),
|
||||
|
@ -92,20 +110,25 @@ public class XmlConverterJob {
|
|||
|
||||
log.info("Found paths: {}", String.join(",", paths));
|
||||
|
||||
final ObjectMapper mapper = new ObjectMapper();
|
||||
mapper.setSerializationInclusion(JsonInclude.Include.NON_EMPTY);
|
||||
spark
|
||||
.read()
|
||||
.load(toSeq(paths))
|
||||
.as(Encoders.kryo(JoinedEntity.class))
|
||||
.map(
|
||||
(MapFunction<JoinedEntity, Tuple2<String, String>>) je -> new Tuple2<>(
|
||||
je.getEntity().getId(),
|
||||
recordFactory.build(je)),
|
||||
Encoders.tuple(Encoders.STRING(), Encoders.STRING()))
|
||||
.javaRDD()
|
||||
.mapToPair(
|
||||
(PairFunction<Tuple2<String, String>, Text, Text>) t -> new Tuple2<>(new Text(t._1()),
|
||||
new Text(t._2())))
|
||||
.saveAsHadoopFile(outputPath, Text.class, Text.class, SequenceFileOutputFormat.class, GzipCodec.class);
|
||||
(MapFunction<JoinedEntity, Tuple2<String, SolrRecord>>) je -> new Tuple2<>(
|
||||
recordFactory.build(je),
|
||||
ProvisionModelSupport.transform(je, contextMapper, vocabularies)),
|
||||
Encoders.tuple(Encoders.STRING(), Encoders.bean(SolrRecord.class)))
|
||||
.map(
|
||||
(MapFunction<Tuple2<String, SolrRecord>, TupleWrapper>) t -> new TupleWrapper(
|
||||
t._1(), mapper.writeValueAsString(t._2())),
|
||||
Encoders.bean(TupleWrapper.class))
|
||||
.write()
|
||||
.mode(SaveMode.Overwrite)
|
||||
.option("compression", "gzip")
|
||||
.json(outputPath);
|
||||
}
|
||||
|
||||
private static void removeOutputDir(final SparkSession spark, final String path) {
|
||||
|
|
|
@ -21,9 +21,8 @@ import org.apache.solr.common.SolrInputDocument;
|
|||
import org.apache.spark.SparkConf;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.apache.spark.sql.Encoders;
|
||||
import org.apache.spark.sql.SaveMode;
|
||||
import org.apache.spark.sql.SparkSession;
|
||||
import org.apache.spark.api.java.function.MapFunction;
|
||||
import org.apache.spark.sql.*;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
|
@ -31,11 +30,13 @@ import com.lucidworks.spark.util.SolrSupport;
|
|||
|
||||
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
|
||||
import eu.dnetlib.dhp.oa.provision.model.SerializableSolrInputDocument;
|
||||
import eu.dnetlib.dhp.oa.provision.model.TupleWrapper;
|
||||
import eu.dnetlib.dhp.oa.provision.utils.ISLookupClient;
|
||||
import eu.dnetlib.dhp.oa.provision.utils.StreamingInputDocumentFactory;
|
||||
import eu.dnetlib.dhp.utils.ISLookupClientFactory;
|
||||
import eu.dnetlib.dhp.utils.saxon.SaxonTransformerFactory;
|
||||
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException;
|
||||
import scala.Tuple2;
|
||||
|
||||
public class XmlIndexingJob {
|
||||
|
||||
|
@ -59,6 +60,8 @@ public class XmlIndexingJob {
|
|||
|
||||
private final String outputPath;
|
||||
|
||||
private boolean shouldIndex;
|
||||
|
||||
private final SparkSession spark;
|
||||
|
||||
public static void main(String[] args) throws Exception {
|
||||
|
@ -101,7 +104,14 @@ public class XmlIndexingJob {
|
|||
.orElse(OutputFormat.SOLR);
|
||||
log.info("outputFormat: {}", outputFormat);
|
||||
|
||||
final boolean shouldIndex = Optional
|
||||
.ofNullable(parser.get("shouldIndex"))
|
||||
.map(Boolean::valueOf)
|
||||
.orElse(false);
|
||||
log.info("shouldIndex: {}", shouldIndex);
|
||||
|
||||
final SparkConf conf = new SparkConf();
|
||||
|
||||
conf.registerKryoClasses(new Class[] {
|
||||
SerializableSolrInputDocument.class
|
||||
});
|
||||
|
@ -113,18 +123,19 @@ public class XmlIndexingJob {
|
|||
final String isLookupUrl = parser.get("isLookupUrl");
|
||||
log.info("isLookupUrl: {}", isLookupUrl);
|
||||
final ISLookupClient isLookup = new ISLookupClient(ISLookupClientFactory.getLookUpService(isLookupUrl));
|
||||
new XmlIndexingJob(spark, inputPath, format, batchSize, outputFormat, outputPath).run(isLookup);
|
||||
new XmlIndexingJob(spark, inputPath, format, batchSize, outputFormat, shouldIndex, outputPath)
|
||||
.run(isLookup);
|
||||
});
|
||||
}
|
||||
|
||||
public XmlIndexingJob(SparkSession spark, String inputPath, String format, Integer batchSize,
|
||||
OutputFormat outputFormat,
|
||||
String outputPath) {
|
||||
OutputFormat outputFormat, boolean shouldIndex, String outputPath) {
|
||||
this.spark = spark;
|
||||
this.inputPath = inputPath;
|
||||
this.format = format;
|
||||
this.batchSize = batchSize;
|
||||
this.outputFormat = outputFormat;
|
||||
this.shouldIndex = shouldIndex;
|
||||
this.outputPath = outputPath;
|
||||
}
|
||||
|
||||
|
@ -140,33 +151,45 @@ public class XmlIndexingJob {
|
|||
final String zkHost = isLookup.getZkHost();
|
||||
log.info("zkHost: {}", zkHost);
|
||||
|
||||
final String version = getRecordDatestamp();
|
||||
|
||||
final String indexRecordXslt = getLayoutTransformer(format, fields, xslt);
|
||||
log.info("indexRecordTransformer {}", indexRecordXslt);
|
||||
|
||||
final JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext());
|
||||
|
||||
JavaRDD<SolrInputDocument> docs = sc
|
||||
.sequenceFile(inputPath, Text.class, Text.class)
|
||||
.map(t -> t._2().toString())
|
||||
.map(s -> toIndexRecord(SaxonTransformerFactory.newInstance(indexRecordXslt), s))
|
||||
.map(s -> new StreamingInputDocumentFactory().parseDocument(s));
|
||||
final Encoder<TupleWrapper> encoder = Encoders.bean(TupleWrapper.class);
|
||||
final Dataset<TupleWrapper> records = spark
|
||||
.read()
|
||||
.schema(encoder.schema())
|
||||
.json(inputPath)
|
||||
.as(encoder)
|
||||
.map(
|
||||
(MapFunction<TupleWrapper, TupleWrapper>) t -> new TupleWrapper(
|
||||
toIndexRecord(SaxonTransformerFactory.newInstance(indexRecordXslt), t.getXml()),
|
||||
t.getJson()),
|
||||
Encoders.bean(TupleWrapper.class));
|
||||
|
||||
switch (outputFormat) {
|
||||
case SOLR:
|
||||
final String collection = ProvisionConstants.getCollectionName(format);
|
||||
if (shouldIndex) {
|
||||
final String collection = ProvisionConstants.getCollectionName(format);
|
||||
|
||||
// SparkSolr >= 4
|
||||
// com.lucidworks.spark.BatchSizeType bt = com.lucidworks.spark.BatchSizeType.NUM_DOCS;
|
||||
// SolrSupport.indexDocs(zkHost, collection, batchSize, bt, docs.rdd());
|
||||
// SparkSolr < 4
|
||||
SolrSupport.indexDocs(zkHost, collection, batchSize, docs.rdd());
|
||||
// SparkSolr >= 4
|
||||
// com.lucidworks.spark.BatchSizeType bt = com.lucidworks.spark.BatchSizeType.NUM_DOCS;
|
||||
// SolrSupport.indexDocs(zkHost, collection, batchSize, bt, docs.rdd());
|
||||
// SparkSolr < 4
|
||||
JavaRDD<SolrInputDocument> docs = records
|
||||
.javaRDD()
|
||||
.map(
|
||||
t -> new StreamingInputDocumentFactory().parseDocument(t.getXml(), t.getJson()));
|
||||
SolrSupport.indexDocs(zkHost, collection, batchSize, docs.rdd());
|
||||
}
|
||||
break;
|
||||
case HDFS:
|
||||
spark
|
||||
.createDataset(
|
||||
docs.map(SerializableSolrInputDocument::new).rdd(),
|
||||
records
|
||||
.map(
|
||||
(MapFunction<TupleWrapper, SerializableSolrInputDocument>) t -> {
|
||||
SolrInputDocument s = new StreamingInputDocumentFactory()
|
||||
.parseDocument(t.getXml(), t.getJson());
|
||||
return new SerializableSolrInputDocument(s);
|
||||
},
|
||||
Encoders.kryo(SerializableSolrInputDocument.class))
|
||||
.write()
|
||||
.mode(SaveMode.Overwrite)
|
||||
|
|
|
@ -5,11 +5,15 @@ import java.io.Serializable;
|
|||
import java.util.LinkedList;
|
||||
import java.util.List;
|
||||
|
||||
import eu.dnetlib.dhp.schema.oaf.OafEntity;
|
||||
import com.fasterxml.jackson.annotation.JsonSubTypes;
|
||||
import com.fasterxml.jackson.annotation.JsonTypeInfo;
|
||||
|
||||
public class JoinedEntity<E extends OafEntity> implements Serializable {
|
||||
import eu.dnetlib.dhp.schema.common.EntityType;
|
||||
import eu.dnetlib.dhp.schema.oaf.*;
|
||||
|
||||
private E entity;
|
||||
public class JoinedEntity implements Serializable {
|
||||
|
||||
private OafEntity entity;
|
||||
|
||||
private List<RelatedEntityWrapper> links;
|
||||
|
||||
|
@ -17,16 +21,16 @@ public class JoinedEntity<E extends OafEntity> implements Serializable {
|
|||
links = new LinkedList<>();
|
||||
}
|
||||
|
||||
public JoinedEntity(E entity) {
|
||||
public JoinedEntity(OafEntity entity) {
|
||||
this();
|
||||
this.entity = entity;
|
||||
}
|
||||
|
||||
public E getEntity() {
|
||||
public OafEntity getEntity() {
|
||||
return entity;
|
||||
}
|
||||
|
||||
public void setEntity(E entity) {
|
||||
public void setEntity(OafEntity entity) {
|
||||
this.entity = entity;
|
||||
}
|
||||
|
||||
|
|
|
@ -1,13 +1,46 @@
|
|||
|
||||
package eu.dnetlib.dhp.oa.provision.model;
|
||||
|
||||
import java.util.List;
|
||||
import static org.apache.commons.lang3.StringUtils.substringBefore;
|
||||
|
||||
import java.io.StringReader;
|
||||
import java.util.*;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import org.apache.commons.lang3.StringUtils;
|
||||
import org.dom4j.Document;
|
||||
import org.dom4j.DocumentException;
|
||||
import org.dom4j.io.SAXReader;
|
||||
import org.jetbrains.annotations.Nullable;
|
||||
|
||||
import com.google.common.base.Splitter;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.common.collect.Sets;
|
||||
|
||||
import eu.dnetlib.dhp.common.vocabulary.VocabularyGroup;
|
||||
import eu.dnetlib.dhp.common.vocabulary.VocabularyTerm;
|
||||
import eu.dnetlib.dhp.oa.provision.RelationList;
|
||||
import eu.dnetlib.dhp.oa.provision.SortableRelation;
|
||||
import eu.dnetlib.dhp.oa.provision.utils.ContextDef;
|
||||
import eu.dnetlib.dhp.oa.provision.utils.ContextMapper;
|
||||
import eu.dnetlib.dhp.schema.common.ModelSupport;
|
||||
import eu.dnetlib.dhp.schema.oaf.*;
|
||||
import eu.dnetlib.dhp.schema.solr.*;
|
||||
import eu.dnetlib.dhp.schema.solr.AccessRight;
|
||||
import eu.dnetlib.dhp.schema.solr.Author;
|
||||
import eu.dnetlib.dhp.schema.solr.Context;
|
||||
import eu.dnetlib.dhp.schema.solr.Country;
|
||||
import eu.dnetlib.dhp.schema.solr.Datasource;
|
||||
import eu.dnetlib.dhp.schema.solr.EoscIfGuidelines;
|
||||
import eu.dnetlib.dhp.schema.solr.Instance;
|
||||
import eu.dnetlib.dhp.schema.solr.Journal;
|
||||
import eu.dnetlib.dhp.schema.solr.OpenAccessColor;
|
||||
import eu.dnetlib.dhp.schema.solr.OpenAccessRoute;
|
||||
import eu.dnetlib.dhp.schema.solr.Organization;
|
||||
import eu.dnetlib.dhp.schema.solr.Project;
|
||||
import eu.dnetlib.dhp.schema.solr.Result;
|
||||
import eu.dnetlib.dhp.schema.solr.Subject;
|
||||
|
||||
public class ProvisionModelSupport {
|
||||
|
||||
|
@ -28,4 +61,631 @@ public class ProvisionModelSupport {
|
|||
RelationList.class));
|
||||
return modelClasses.toArray(new Class[] {});
|
||||
}
|
||||
|
||||
public static SolrRecord transform(JoinedEntity je, ContextMapper contextMapper, VocabularyGroup vocs) {
|
||||
SolrRecord s = new SolrRecord();
|
||||
final OafEntity e = je.getEntity();
|
||||
s
|
||||
.setHeader(
|
||||
SolrRecordHeader
|
||||
.newInstance(
|
||||
e.getId(), e.getOriginalId(), RecordType.valueOf(e.getClass().getSimpleName().toLowerCase())));
|
||||
s.setCollectedfrom(asProvenance(e.getCollectedfrom()));
|
||||
s.setContext(asContext(e.getContext(), contextMapper));
|
||||
s.setPid(asPid(e.getPid()));
|
||||
|
||||
if (e instanceof eu.dnetlib.dhp.schema.oaf.Result) {
|
||||
s.setResult(mapResult((eu.dnetlib.dhp.schema.oaf.Result) e));
|
||||
} else if (e instanceof eu.dnetlib.dhp.schema.oaf.Datasource) {
|
||||
s.setDatasource(mapDatasource((eu.dnetlib.dhp.schema.oaf.Datasource) e));
|
||||
} else if (e instanceof eu.dnetlib.dhp.schema.oaf.Organization) {
|
||||
s.setOrganization(mapOrganization((eu.dnetlib.dhp.schema.oaf.Organization) e));
|
||||
} else if (e instanceof eu.dnetlib.dhp.schema.oaf.Project) {
|
||||
s.setProject(mapProject((eu.dnetlib.dhp.schema.oaf.Project) e, vocs));
|
||||
}
|
||||
s
|
||||
.setLinks(
|
||||
Optional
|
||||
.ofNullable(je.getLinks())
|
||||
.map(
|
||||
links -> links
|
||||
.stream()
|
||||
.map(rew -> mapRelatedRecord(rew, vocs))
|
||||
.collect(Collectors.toList()))
|
||||
.orElse(null));
|
||||
|
||||
return s;
|
||||
}
|
||||
|
||||
private static RelatedRecord mapRelatedRecord(RelatedEntityWrapper rew, VocabularyGroup vocs) {
|
||||
RelatedRecord rr = new RelatedRecord();
|
||||
|
||||
final RelatedEntity re = rew.getTarget();
|
||||
final RecordType relatedRecordType = RecordType.valueOf(re.getType());
|
||||
final Relation relation = rew.getRelation();
|
||||
rr
|
||||
.setHeader(
|
||||
RelatedRecordHeader
|
||||
.newInstance(
|
||||
relation.getRelType(),
|
||||
relation.getRelClass(),
|
||||
relation.getTarget(), relatedRecordType));
|
||||
|
||||
rr.setAcronym(re.getAcronym());
|
||||
rr.setCode(re.getCode());
|
||||
rr.setContracttype(mapCodeLabel(re.getContracttype()));
|
||||
rr.setCollectedfrom(asProvenance(re.getCollectedfrom()));
|
||||
rr.setCodeRepositoryUrl(re.getCodeRepositoryUrl());
|
||||
rr.setCountry(asCountry(re.getCountry()));
|
||||
rr.setDatasourcetype(mapCodeLabel(re.getDatasourcetype()));
|
||||
rr.setDatasourcetypeui(mapCodeLabel(re.getDatasourcetypeui()));
|
||||
rr.setDateofacceptance(re.getDateofacceptance());
|
||||
rr.setFunding(mapFunding(re.getFundingtree(), vocs));
|
||||
rr.setInstances(mapInstances(re.getInstances()));
|
||||
rr.setLegalname(re.getLegalname());
|
||||
rr.setLegalshortname(re.getLegalshortname());
|
||||
rr.setOfficialname(re.getOfficialname());
|
||||
rr.setOpenairecompatibility(mapCodeLabel(re.getOpenairecompatibility()));
|
||||
rr.setPid(asPid(re.getPid()));
|
||||
rr.setProjectTitle(rr.getProjectTitle());
|
||||
rr.setPublisher(re.getPublisher());
|
||||
rr.setResulttype(mapQualifier(re.getResulttype()));
|
||||
rr.setTitle(Optional.ofNullable(re.getTitle()).map(StructuredProperty::getValue).orElse(null));
|
||||
|
||||
return rr;
|
||||
}
|
||||
|
||||
private static Project mapProject(eu.dnetlib.dhp.schema.oaf.Project p, VocabularyGroup vocs) {
|
||||
Project ps = new Project();
|
||||
ps.setAcronym(mapField(p.getAcronym()));
|
||||
ps.setCode(mapField(p.getCode()));
|
||||
ps.setContracttype(mapCodeLabel(p.getContracttype()));
|
||||
ps.setCurrency(mapField(p.getCurrency()));
|
||||
ps.setDuration(mapField(p.getDuration()));
|
||||
ps.setCallidentifier(mapField(p.getCallidentifier()));
|
||||
ps.setEcarticle29_3(mapField(p.getEcarticle29_3()));
|
||||
ps.setEnddate(mapField(p.getEnddate()));
|
||||
ps.setFundedamount(p.getFundedamount());
|
||||
ps.setKeywords(mapField(p.getKeywords()));
|
||||
ps.setStartdate(mapField(p.getStartdate()));
|
||||
ps.setSubjects(asSubjectSP(p.getSubjects()));
|
||||
ps.setSummary(mapField(p.getSummary()));
|
||||
ps.setTitle(mapField(p.getTitle()));
|
||||
ps.setTotalcost(p.getTotalcost());
|
||||
ps.setWebsiteurl(mapField(p.getWebsiteurl()));
|
||||
ps.setFunding(mapFundingField(p.getFundingtree(), vocs));
|
||||
return ps;
|
||||
}
|
||||
|
||||
private static Funding mapFunding(List<String> fundingtree, VocabularyGroup vocs) {
|
||||
SAXReader reader = new SAXReader();
|
||||
return Optional
|
||||
.ofNullable(fundingtree)
|
||||
.flatMap(
|
||||
ftree -> ftree
|
||||
.stream()
|
||||
.map(ft -> {
|
||||
try {
|
||||
Document doc = reader.read(new StringReader(ft));
|
||||
String countryCode = doc.valueOf("/fundingtree/funder/jurisdiction/text()");
|
||||
Country country = vocs
|
||||
.find("dnet:countries")
|
||||
.map(voc -> voc.getTerm(countryCode))
|
||||
.map(VocabularyTerm::getName)
|
||||
.map(label -> Country.newInstance(countryCode, label))
|
||||
.orElse(null);
|
||||
|
||||
String level0_id = doc.valueOf("//funding_level_0/id/text()");
|
||||
String level1_id = doc.valueOf("//funding_level_1/id/text()");
|
||||
String level2_id = doc.valueOf("//funding_level_2/id/text()");
|
||||
|
||||
return Funding
|
||||
.newInstance(
|
||||
Funder
|
||||
.newInstance(
|
||||
doc.valueOf("/fundingtree/funder/id/text()"),
|
||||
doc.valueOf("/fundingtree/funder/shortname/text()"),
|
||||
doc.valueOf("/fundingtree/funder/name/text()"),
|
||||
country, new ArrayList<>()),
|
||||
Optional
|
||||
.ofNullable(level0_id)
|
||||
.map(
|
||||
id -> FundingLevel
|
||||
.newInstance(
|
||||
id,
|
||||
doc.valueOf("//funding_level_0/description/text()"),
|
||||
doc.valueOf("//funding_level_0/name/text()")))
|
||||
.orElse(null),
|
||||
Optional
|
||||
.ofNullable(level1_id)
|
||||
.map(
|
||||
id -> FundingLevel
|
||||
.newInstance(
|
||||
id,
|
||||
doc.valueOf("//funding_level_1/description/text()"),
|
||||
doc.valueOf("//funding_level_1/name/text()")))
|
||||
.orElse(null),
|
||||
Optional
|
||||
.ofNullable(level2_id)
|
||||
.map(
|
||||
id -> FundingLevel
|
||||
.newInstance(
|
||||
id,
|
||||
doc.valueOf("//funding_level_2/description/text()"),
|
||||
doc.valueOf("//funding_level_2/name/text()")))
|
||||
.orElse(null));
|
||||
|
||||
} catch (DocumentException e) {
|
||||
throw new IllegalArgumentException(e);
|
||||
}
|
||||
})
|
||||
.findFirst())
|
||||
.orElse(null);
|
||||
}
|
||||
|
||||
private static Funding mapFundingField(List<Field<String>> fundingtree, VocabularyGroup vocs) {
|
||||
return mapFunding(
|
||||
Optional
|
||||
.ofNullable(fundingtree)
|
||||
.map(fts -> fts.stream().map(Field::getValue).collect(Collectors.toList()))
|
||||
.orElse(null),
|
||||
vocs);
|
||||
}
|
||||
|
||||
private static Organization mapOrganization(eu.dnetlib.dhp.schema.oaf.Organization o) {
|
||||
Organization org = new Organization();
|
||||
org.setCountry(mapCodeLabel(o.getCountry()));
|
||||
org.setLegalname(mapField(o.getLegalname()));
|
||||
org.setLegalshortname(mapField(o.getLegalshortname()));
|
||||
org.setAlternativeNames(mapFieldList(o.getAlternativeNames()));
|
||||
org.setWebsiteurl(mapField(o.getWebsiteurl()));
|
||||
org.setLogourl(mapField(o.getLogourl()));
|
||||
|
||||
org.setEcenterprise(mapField(o.getEcenterprise()));
|
||||
org.setEchighereducation(mapField(o.getEchighereducation()));
|
||||
org.setEclegalbody(mapField(o.getEclegalbody()));
|
||||
org.setEcinternationalorganization(mapField(o.getEcinternationalorganization()));
|
||||
org.setEcinternationalorganizationeurinterests(mapField(o.getEcinternationalorganizationeurinterests()));
|
||||
org.setEclegalperson(mapField(o.getEclegalperson()));
|
||||
org.setEcnonprofit(mapField(o.getEcnonprofit()));
|
||||
org.setEcnutscode(mapField(o.getEcnutscode()));
|
||||
org.setEcresearchorganization(mapField(o.getEcresearchorganization()));
|
||||
org.setEcsmevalidated(mapField(o.getEcsmevalidated()));
|
||||
|
||||
return org;
|
||||
}
|
||||
|
||||
private static Datasource mapDatasource(eu.dnetlib.dhp.schema.oaf.Datasource d) {
|
||||
Datasource ds = new Datasource();
|
||||
ds.setEnglishname(mapField(d.getEnglishname()));
|
||||
ds.setOfficialname(mapField(d.getOfficialname()));
|
||||
ds.setDescription(mapField(d.getDescription()));
|
||||
ds.setJournal(mapJournal(d.getJournal()));
|
||||
ds.setLogourl(mapField(d.getLogourl()));
|
||||
ds.setAccessinfopackage(mapFieldList(d.getAccessinfopackage()));
|
||||
ds.setCertificates(mapField(d.getCertificates()));
|
||||
ds.setCitationguidelineurl(mapField(d.getCitationguidelineurl()));
|
||||
ds.setConsenttermsofuse(d.getConsenttermsofuse());
|
||||
ds.setConsenttermsofusedate(d.getConsenttermsofusedate());
|
||||
ds.setContactemail(mapField(d.getContactemail()));
|
||||
ds.setContentpolicies(mapCodeLabel(d.getContentpolicies()));
|
||||
ds.setDatabaseaccessrestriction(mapField(d.getDatabaseaccessrestriction()));
|
||||
ds.setDatabaseaccesstype(mapField(d.getDatabaseaccesstype()));
|
||||
ds.setDataprovider(mapField(d.getDataprovider()));
|
||||
ds.setDatasourcetype(mapCodeLabel(d.getDatasourcetype()));
|
||||
ds.setDatasourcetypeui(mapCodeLabel(d.getDatasourcetypeui()));
|
||||
ds.setDatauploadrestriction(mapField(d.getDatauploadrestriction()));
|
||||
ds.setDatauploadtype(mapField(d.getDatauploadtype()));
|
||||
ds.setDateofvalidation(mapField(d.getDateofvalidation()));
|
||||
ds.setEoscdatasourcetype(mapCodeLabel(d.getEoscdatasourcetype()));
|
||||
ds.setEosctype(mapCodeLabel(d.getEosctype()));
|
||||
ds.setFulltextdownload(d.getFulltextdownload());
|
||||
ds.setJurisdiction(mapCodeLabel(d.getJurisdiction()));
|
||||
ds.setLanguages(d.getLanguages());
|
||||
ds.setLatitude(mapField(d.getLatitude()));
|
||||
ds.setLongitude(mapField(d.getLongitude()));
|
||||
ds.setLastconsenttermsofusedate(d.getLastconsenttermsofusedate());
|
||||
ds.setMissionstatementurl(mapField(d.getMissionstatementurl()));
|
||||
ds.setNamespaceprefix(mapField(d.getNamespaceprefix()));
|
||||
ds.setOdcontenttypes(mapFieldList(d.getOdcontenttypes()));
|
||||
ds.setOdlanguages(mapFieldList(d.getOdlanguages()));
|
||||
ds.setOdnumberofitems(mapField(d.getOdnumberofitems()));
|
||||
ds.setOdnumberofitemsdate(mapField(d.getOdnumberofitemsdate()));
|
||||
ds.setOdpolicies(mapField(d.getOdpolicies()));
|
||||
ds.setOpenairecompatibility(mapCodeLabel(d.getOpenairecompatibility()));
|
||||
ds.setPidsystems(mapField(d.getPidsystems()));
|
||||
ds.setPolicies(mapCodeLabelKV(d.getPolicies()));
|
||||
ds.setPreservationpolicyurl(d.getPreservationpolicyurl());
|
||||
ds.setProvidedproducttypes(ds.getProvidedproducttypes());
|
||||
ds.setReleaseenddate(mapField(d.getReleasestartdate()));
|
||||
ds.setReleasestartdate(mapField(d.getReleasestartdate()));
|
||||
ds.setResearchentitytypes(ds.getResearchentitytypes());
|
||||
ds.setResearchproductaccesspolicies(d.getResearchproductaccesspolicies());
|
||||
ds.setResearchproductmetadataaccesspolicies(d.getResearchproductmetadataaccesspolicies());
|
||||
ds.setServiceprovider(mapField(d.getServiceprovider()));
|
||||
ds.setSubjects(asSubjectSP(d.getSubjects()));
|
||||
ds.setSubmissionpolicyurl(d.getSubmissionpolicyurl());
|
||||
ds.setThematic(d.getThematic());
|
||||
ds.setVersioncontrol(d.getVersioncontrol());
|
||||
ds.setVersioning(mapField(d.getVersioning()));
|
||||
|
||||
return ds;
|
||||
}
|
||||
|
||||
private static Result mapResult(eu.dnetlib.dhp.schema.oaf.Result r) {
|
||||
Result rs = new Result();
|
||||
|
||||
rs.setResulttype(mapQualifier(r.getResulttype()));
|
||||
rs.setAuthor(asAuthor(r.getAuthor()));
|
||||
rs.setMaintitle(getMaintitle(r.getTitle()));
|
||||
rs.setOtherTitles(getOtherTitles(r.getTitle()));
|
||||
rs.setDescription(mapFieldList(r.getDescription()));
|
||||
rs.setSubject(asSubject(r.getSubject()));
|
||||
rs.setPublicationdate(mapField(r.getDateofacceptance()));
|
||||
rs.setPublisher(mapField(r.getPublisher()));
|
||||
rs.setEmbargoenddate(mapField(r.getEmbargoenddate()));
|
||||
rs.setSource(mapFieldList(r.getSource()));
|
||||
rs.setFormat(mapFieldList(r.getFormat()));
|
||||
rs.setContributor(mapFieldList(r.getContributor()));
|
||||
rs.setCoverage(mapFieldList(r.getCoverage()));
|
||||
rs
|
||||
.setBestaccessright(
|
||||
BestAccessRight
|
||||
.newInstance(r.getBestaccessright().getClassid(), r.getBestaccessright().getClassname()));
|
||||
rs.setFulltext(mapFieldList(r.getFulltext()));
|
||||
rs.setCountry(asCountry(r.getCountry()));
|
||||
rs.setEoscifguidelines(asEOSCIF(r.getEoscifguidelines()));
|
||||
|
||||
rs.setGreen(r.getIsGreen());
|
||||
rs
|
||||
.setOpenAccessColor(
|
||||
Optional
|
||||
.ofNullable(r.getOpenAccessColor())
|
||||
.map(color -> OpenAccessColor.valueOf(color.toString()))
|
||||
.orElse(null));
|
||||
rs.setInDiamondJournal(r.getIsInDiamondJournal());
|
||||
rs.setPubliclyFunded(r.getPubliclyFunded());
|
||||
rs.setTransformativeAgreement(r.getTransformativeAgreement());
|
||||
|
||||
rs.setInstance(mapInstances(r.getInstance()));
|
||||
|
||||
if (r instanceof Publication) {
|
||||
Publication pub = (Publication) r;
|
||||
rs.setJournal(mapJournal(pub.getJournal()));
|
||||
} else if (r instanceof Dataset) {
|
||||
Dataset d = (Dataset) r;
|
||||
rs.setSize(mapField(d.getSize()));
|
||||
rs.setVersion(mapField(d.getVersion()));
|
||||
} else if (r instanceof Software) {
|
||||
Software sw = (Software) r;
|
||||
rs.setCodeRepositoryUrl(mapField(sw.getCodeRepositoryUrl()));
|
||||
rs.setProgrammingLanguage(mapQualifier(sw.getProgrammingLanguage()));
|
||||
rs.setDocumentationUrl(mapFieldList(sw.getDocumentationUrl()));
|
||||
} else if (r instanceof OtherResearchProduct) {
|
||||
OtherResearchProduct orp = (OtherResearchProduct) r;
|
||||
rs.setContactperson(mapFieldList(orp.getContactperson()));
|
||||
rs.setContactgroup(mapFieldList(orp.getContactgroup()));
|
||||
rs.setTool(mapFieldList(orp.getTool()));
|
||||
}
|
||||
return rs;
|
||||
}
|
||||
|
||||
@Nullable
|
||||
private static List<String> getOtherTitles(List<StructuredProperty> titleList) {
|
||||
return Optional
|
||||
.ofNullable(titleList)
|
||||
.map(
|
||||
titles -> titles
|
||||
.stream()
|
||||
.filter(
|
||||
t -> !"main title"
|
||||
.equals(
|
||||
Optional
|
||||
.ofNullable(t.getQualifier())
|
||||
.map(Qualifier::getClassid)
|
||||
.orElse(null)))
|
||||
.map(StructuredProperty::getValue)
|
||||
.collect(Collectors.toList()))
|
||||
.orElse(null);
|
||||
}
|
||||
|
||||
private static String getMaintitle(List<StructuredProperty> titleList) {
|
||||
return Optional
|
||||
.ofNullable(titleList)
|
||||
.flatMap(
|
||||
titles -> titles
|
||||
.stream()
|
||||
.filter(
|
||||
t -> "main title"
|
||||
.equals(
|
||||
Optional
|
||||
.ofNullable(t.getQualifier())
|
||||
.map(Qualifier::getClassid)
|
||||
.orElse(null)))
|
||||
.map(StructuredProperty::getValue)
|
||||
.findFirst())
|
||||
.orElse(null);
|
||||
}
|
||||
|
||||
private static List<Instance> mapInstances(List<eu.dnetlib.dhp.schema.oaf.Instance> instanceList) {
|
||||
return Optional
|
||||
.ofNullable(instanceList)
|
||||
.map(
|
||||
instances -> instances
|
||||
.stream()
|
||||
.map(instance -> {
|
||||
Instance i = new Instance();
|
||||
i.setCollectedfrom(asProvenance(instance.getCollectedfrom()));
|
||||
i.setHostedby(asProvenance(instance.getHostedby()));
|
||||
i.setFulltext(i.getFulltext());
|
||||
i.setPid(asPid(instance.getPid()));
|
||||
i.setAlternateIdentifier(asPid(instance.getAlternateIdentifier()));
|
||||
i.setAccessright(mapAccessRight(instance.getAccessright()));
|
||||
i.setInstancetype(mapQualifier(instance.getInstancetype()));
|
||||
i.setLicense(mapField(instance.getLicense()));
|
||||
i.setUrl(instance.getUrl());
|
||||
i.setRefereed(mapQualifier(instance.getRefereed()));
|
||||
i.setDateofacceptance(mapField(instance.getDateofacceptance()));
|
||||
i.setDistributionlocation(instance.getDistributionlocation());
|
||||
i.setProcessingcharges(getProcessingcharges(instance));
|
||||
return i;
|
||||
})
|
||||
.collect(Collectors.toList()))
|
||||
.orElse(null);
|
||||
}
|
||||
|
||||
private static APC getProcessingcharges(eu.dnetlib.dhp.schema.oaf.Instance instance) {
|
||||
return Optional
|
||||
.of(
|
||||
APC
|
||||
.newInstance(
|
||||
mapField(instance.getProcessingchargecurrency()),
|
||||
mapField(instance.getProcessingchargeamount())))
|
||||
.filter(apc -> Objects.nonNull(apc.getAmount()) && Objects.nonNull(apc.getCurrency()))
|
||||
.orElse(null);
|
||||
}
|
||||
|
||||
private static AccessRight mapAccessRight(eu.dnetlib.dhp.schema.oaf.AccessRight accessright) {
|
||||
return AccessRight
|
||||
.newInstance(
|
||||
mapQualifier(accessright),
|
||||
Optional
|
||||
.ofNullable(accessright.getOpenAccessRoute())
|
||||
.map(route -> OpenAccessRoute.valueOf(route.toString()))
|
||||
.orElse(null));
|
||||
}
|
||||
|
||||
private static <T> T mapField(eu.dnetlib.dhp.schema.oaf.Field<T> f) {
|
||||
return Optional.ofNullable(f).map(Field::getValue).orElse(null);
|
||||
}
|
||||
|
||||
private static <T> List<T> mapFieldList(List<eu.dnetlib.dhp.schema.oaf.Field<T>> fl) {
|
||||
return Optional
|
||||
.ofNullable(fl)
|
||||
.map(v -> v.stream().map(Field::getValue).collect(Collectors.toList()))
|
||||
.orElse(null);
|
||||
}
|
||||
|
||||
private static String mapQualifier(eu.dnetlib.dhp.schema.oaf.Qualifier q) {
|
||||
return Optional.ofNullable(q).map(Qualifier::getClassid).orElse(null);
|
||||
}
|
||||
|
||||
private static Journal mapJournal(eu.dnetlib.dhp.schema.oaf.Journal joaf) {
|
||||
return Optional
|
||||
.ofNullable(joaf)
|
||||
.map(jo -> {
|
||||
Journal j = new Journal();
|
||||
j.setConferencedate(jo.getConferencedate());
|
||||
j.setConferenceplace(jo.getConferenceplace());
|
||||
j.setEdition(jo.getEdition());
|
||||
j.setSp(jo.getSp());
|
||||
j.setEp(jo.getEp());
|
||||
j.setVol(jo.getVol());
|
||||
j.setIss(jo.getEdition());
|
||||
j.setName(jo.getName());
|
||||
j.setIssnPrinted(jo.getIssnPrinted());
|
||||
j.setIssnOnline(jo.getIssnOnline());
|
||||
j.setIssnLinking(jo.getIssnLinking());
|
||||
return j;
|
||||
})
|
||||
.orElse(null);
|
||||
}
|
||||
|
||||
private static List<Provenance> asProvenance(List<KeyValue> keyValueList) {
|
||||
return Optional
|
||||
.ofNullable(keyValueList)
|
||||
.map(
|
||||
kvs -> kvs
|
||||
.stream()
|
||||
.map(ProvisionModelSupport::asProvenance)
|
||||
.collect(Collectors.toList()))
|
||||
.orElse(null);
|
||||
}
|
||||
|
||||
private static Provenance asProvenance(KeyValue keyValue) {
|
||||
return Optional.ofNullable(keyValue).map(cf -> Provenance.newInstance(cf.getKey(), cf.getValue())).orElse(null);
|
||||
}
|
||||
|
||||
private static List<Context> asContext(List<eu.dnetlib.dhp.schema.oaf.Context> ctxList,
|
||||
ContextMapper contextMapper) {
|
||||
|
||||
final Set<String> contexts = Optional
|
||||
.ofNullable(ctxList)
|
||||
.map(
|
||||
ctx -> ctx
|
||||
.stream()
|
||||
.map(eu.dnetlib.dhp.schema.oaf.Context::getId)
|
||||
.collect(Collectors.toCollection(HashSet::new)))
|
||||
.orElse(new HashSet<>());
|
||||
|
||||
/* FIXME: Workaround for CLARIN mining issue: #3670#note-29 */
|
||||
if (contexts.contains("dh-ch::subcommunity::2")) {
|
||||
contexts.add("clarin");
|
||||
}
|
||||
|
||||
return Optional
|
||||
.ofNullable(contexts)
|
||||
.map(
|
||||
ctx -> ctx
|
||||
.stream()
|
||||
.map(contextPath -> {
|
||||
Context context = new Context();
|
||||
String id = "";
|
||||
Map<String, Category> categoryMap = Maps.newHashMap();
|
||||
for (final String token : Splitter.on("::").split(contextPath)) {
|
||||
id += token;
|
||||
|
||||
final ContextDef def = contextMapper.get(id);
|
||||
|
||||
if (def == null) {
|
||||
continue;
|
||||
}
|
||||
if (def.getName().equals("context")) {
|
||||
context.setId(def.getId());
|
||||
context.setLabel(def.getLabel());
|
||||
context.setType(def.getType());
|
||||
}
|
||||
if (def.getName().equals("category")) {
|
||||
Category category = Category.newInstance(def.getId(), def.getLabel());
|
||||
if (Objects.isNull(context.getCategory())) {
|
||||
context.setCategory(Lists.newArrayList());
|
||||
}
|
||||
context.getCategory().add(category);
|
||||
categoryMap.put(def.getId(), category);
|
||||
}
|
||||
if (def.getName().equals("concept")) {
|
||||
String parentId = StringUtils.substringBeforeLast(def.getId(), "::");
|
||||
if (categoryMap.containsKey(parentId)) {
|
||||
categoryMap
|
||||
.get(parentId)
|
||||
.getConcept()
|
||||
.add(Concept.newInstance(def.getId(), def.getLabel()));
|
||||
}
|
||||
}
|
||||
id += "::";
|
||||
}
|
||||
return context;
|
||||
})
|
||||
.collect(Collectors.toList()))
|
||||
.orElse(null);
|
||||
}
|
||||
|
||||
private static List<Pid> asPid(List<StructuredProperty> pidList) {
|
||||
return Optional
|
||||
.ofNullable(pidList)
|
||||
.map(
|
||||
pids -> pids
|
||||
.stream()
|
||||
.map(p -> Pid.newInstance(p.getQualifier().getClassid(), p.getValue()))
|
||||
.collect(Collectors.toList()))
|
||||
.orElse(null);
|
||||
}
|
||||
|
||||
private static List<Author> asAuthor(List<eu.dnetlib.dhp.schema.oaf.Author> authorList) {
|
||||
return Optional
|
||||
.ofNullable(authorList)
|
||||
.map(
|
||||
authors -> authors
|
||||
.stream()
|
||||
.map(
|
||||
a -> Author
|
||||
.newInstance(a.getFullname(), a.getName(), a.getSurname(), a.getRank(), asPid(a.getPid())))
|
||||
.collect(Collectors.toList()))
|
||||
.orElse(null);
|
||||
}
|
||||
|
||||
private static List<Subject> asSubject(List<eu.dnetlib.dhp.schema.oaf.Subject> subjectList) {
|
||||
return Optional
|
||||
.ofNullable(subjectList)
|
||||
.map(
|
||||
subjects -> subjects
|
||||
.stream()
|
||||
.filter(s -> Objects.nonNull(s.getQualifier()))
|
||||
.filter(s -> Objects.nonNull(s.getQualifier().getClassid()))
|
||||
.map(s -> Subject.newInstance(s.getValue(), s.getQualifier().getClassid()))
|
||||
.collect(Collectors.toList()))
|
||||
.orElse(null);
|
||||
}
|
||||
|
||||
private static List<Subject> asSubjectSP(List<eu.dnetlib.dhp.schema.oaf.StructuredProperty> subjectList) {
|
||||
return Optional
|
||||
.ofNullable(subjectList)
|
||||
.map(
|
||||
subjects -> subjects
|
||||
.stream()
|
||||
.filter(s -> Objects.nonNull(s.getQualifier()))
|
||||
.filter(s -> Objects.nonNull(s.getQualifier().getClassid()))
|
||||
.map(s -> Subject.newInstance(s.getValue(), s.getQualifier().getClassid()))
|
||||
.collect(Collectors.toList()))
|
||||
.orElse(null);
|
||||
}
|
||||
|
||||
private static Country asCountry(eu.dnetlib.dhp.schema.oaf.Qualifier country) {
|
||||
return Optional
|
||||
.ofNullable(country)
|
||||
.filter(c -> Objects.nonNull(c.getClassid()) && Objects.nonNull(c.getClassname()))
|
||||
.map(c -> Country.newInstance(c.getClassid(), c.getClassname()))
|
||||
.orElse(null);
|
||||
}
|
||||
|
||||
private static List<Country> asCountry(List<eu.dnetlib.dhp.schema.oaf.Country> countryList) {
|
||||
return Optional
|
||||
.ofNullable(countryList)
|
||||
.map(
|
||||
countries -> countries
|
||||
.stream()
|
||||
.map(c -> Country.newInstance(c.getClassid(), c.getClassname()))
|
||||
.collect(Collectors.toList()))
|
||||
.orElse(null);
|
||||
}
|
||||
|
||||
private static List<EoscIfGuidelines> asEOSCIF(List<eu.dnetlib.dhp.schema.oaf.EoscIfGuidelines> eoscIfGuidelines) {
|
||||
return Optional
|
||||
.ofNullable(eoscIfGuidelines)
|
||||
.map(
|
||||
eoscif -> eoscif
|
||||
.stream()
|
||||
.map(
|
||||
e -> EoscIfGuidelines
|
||||
.newInstance(e.getCode(), e.getLabel(), e.getUrl(), e.getSemanticRelation()))
|
||||
.collect(Collectors.toList()))
|
||||
.orElse(null);
|
||||
}
|
||||
|
||||
private static List<CodeLabel> mapCodeLabelKV(List<KeyValue> kvList) {
|
||||
return Optional
|
||||
.ofNullable(kvList)
|
||||
.map(
|
||||
kvs -> kvs
|
||||
.stream()
|
||||
.map(ProvisionModelSupport::mapCodeLabel)
|
||||
.collect(Collectors.toList()))
|
||||
.orElse(null);
|
||||
}
|
||||
|
||||
private static List<CodeLabel> mapCodeLabel(List<Qualifier> qualifiers) {
|
||||
return Optional
|
||||
.ofNullable(qualifiers)
|
||||
.map(
|
||||
list -> list
|
||||
.stream()
|
||||
.map(ProvisionModelSupport::mapCodeLabel)
|
||||
.collect(Collectors.toList()))
|
||||
.orElse(null);
|
||||
}
|
||||
|
||||
private static CodeLabel mapCodeLabel(Qualifier qualifier) {
|
||||
return Optional
|
||||
.ofNullable(qualifier)
|
||||
.map(q -> CodeLabel.newInstance(q.getClassid(), q.getClassname()))
|
||||
.orElse(null);
|
||||
}
|
||||
|
||||
private static CodeLabel mapCodeLabel(KeyValue kv) {
|
||||
return Optional
|
||||
.ofNullable(kv)
|
||||
.map(q -> CodeLabel.newInstance(kv.getKey(), kv.getValue()))
|
||||
.orElse(null);
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -13,6 +13,8 @@ import eu.dnetlib.dhp.schema.oaf.StructuredProperty;
|
|||
|
||||
public class RelatedEntity implements Serializable {
|
||||
|
||||
private static final long serialVersionUID = -4982643490443810597L;
|
||||
|
||||
private String id;
|
||||
private String type;
|
||||
|
||||
|
|
|
@ -9,6 +9,8 @@ import eu.dnetlib.dhp.schema.oaf.Relation;
|
|||
|
||||
public class RelatedEntityWrapper implements Serializable {
|
||||
|
||||
private static final long serialVersionUID = -2624854064081757234L;
|
||||
|
||||
private Relation relation;
|
||||
private RelatedEntity target;
|
||||
|
||||
|
|
|
@ -1,6 +1,7 @@
|
|||
|
||||
package eu.dnetlib.dhp.oa.provision.utils;
|
||||
|
||||
import java.io.Serializable;
|
||||
import java.io.StringReader;
|
||||
import java.io.StringWriter;
|
||||
import java.util.HashMap;
|
||||
|
@ -32,7 +33,7 @@ import com.google.common.collect.Lists;
|
|||
*
|
||||
* @author claudio
|
||||
*/
|
||||
public class StreamingInputDocumentFactory {
|
||||
public class StreamingInputDocumentFactory implements Serializable {
|
||||
|
||||
private static final String INDEX_FIELD_PREFIX = "__";
|
||||
|
||||
|
@ -40,6 +41,8 @@ public class StreamingInputDocumentFactory {
|
|||
|
||||
private static final String INDEX_RESULT = INDEX_FIELD_PREFIX + RESULT;
|
||||
|
||||
private static final String INDEX_JSON_RESULT = INDEX_FIELD_PREFIX + "json";
|
||||
|
||||
private static final String INDEX_RECORD_ID = INDEX_FIELD_PREFIX + "indexrecordidentifier";
|
||||
|
||||
private static final String DEFAULTDNETRESULT = "dnetResult";
|
||||
|
@ -71,13 +74,17 @@ public class StreamingInputDocumentFactory {
|
|||
this.resultName = resultName;
|
||||
}
|
||||
|
||||
public SolrInputDocument parseDocument(final String inputDocument) {
|
||||
public SolrInputDocument parseDocument(final String xml) {
|
||||
return parseDocument(xml, "");
|
||||
}
|
||||
|
||||
public SolrInputDocument parseDocument(final String xml, final String json) {
|
||||
|
||||
final StringWriter results = new StringWriter();
|
||||
final List<Namespace> nsList = Lists.newLinkedList();
|
||||
try {
|
||||
|
||||
XMLEventReader parser = inputFactory.get().createXMLEventReader(new StringReader(inputDocument));
|
||||
XMLEventReader parser = inputFactory.get().createXMLEventReader(new StringReader(xml));
|
||||
|
||||
final SolrInputDocument indexDocument = new SolrInputDocument(new HashMap<>());
|
||||
|
||||
|
@ -95,13 +102,13 @@ public class StreamingInputDocumentFactory {
|
|||
} else if (TARGETFIELDS.equals(localName)) {
|
||||
parseTargetFields(indexDocument, parser);
|
||||
} else if (resultName.equals(localName)) {
|
||||
copyResult(indexDocument, results, parser, nsList, resultName);
|
||||
copyResult(indexDocument, json, results, parser, nsList, resultName);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if (!indexDocument.containsKey(INDEX_RECORD_ID)) {
|
||||
throw new IllegalStateException("cannot extract record ID from: " + inputDocument);
|
||||
throw new IllegalStateException("cannot extract record ID from: " + xml);
|
||||
}
|
||||
|
||||
return indexDocument;
|
||||
|
@ -171,6 +178,7 @@ public class StreamingInputDocumentFactory {
|
|||
*/
|
||||
protected void copyResult(
|
||||
final SolrInputDocument indexDocument,
|
||||
final String json,
|
||||
final StringWriter results,
|
||||
final XMLEventReader parser,
|
||||
final List<Namespace> nsList,
|
||||
|
@ -205,6 +213,7 @@ public class StreamingInputDocumentFactory {
|
|||
}
|
||||
writer.close();
|
||||
indexDocument.addField(INDEX_RESULT, results.toString());
|
||||
indexDocument.addField(INDEX_JSON_RESULT, json);
|
||||
} finally {
|
||||
outputFactory.remove();
|
||||
eventFactory.remove();
|
||||
|
|
|
@ -29,6 +29,12 @@
|
|||
"paramDescription": "decides the job output format, SOLR | HDFS",
|
||||
"paramRequired": false
|
||||
},
|
||||
{
|
||||
"paramName": "si",
|
||||
"paramLongName": "shouldIndex",
|
||||
"paramDescription": "should the action actually index the records?",
|
||||
"paramRequired": true
|
||||
},
|
||||
{
|
||||
"paramName": "op",
|
||||
"paramLongName": "outputPath",
|
||||
|
|
|
@ -16,5 +16,11 @@
|
|||
"paramLongName": "contextApiBaseUrl",
|
||||
"paramDescription": "URL of the context API",
|
||||
"paramRequired": true
|
||||
},
|
||||
{
|
||||
"paramName": "isu",
|
||||
"paramLongName": "isLookupUrl",
|
||||
"paramDescription": "URL of the context ISLookup Service",
|
||||
"paramRequired": true
|
||||
}
|
||||
]
|
||||
|
|
|
@ -28,5 +28,11 @@
|
|||
"paramLongName": "commit",
|
||||
"paramDescription": "should the action be followed by a commit?",
|
||||
"paramRequired": false
|
||||
},
|
||||
{
|
||||
"paramName": "i",
|
||||
"paramLongName": "shouldIndex",
|
||||
"paramDescription": "should the action actually index the records?",
|
||||
"paramRequired": true
|
||||
}
|
||||
]
|
|
@ -591,22 +591,18 @@
|
|||
--conf spark.sql.shuffle.partitions=3840
|
||||
--conf spark.network.timeout=${sparkNetworkTimeout}
|
||||
</spark-opts>
|
||||
<arg>--inputPath</arg><arg>${workingDir}/join_entities</arg>
|
||||
<arg>--outputPath</arg><arg>${workingDir}/xml</arg>
|
||||
<!--<arg>- -inputPath</arg><arg>${workingDir}/join_entities</arg>-->
|
||||
<arg>--inputPath</arg><arg>/user/claudio.atzori/data/provision/join_entities</arg>
|
||||
|
||||
<!--<arg>- -outputPath</arg><arg>${workingDir}/xml_json</arg>-->
|
||||
<arg>--outputPath</arg><arg>/user/claudio.atzori/data/provision/xml_json_test</arg>
|
||||
<arg>--contextApiBaseUrl</arg><arg>${contextApiBaseUrl}</arg>
|
||||
<arg>--isLookupUrl</arg><arg>${isLookupUrl}</arg>
|
||||
</spark>
|
||||
<ok to="should_index"/>
|
||||
<ok to="drop_solr_collection"/>
|
||||
<error to="Kill"/>
|
||||
</action>
|
||||
|
||||
<decision name="should_index">
|
||||
<switch>
|
||||
<case to="drop_solr_collection">${wf:conf('shouldIndex') eq 'true'}</case>
|
||||
<case to="End">${wf:conf('shouldIndex') eq 'false'}</case>
|
||||
<default to="drop_solr_collection"/>
|
||||
</switch>
|
||||
</decision>
|
||||
|
||||
<action name="drop_solr_collection">
|
||||
<java>
|
||||
<configuration>
|
||||
|
@ -621,6 +617,7 @@
|
|||
<arg>--action</arg><arg>DELETE_BY_QUERY</arg>
|
||||
<arg>--query</arg><arg>${solrDeletionQuery}</arg>
|
||||
<arg>--commit</arg><arg>true</arg>
|
||||
<arg>--shouldIndex</arg><arg>${shouldIndex}</arg>
|
||||
</java>
|
||||
<ok to="to_solr_index"/>
|
||||
<error to="Kill"/>
|
||||
|
@ -646,12 +643,17 @@
|
|||
--conf spark.hadoop.mapreduce.map.speculative=false
|
||||
--conf spark.hadoop.mapreduce.reduce.speculative=false
|
||||
</spark-opts>
|
||||
<arg>--inputPath</arg><arg>${workingDir}/xml</arg>
|
||||
<!--<arg>- -inputPath</arg><arg>${workingDir}/xml_json</arg>-->
|
||||
<arg>--inputPath</arg><arg>/user/claudio.atzori/data/provision/xml_json_test</arg>
|
||||
<arg>--isLookupUrl</arg><arg>${isLookupUrl}</arg>
|
||||
<arg>--format</arg><arg>${format}</arg>
|
||||
<arg>--batchSize</arg><arg>${batchSize}</arg>
|
||||
<arg>--outputFormat</arg><arg>${outputFormat}</arg>
|
||||
<arg>--outputPath</arg><arg>${workingDir}/solr_documents</arg>
|
||||
|
||||
<!--<arg>- -outputPath</arg><arg>${workingDir}/solr_documents</arg>-->
|
||||
<arg>--outputPath</arg><arg>/user/claudio.atzori/data/provision/solr_documents</arg>
|
||||
|
||||
<arg>--shouldIndex</arg><arg>${shouldIndex}</arg>
|
||||
</spark>
|
||||
<ok to="commit_solr_collection"/>
|
||||
<error to="Kill"/>
|
||||
|
@ -669,6 +671,7 @@
|
|||
<arg>--isLookupUrl</arg><arg>${isLookupUrl}</arg>
|
||||
<arg>--format</arg><arg>${format}</arg>
|
||||
<arg>--action</arg><arg>COMMIT</arg>
|
||||
<arg>--shouldIndex</arg><arg>${shouldIndex}</arg>
|
||||
</java>
|
||||
<ok to="End"/>
|
||||
<error to="Kill"/>
|
||||
|
|
|
@ -57,7 +57,7 @@ public class EOSCFuture_Test {
|
|||
IOUtils.toString(getClass().getResourceAsStream("eosc-future/photic-zone.json")),
|
||||
OtherResearchProduct.class);
|
||||
|
||||
final String xml = xmlRecordFactory.build(new JoinedEntity<>(p));
|
||||
final String xml = xmlRecordFactory.build(new JoinedEntity(p));
|
||||
|
||||
assertNotNull(xml);
|
||||
|
||||
|
|
|
@ -63,7 +63,7 @@ public class IndexRecordTransformerTest {
|
|||
final Project pj = load("project.json", Project.class);
|
||||
final Relation rel = load("relToValidatedProject.json", Relation.class);
|
||||
|
||||
final JoinedEntity je = new JoinedEntity<>(p);
|
||||
final JoinedEntity je = new JoinedEntity(p);
|
||||
je
|
||||
.setLinks(
|
||||
Lists
|
||||
|
@ -86,7 +86,7 @@ public class IndexRecordTransformerTest {
|
|||
|
||||
final Publication p = load("publication.json", Publication.class);
|
||||
|
||||
final JoinedEntity<Publication> je = new JoinedEntity<>(p);
|
||||
final JoinedEntity je = new JoinedEntity(p);
|
||||
final String record = xmlRecordFactory.build(je);
|
||||
assertNotNull(record);
|
||||
SolrInputDocument solrDoc = testRecordTransformation(record);
|
||||
|
@ -102,7 +102,7 @@ public class IndexRecordTransformerTest {
|
|||
|
||||
final Publication p = load("riunet.json", Publication.class);
|
||||
|
||||
final JoinedEntity je = new JoinedEntity<>(p);
|
||||
final JoinedEntity je = new JoinedEntity(p);
|
||||
final String record = xmlRecordFactory.build(je);
|
||||
assertNotNull(record);
|
||||
testRecordTransformation(record);
|
||||
|
|
|
@ -0,0 +1,51 @@
|
|||
/*
|
||||
* Copyright (c) 2024.
|
||||
* SPDX-FileCopyrightText: © 2023 Consiglio Nazionale delle Ricerche
|
||||
* SPDX-License-Identifier: AGPL-3.0-or-later
|
||||
*/
|
||||
|
||||
package eu.dnetlib.dhp.oa.provision;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonInclude;
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
|
||||
import eu.dnetlib.dhp.oa.provision.model.JoinedEntity;
|
||||
import eu.dnetlib.dhp.schema.oaf.Field;
|
||||
import eu.dnetlib.dhp.schema.oaf.Journal;
|
||||
import eu.dnetlib.dhp.schema.oaf.Organization;
|
||||
import eu.dnetlib.dhp.schema.oaf.Publication;
|
||||
|
||||
class JoinedEntityTest {
|
||||
|
||||
private static final Logger log = LoggerFactory.getLogger(JoinedEntityTest.class);
|
||||
|
||||
@Test
|
||||
void test_serialisation() throws IOException {
|
||||
|
||||
Publication p = new Publication();
|
||||
p.setId("p1");
|
||||
Journal j = new Journal();
|
||||
j.setIss("1234-5678");
|
||||
p.setJournal(j);
|
||||
|
||||
Organization o = new Organization();
|
||||
o.setId("o1");
|
||||
Field<String> lName = new Field<>();
|
||||
lName.setValue("CNR");
|
||||
o.setLegalname(lName);
|
||||
|
||||
ObjectMapper mapper = new ObjectMapper();
|
||||
mapper.setSerializationInclusion(JsonInclude.Include.NON_EMPTY);
|
||||
|
||||
final String json = mapper.writeValueAsString(new JoinedEntity(p));
|
||||
log.info(json);
|
||||
|
||||
}
|
||||
|
||||
}
|
|
@ -86,7 +86,7 @@ public class SolrConfigExploreTest extends SolrExploreTest {
|
|||
|
||||
String inputPath = "src/test/resources/eu/dnetlib/dhp/oa/provision/xml";
|
||||
|
||||
new XmlIndexingJob(spark, inputPath, FORMAT, batchSize, XmlIndexingJob.OutputFormat.SOLR, null)
|
||||
new XmlIndexingJob(spark, inputPath, FORMAT, batchSize, XmlIndexingJob.OutputFormat.SOLR, true, null)
|
||||
.run(isLookupClient);
|
||||
Assertions.assertEquals(0, miniCluster.getSolrClient().commit().getStatus());
|
||||
|
||||
|
|
|
@ -95,7 +95,7 @@ public class SolrConfigTest extends SolrTest {
|
|||
|
||||
String inputPath = "src/test/resources/eu/dnetlib/dhp/oa/provision/xml";
|
||||
|
||||
new XmlIndexingJob(spark, inputPath, FORMAT, batchSize, XmlIndexingJob.OutputFormat.SOLR, null)
|
||||
new XmlIndexingJob(spark, inputPath, FORMAT, batchSize, XmlIndexingJob.OutputFormat.SOLR, false, null)
|
||||
.run(isLookupClient);
|
||||
Assertions.assertEquals(0, miniCluster.getSolrClient().commit().getStatus());
|
||||
|
||||
|
|
|
@ -1,35 +1,35 @@
|
|||
|
||||
package eu.dnetlib.dhp.oa.provision;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.*;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.StringReader;
|
||||
import java.net.URI;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.hadoop.io.Text;
|
||||
import org.apache.solr.client.solrj.SolrQuery;
|
||||
import org.apache.solr.client.solrj.response.QueryResponse;
|
||||
import org.apache.solr.common.SolrInputField;
|
||||
import org.apache.solr.common.params.CommonParams;
|
||||
import org.apache.solr.common.params.ModifiableSolrParams;
|
||||
import org.apache.spark.SparkConf;
|
||||
import org.apache.spark.api.java.JavaPairRDD;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
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.SparkSession;
|
||||
import org.dom4j.io.SAXReader;
|
||||
import org.junit.jupiter.api.*;
|
||||
import org.junit.jupiter.api.AfterAll;
|
||||
import org.junit.jupiter.api.BeforeAll;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.junit.jupiter.api.extension.ExtendWith;
|
||||
import org.mockito.Mock;
|
||||
import org.mockito.Mockito;
|
||||
import org.mockito.junit.jupiter.MockitoExtension;
|
||||
|
||||
import eu.dnetlib.dhp.oa.provision.model.SerializableSolrInputDocument;
|
||||
import eu.dnetlib.dhp.oa.provision.model.TupleWrapper;
|
||||
import eu.dnetlib.dhp.oa.provision.utils.ISLookupClient;
|
||||
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException;
|
||||
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService;
|
||||
|
@ -70,6 +70,7 @@ public class XmlIndexingJobTest extends SolrTest {
|
|||
|
||||
SparkConf conf = new SparkConf();
|
||||
conf.setAppName(XmlIndexingJobTest.class.getSimpleName());
|
||||
|
||||
conf.registerKryoClasses(new Class[] {
|
||||
SerializableSolrInputDocument.class
|
||||
});
|
||||
|
@ -97,12 +98,15 @@ public class XmlIndexingJobTest extends SolrTest {
|
|||
|
||||
String inputPath = "src/test/resources/eu/dnetlib/dhp/oa/provision/xml";
|
||||
|
||||
long nRecord = JavaSparkContext
|
||||
.fromSparkContext(spark.sparkContext())
|
||||
.sequenceFile(inputPath, Text.class, Text.class)
|
||||
.count();
|
||||
Dataset<TupleWrapper> records = spark
|
||||
.read()
|
||||
.schema(Encoders.bean(TupleWrapper.class).schema())
|
||||
.json(inputPath)
|
||||
.as(Encoders.bean(TupleWrapper.class));
|
||||
|
||||
new XmlIndexingJob(spark, inputPath, FORMAT, batchSize, XmlIndexingJob.OutputFormat.SOLR, null)
|
||||
long nRecord = records.count();
|
||||
|
||||
new XmlIndexingJob(spark, inputPath, FORMAT, batchSize, XmlIndexingJob.OutputFormat.SOLR, true, null)
|
||||
.run(isLookupClient);
|
||||
|
||||
assertEquals(0, miniCluster.getSolrClient().commit().getStatus());
|
||||
|
@ -137,40 +141,93 @@ public class XmlIndexingJobTest extends SolrTest {
|
|||
assertEquals(
|
||||
0, rsp.getResults().getNumFound(),
|
||||
"the number of indexed records having peerreviewed = true");
|
||||
|
||||
rsp = miniCluster
|
||||
.getSolrClient()
|
||||
.query(
|
||||
new SolrQuery()
|
||||
.add(CommonParams.Q, "objidentifier:\"iddesignpres::ae77e56e84ad058d9e7f19fa2f7325db\"")
|
||||
.add(CommonParams.FL, "__json"));
|
||||
assertEquals(
|
||||
1, rsp.getResults().getNumFound(),
|
||||
"the number of indexed records having the given identifier");
|
||||
Optional<Object> json = rsp
|
||||
.getResults()
|
||||
.stream()
|
||||
.map(d -> d.getFieldValues("__json"))
|
||||
.flatMap(d -> d.stream())
|
||||
.findFirst();
|
||||
|
||||
assertTrue(json.isPresent());
|
||||
|
||||
log.info((String) json.get());
|
||||
|
||||
}
|
||||
|
||||
@Test
|
||||
void testXmlIndexingJob_saveOnHDFS() throws Exception {
|
||||
final String ID_XPATH = "//header/*[local-name()='objIdentifier']";
|
||||
final String ID_XPATH = "//*[local-name()='header']/*[local-name()='objIdentifier']";
|
||||
|
||||
String inputPath = "src/test/resources/eu/dnetlib/dhp/oa/provision/xml";
|
||||
// String inputPath = "/Users/claudio/workspace/data/index";
|
||||
|
||||
final JavaPairRDD<Text, Text> xmlRecords = JavaSparkContext
|
||||
.fromSparkContext(spark.sparkContext())
|
||||
.sequenceFile(inputPath, Text.class, Text.class);
|
||||
long nRecord = xmlRecords.count();
|
||||
long xmlIdUnique = xmlRecords
|
||||
.map(t -> t._2().toString())
|
||||
.map(s -> new SAXReader().read(new StringReader(s)).valueOf(ID_XPATH))
|
||||
Dataset<TupleWrapper> records = spark
|
||||
.read()
|
||||
.schema(Encoders.bean(TupleWrapper.class).schema())
|
||||
.json(inputPath)
|
||||
.as(Encoders.bean(TupleWrapper.class));
|
||||
|
||||
records.printSchema();
|
||||
|
||||
long nRecord = records.count();
|
||||
log.info("found {} records", nRecord);
|
||||
|
||||
final Dataset<String> ids = records
|
||||
.map((MapFunction<TupleWrapper, String>) TupleWrapper::getXml, Encoders.STRING())
|
||||
.map(
|
||||
(MapFunction<String, String>) s -> new SAXReader().read(new StringReader(s)).valueOf(ID_XPATH),
|
||||
Encoders.STRING());
|
||||
|
||||
log.info("found {} ids", ids.count());
|
||||
|
||||
long xmlIdUnique = ids
|
||||
.distinct()
|
||||
.count();
|
||||
|
||||
log.info("found {} unique ids", xmlIdUnique);
|
||||
|
||||
assertEquals(nRecord, xmlIdUnique, "IDs should be unique among input records");
|
||||
|
||||
final String outputPath = workingDir.resolve("outputPath").toAbsolutePath().toString();
|
||||
new XmlIndexingJob(spark, inputPath, FORMAT, batchSize, XmlIndexingJob.OutputFormat.HDFS, outputPath)
|
||||
new XmlIndexingJob(spark, inputPath, FORMAT, batchSize, XmlIndexingJob.OutputFormat.HDFS, false, outputPath)
|
||||
.run(isLookupClient);
|
||||
|
||||
final Dataset<SerializableSolrInputDocument> solrDocs = spark
|
||||
.read()
|
||||
.load(outputPath)
|
||||
.as(Encoders.kryo(SerializableSolrInputDocument.class));
|
||||
|
||||
solrDocs.foreach(doc -> {
|
||||
assertNotNull(doc.get("__result"));
|
||||
assertNotNull(doc.get("__json"));
|
||||
});
|
||||
|
||||
long docIdUnique = solrDocs.map((MapFunction<SerializableSolrInputDocument, String>) doc -> {
|
||||
final SolrInputField id = doc.getField("__indexrecordidentifier");
|
||||
return id.getFirstValue().toString();
|
||||
}, Encoders.STRING())
|
||||
.distinct()
|
||||
.count();
|
||||
assertEquals(xmlIdUnique, docIdUnique, "IDs should be unique among the output records");
|
||||
assertEquals(xmlIdUnique, docIdUnique, "IDs should be unique among the output XML records");
|
||||
|
||||
long jsonUnique = solrDocs
|
||||
.map(
|
||||
(MapFunction<SerializableSolrInputDocument, String>) je -> (String) je.getField("__json").getValue(),
|
||||
Encoders.STRING())
|
||||
.distinct()
|
||||
.count();
|
||||
|
||||
assertEquals(jsonUnique, docIdUnique, "IDs should be unique among the output JSON records");
|
||||
|
||||
}
|
||||
|
||||
|
|
|
@ -42,7 +42,7 @@ public class XmlRecordFactoryTest {
|
|||
final Publication p = OBJECT_MAPPER
|
||||
.readValue(IOUtils.toString(getClass().getResourceAsStream("publication.json")), Publication.class);
|
||||
|
||||
final String xml = xmlRecordFactory.build(new JoinedEntity<>(p));
|
||||
final String xml = xmlRecordFactory.build(new JoinedEntity(p));
|
||||
|
||||
assertNotNull(xml);
|
||||
|
||||
|
@ -117,7 +117,7 @@ public class XmlRecordFactoryTest {
|
|||
final List<RelatedEntityWrapper> links = Lists.newArrayList();
|
||||
final RelatedEntityWrapper rew = new RelatedEntityWrapper(rel, relatedProject);
|
||||
links.add(rew);
|
||||
final JoinedEntity je = new JoinedEntity<>(p);
|
||||
final JoinedEntity je = new JoinedEntity(p);
|
||||
je.setLinks(links);
|
||||
|
||||
final String xml = xmlRecordFactory.build(je);
|
||||
|
@ -148,7 +148,7 @@ public class XmlRecordFactoryTest {
|
|||
final List<RelatedEntityWrapper> links = Lists.newArrayList();
|
||||
final RelatedEntityWrapper rew = new RelatedEntityWrapper(rel, relatedProject);
|
||||
links.add(rew);
|
||||
final JoinedEntity je = new JoinedEntity<>(p);
|
||||
final JoinedEntity je = new JoinedEntity(p);
|
||||
je.setLinks(links);
|
||||
|
||||
final String xml = xmlRecordFactory.build(je);
|
||||
|
@ -171,7 +171,7 @@ public class XmlRecordFactoryTest {
|
|||
final Datasource d = OBJECT_MAPPER
|
||||
.readValue(IOUtils.toString(getClass().getResourceAsStream("datasource.json")), Datasource.class);
|
||||
|
||||
final String xml = xmlRecordFactory.build(new JoinedEntity<>(d));
|
||||
final String xml = xmlRecordFactory.build(new JoinedEntity(d));
|
||||
|
||||
assertNotNull(xml);
|
||||
|
||||
|
@ -210,7 +210,7 @@ public class XmlRecordFactoryTest {
|
|||
IOUtils.toString(getClass().getResourceAsStream("d4science-1-training.json")),
|
||||
OtherResearchProduct.class);
|
||||
|
||||
final String xml = xmlRecordFactory.build(new JoinedEntity<>(p));
|
||||
final String xml = xmlRecordFactory.build(new JoinedEntity(p));
|
||||
|
||||
assertNotNull(xml);
|
||||
|
||||
|
@ -233,7 +233,7 @@ public class XmlRecordFactoryTest {
|
|||
IOUtils.toString(getClass().getResourceAsStream("d4science-2-dataset.json")),
|
||||
OtherResearchProduct.class);
|
||||
|
||||
final String xml = xmlRecordFactory.build(new JoinedEntity<>(p));
|
||||
final String xml = xmlRecordFactory.build(new JoinedEntity(p));
|
||||
|
||||
assertNotNull(xml);
|
||||
|
||||
|
@ -256,7 +256,7 @@ public class XmlRecordFactoryTest {
|
|||
IOUtils.toString(getClass().getResourceAsStream("iris-odf-4.json")),
|
||||
Publication.class);
|
||||
|
||||
final String xml = xmlRecordFactory.build(new JoinedEntity<>(p));
|
||||
final String xml = xmlRecordFactory.build(new JoinedEntity(p));
|
||||
|
||||
assertNotNull(xml);
|
||||
|
||||
|
|
|
@ -195,186 +195,147 @@
|
|||
<fieldType name="tlong" class="solr.TrieLongField" positionIncrementGap="0" docValues="true" precisionStep="8"/>
|
||||
<fieldType name="tlongs" class="solr.TrieLongField" positionIncrementGap="0" docValues="true" multiValued="true" precisionStep="8"/>
|
||||
<field name="__all" type="text_common" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="__deleted" type="boolean" default="false" omitNorms="true" omitTermFreqAndPositions="true" indexed="true" stored="false"/>
|
||||
<field name="__dsid" type="string" omitNorms="true" omitTermFreqAndPositions="true" indexed="true" stored="true"/>
|
||||
<field name="__dsversion" type="pdate" omitNorms="true" omitTermFreqAndPositions="true" indexed="true" stored="true"/>
|
||||
<field name="__indexrecordidentifier" type="string" multiValued="false" indexed="true" required="true" stored="true"/>
|
||||
<field name="__result" type="string" docValues="false" multiValued="false" indexed="false" stored="true"/>
|
||||
<field name="__json" type="string" docValues="false" multiValued="false" indexed="false" stored="true"/>
|
||||
<field name="_root_" type="string" docValues="false" indexed="true" stored="false"/>
|
||||
<field name="_version_" type="long" multiValued="false" indexed="true" stored="true"/>
|
||||
<field name="authorid" type="string_ci" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="authoridtype" type="string_ci" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="categoryid" type="string" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="categoryname" type="string" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="collectedfrom" type="string" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="collectedfromdatasourceid" type="string" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="collectedfromname" type="string" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="community" type="string" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="communityid" type="string" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="communityname" type="string" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="conceptid" type="string" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="conceptname" type="string" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="contextid" type="string" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="contextname" type="string" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="contexttype" type="string" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="counter_affiliation" type="pint" multiValued="false" indexed="true" stored="true"/>
|
||||
<field name="counter_authorship" type="pint" multiValued="false" indexed="true" stored="true"/>
|
||||
<field name="counter_dedup" type="pint" multiValued="false" indexed="true" stored="true"/>
|
||||
<field name="counter_doi" type="pint" multiValued="false" indexed="true" stored="true"/>
|
||||
<field name="counter_outcome" type="pint" multiValued="false" indexed="true" stored="true"/>
|
||||
<field name="counter_outcome_claimed" type="pint" multiValued="false" indexed="true" stored="true"/>
|
||||
<field name="counter_outcome_collected" type="pint" multiValued="false" indexed="true" stored="true"/>
|
||||
<field name="counter_outcome_inferred" type="pint" multiValued="false" indexed="true" stored="true"/>
|
||||
<field name="counter_participation" type="pint" multiValued="false" indexed="true" stored="true"/>
|
||||
<field name="counter_publicationdataset" type="pint" multiValued="false" indexed="true" stored="true"/>
|
||||
<field name="counter_publicationdataset_claimed" type="pint" multiValued="false" indexed="true" stored="true"/>
|
||||
<field name="counter_publicationdataset_collected" type="pint" multiValued="false" indexed="true" stored="true"/>
|
||||
<field name="counter_publicationdataset_inferred" type="pint" multiValued="false" indexed="true" stored="true"/>
|
||||
<field name="counter_similarity" type="pint" multiValued="false" indexed="true" stored="true"/>
|
||||
<field name="country" type="string" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="datasourcecompatibilityid" type="string" multiValued="false" indexed="true" stored="false"/>
|
||||
<field name="datasourcecompatibilityname" type="string" multiValued="false" indexed="true" stored="false"/>
|
||||
<field name="datasourceenglishname" type="text_common" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="datasourceodcontenttypes" type="string" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="datasourceoddescription" type="text_common" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="datasourceodlanguages" type="string" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="datasourceodsubjects" type="string" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="datasourceofficialname" type="text_common" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="datasourcesubject" type="ngramtext" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="datasourcetypename" type="string" multiValued="false" indexed="true" stored="false"/>
|
||||
<field name="datasourcetypeuiid" type="string" multiValued="false" indexed="true" stored="false"/>
|
||||
<field name="datasourcetypeuiname" type="string" multiValued="false" indexed="true" stored="false"/>
|
||||
<field name="dateofcollection" type="pdate" multiValued="false" indexed="true" stored="false"/>
|
||||
<field name="deletedbyinference" type="string" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="externalrefclass" type="string" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="externalrefid" type="string" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="externalreflabel" type="text_common" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="externalrefsite" type="string" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="funder" type="string" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="funderid" type="string" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="funderjurisdiction" type="string" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="fundername" type="string" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="funderoriginalname" type="string" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="fundershortname" type="string" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="fundinglevel0_description" type="text_common" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="fundinglevel0_id" type="string" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="fundinglevel0_name" type="string" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="fundinglevel1_description" type="text_common" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="fundinglevel1_id" type="string" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="fundinglevel1_name" type="string" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="fundinglevel2_description" type="text_common" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="fundinglevel2_id" type="string" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="fundinglevel2_name" type="string" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="inferenceprovenance" type="string" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="inferred" type="string" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="instancetypename" type="string" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="oafentity" type="string" multiValued="true" indexed="false" stored="false"/>
|
||||
<field name="oaftype" type="string" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="objidentifier" type="string" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="organizationalternativenames" type="ngramtext" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="organizationcountryname" type="string" multiValued="false" indexed="true" stored="false"/>
|
||||
<field name="organizationdupid" type="string" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="organizationecenterprise" type="text_common" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="organizationecinternationalorganization" type="text_common" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="organizationecinternationalorganizationeurinterests" type="text_common" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="organizationeclegalbody" type="text_common" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="organizationeclegalperson" type="text_common" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="organizationecnonprofit" type="text_common" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="organizationecnutscode" type="text_common" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="organizationecresearchorganization" type="text_common" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="organizationecsmevalidated" type="text_common" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="organizationlegalname" type="ngramtext" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="organizationlegalshortname" type="ngramtext" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="originalid" type="string_ci" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="pid" type="string_ci" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="pidclassid" type="string" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="pidclassname" type="string" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="projectacronym" type="ngramtext" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="projectcallidentifier" type="string" multiValued="false" indexed="true" stored="false"/>
|
||||
<field name="projectcode" type="ngramtext" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="projectcode_nt" type="string" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="projectcontracttypename" type="string" multiValued="false" indexed="true" stored="false"/>
|
||||
<field name="projectduration" type="string" multiValued="false" indexed="true" stored="false"/>
|
||||
<field name="projectecarticle29_3" type="string" multiValued="false" indexed="true" stored="false"/>
|
||||
<field name="projectecsc39" type="string" multiValued="false" indexed="true" stored="false"/>
|
||||
<field name="projectenddate" type="pdate" multiValued="false" indexed="true" stored="false"/>
|
||||
<field name="projectendyear" type="string" multiValued="false" indexed="true" stored="false"/>
|
||||
<field name="projectkeywords" type="text_common" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="projectoamandatepublications" type="string" multiValued="false" indexed="true" stored="false"/>
|
||||
<field name="projectstartdate" type="pdate" multiValued="false" indexed="true" stored="false"/>
|
||||
<field name="projectstartyear" type="string" multiValued="false" indexed="true" stored="false"/>
|
||||
<field name="projectsubject" type="string" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="projecttitle" type="ngramtext" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="provenanceactionclassid" type="string" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="relclass" type="string" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="relcollectedfromid" type="string" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="relcollectedfromname" type="string" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="relcontracttypeid" type="string" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="relcontracttypename" type="text_common" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="reldatasourcecompatibilityid" type="string" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="relfunder" type="string" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="relfunderid" type="string" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="relfunderjurisdiction" type="string" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="relfundername" type="string" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="relfundershortname" type="string" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="relfundinglevel0_id" type="string" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="relfundinglevel0_name" type="string" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="relfundinglevel1_id" type="string" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="relfundinglevel1_name" type="string" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="relfundinglevel2_id" type="string" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="relfundinglevel2_name" type="string" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="relinferenceprovenance" type="string" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="relinferred" type="string" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="relorganizationcountryid" type="string" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="relorganizationcountryname" type="text_common" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="relorganizationid" type="string" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="relorganizationname" type="text_common" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="relorganizationshortname" type="text_common" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="relproject" type="string" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="relprojectcode" type="string" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="relprojectid" type="string" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="relprojectname" type="string" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="relprojecttitle" type="text_common" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="relprovenanceactionclassid" type="string" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="relresultid" type="string" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="relresulttype" type="string" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="reltrust" type="string" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="resultacceptanceyear" type="string" multiValued="false" indexed="true" stored="false"/>
|
||||
<field name="resultaccessright" type="string" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="resultauthor" type="text_common" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="resultauthor_nt" type="string_ci" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="resultbestaccessright" type="string" multiValued="false" indexed="true" stored="false"/>
|
||||
<field name="resultdateofacceptance" type="pdate" multiValued="false" indexed="true" stored="false"/>
|
||||
<field name="resultdescription" type="text_common" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="resultdupid" type="string" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="resultembargoenddate" type="pdate" multiValued="false" indexed="true" stored="false"/>
|
||||
<field name="resultembargoendyear" type="string" multiValued="false" indexed="true" stored="false"/>
|
||||
<field name="resulthostingdatasource" type="string" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="resulthostingdatasourceid" type="string" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="resulthostingdatasourcename" type="string" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="resultidentifier" type="text_common" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="resultlanguagename" type="string" multiValued="false" indexed="true" stored="false"/>
|
||||
<field name="resultlicense" type="string" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="resultpublisher" type="text_common" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="resultsource" type="text_common" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="resultsubject" type="string" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="resultsubjectclass" type="string" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="resulttitle" type="text_common" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="resulttypeid" type="string" multiValued="false" indexed="true" stored="false"/>
|
||||
<field name="resulttypename" type="string" multiValued="false" indexed="true" stored="false"/>
|
||||
<field name="semrelid" type="string" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="authorid" type="string_ci" docValues="false" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="categoryid" type="string" docValues="false" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="citation_count" type="pint" docValues="false" multiValued="false" indexed="true" stored="false"/>
|
||||
<field name="citation_count_class" type="string" docValues="false" multiValued="false" indexed="true" stored="false"/>
|
||||
<field name="collectedfromdatasourceid" type="string" docValues="false" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="collectedfromname" type="string" docValues="false" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="community" type="string" docValues="false" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="communityid" type="string" docValues="false" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="conceptname" type="string" docValues="false" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="contextid" type="string" docValues="false" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="contextname" type="string" docValues="false" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="country" type="string" docValues="false" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="countrynojurisdiction" type="string" docValues="false" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="datasourcecompatibilityid" type="string" docValues="false" multiValued="false" indexed="true" stored="false"/>
|
||||
<field name="datasourcecompatibilityname" type="string" docValues="false" multiValued="false" indexed="true" stored="false"/>
|
||||
<field name="datasourceenglishname" type="text_common" docValues="false" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="datasourcejurisdiction" type="string" docValues="false" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="datasourceodcontenttypes" type="string" docValues="false" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="datasourceoddescription" type="text_common" docValues="false" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="datasourceodlanguages" type="string" docValues="false" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="datasourceodsubjects" type="string" docValues="false" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="datasourceofficialname" type="text_common" docValues="false" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="datasourcesubject" type="ngramtext" docValues="false" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="datasourcethematic" type="string" docValues="false" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="datasourcetypename" type="string" docValues="false" multiValued="false" indexed="true" stored="false"/>
|
||||
<field name="datasourcetypeuiid" type="string" docValues="false" multiValued="false" indexed="true" stored="false"/>
|
||||
<field name="datasourcetypeuiname" type="string" docValues="false" multiValued="false" indexed="true" stored="false"/>
|
||||
<field name="dateofcollection" type="date" docValues="false" multiValued="false" indexed="true" stored="false"/>
|
||||
<field name="deletedbyinference" type="string" docValues="false" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="eoscdatasourcetype" type="string" docValues="false" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="eoscifguidelines" type="string" docValues="false" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="eosctype" type="string" docValues="false" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="externalreflabel" type="text_common" docValues="false" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="fos" type="string" docValues="false" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="foslabel" type="string" docValues="false" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="funder" type="string" docValues="false" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="funderid" type="string" docValues="false" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="fundershortname" type="string" docValues="false" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="fundinglevel0_description" type="text_common" docValues="false" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="fundinglevel0_id" type="string" docValues="false" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="fundinglevel0_name" type="string" docValues="false" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="fundinglevel1_description" type="text_common" docValues="false" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="fundinglevel1_id" type="string" docValues="false" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="fundinglevel1_name" type="string" docValues="false" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="fundinglevel2_description" type="text_common" docValues="false" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="fundinglevel2_id" type="string" docValues="false" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="fundinglevel2_name" type="string" docValues="false" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="haslicense" type="boolean" docValues="false" multiValued="false" indexed="true" stored="false"/>
|
||||
<field name="impulse" type="pint" docValues="false" multiValued="false" indexed="true" stored="false"/>
|
||||
<field name="impulse_class" type="string" docValues="false" multiValued="false" indexed="true" stored="false"/>
|
||||
<field name="influence" type="pfloat" docValues="false" multiValued="false" indexed="true" stored="false"/>
|
||||
<field name="influence_class" type="string" docValues="false" multiValued="false" indexed="true" stored="false"/>
|
||||
<field name="instancetypename" type="string" docValues="false" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="isgreen" type="boolean" docValues="false" multiValued="false" indexed="true" stored="false"/>
|
||||
<field name="isindiamondjournal" type="boolean" docValues="false" multiValued="false" indexed="true" stored="false"/>
|
||||
<field name="oafentity" type="string" docValues="false" multiValued="true" indexed="false" stored="false"/>
|
||||
<field name="oaftype" type="string" docValues="false" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="objidentifier" type="string" docValues="false" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="openaccesscolor" type="string" docValues="false" multiValued="false" indexed="true" stored="false"/>
|
||||
<field name="orcidtypevalue" type="string_ci" docValues="false" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="organizationalternativenames" type="ngramtext" docValues="false" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="organizationdupid" type="string" docValues="false" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="organizationlegalname" type="ngramtext" docValues="false" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="organizationlegalshortname" type="ngramtext" docValues="false" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="originalid" type="string_ci" docValues="false" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="peerreviewed" type="boolean" docValues="false" multiValued="false" indexed="true" stored="false"/>
|
||||
<field name="pid" type="string_ci" docValues="false" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="pidclassid" type="string" docValues="false" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="popularity" type="pfloat" docValues="false" multiValued="false" indexed="true" stored="false"/>
|
||||
<field name="popularity_alt" type="pfloat" docValues="false" multiValued="false" indexed="true" stored="false"/>
|
||||
<field name="popularity_alt_class" type="string" docValues="false" multiValued="false" indexed="true" stored="false"/>
|
||||
<field name="popularity_class" type="string" docValues="false" multiValued="false" indexed="true" stored="false"/>
|
||||
<field name="projectacronym" type="ngramtext" docValues="false" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="projectcallidentifier" type="string" docValues="false" multiValued="false" indexed="true" stored="false"/>
|
||||
<field name="projectcode" type="ngramtext" docValues="false" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="projectcode_nt" type="string" docValues="false" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="projectduration" type="string" docValues="false" multiValued="false" indexed="true" stored="false"/>
|
||||
<field name="projectecsc39" type="string" docValues="false" multiValued="false" indexed="true" stored="false"/>
|
||||
<field name="projectenddate" type="date" docValues="false" multiValued="false" indexed="true" stored="false"/>
|
||||
<field name="projectendyear" type="string" docValues="false" multiValued="false" indexed="true" stored="false"/>
|
||||
<field name="projectkeywords" type="text_common" docValues="false" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="projectoamandatepublications" type="string" docValues="false" multiValued="false" indexed="true" stored="false"/>
|
||||
<field name="projectstartdate" type="date" docValues="false" multiValued="false" indexed="true" stored="false"/>
|
||||
<field name="projectstartyear" type="string" docValues="false" multiValued="false" indexed="true" stored="false"/>
|
||||
<field name="projecttitle" type="ngramtext" docValues="false" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="projecttitle_alternative" type="text_en" docValues="false" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="provenanceactionclassid" type="string" docValues="false" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="publiclyfunded" type="boolean" docValues="false" multiValued="false" indexed="true" stored="false"/>
|
||||
<field name="relclass" type="string" docValues="false" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="relcontracttypename" type="text_common" docValues="false" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="reldatasourcecompatibilityid" type="string" docValues="false" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="relfunder" type="string" docValues="false" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="relfunderid" type="string" docValues="false" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="relfundershortname" type="string" docValues="false" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="relfundinglevel0_id" type="string" docValues="false" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="relfundinglevel0_name" type="string" docValues="false" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="relfundinglevel1_id" type="string" docValues="false" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="relfundinglevel1_name" type="string" docValues="false" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="relfundinglevel2_id" type="string" docValues="false" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="relfundinglevel2_name" type="string" docValues="false" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="relorganizationcountryid" type="string" docValues="false" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="relorganizationcountryname" type="text_common" docValues="false" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="relorganizationid" type="string" docValues="false" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="relorganizationname" type="text_common" docValues="false" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="relorganizationshortname" type="text_common" docValues="false" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="relproject" type="string" docValues="false" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="relprojectcode" type="string" docValues="false" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="relprojectid" type="string" docValues="false" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="relprojectname" type="string" docValues="false" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="relprojecttitle" type="text_common" docValues="false" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="relresultid" type="string" docValues="false" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="relresulttype" type="string" docValues="false" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="resultacceptanceyear" type="string" docValues="false" multiValued="false" indexed="true" stored="false"/>
|
||||
<field name="resultauthor" type="text_common" docValues="false" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="resultbestaccessright" type="string" docValues="false" multiValued="false" indexed="true" stored="false"/>
|
||||
<field name="resultdateofacceptance" type="date" docValues="false" multiValued="false" indexed="true" stored="false"/>
|
||||
<field name="resultdescription" type="text_en" docValues="false" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="resultdupid" type="string" docValues="false" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="resultembargoenddate" type="date" docValues="false" multiValued="false" indexed="true" stored="false"/>
|
||||
<field name="resultembargoendyear" type="string" docValues="false" multiValued="false" indexed="true" stored="false"/>
|
||||
<field name="resulthostingdatasource" type="string" docValues="false" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="resulthostingdatasourceid" type="string" docValues="false" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="resultidentifier" type="string_ci" docValues="false" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="resultlanguagename" type="string" docValues="false" multiValued="false" indexed="true" stored="false"/>
|
||||
<field name="resultpublisher" type="text_common" docValues="false" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="resultsource" type="text_common" docValues="false" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="resultsubject" type="text_en" docValues="false" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="resulttitle" type="text_en" docValues="false" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="resulttypeid" type="string" docValues="false" multiValued="false" indexed="true" stored="false"/>
|
||||
<field name="sdg" type="string" docValues="false" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="semrelid" type="string" docValues="false" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="status" type="string_ci" docValues="false" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="text" type="text_common" indexed="false" stored="false"/>
|
||||
<field name="trust" type="string" multiValued="true" indexed="true" stored="false"/>
|
||||
|
||||
<field name="versioning" type="string" multiValued="true" indexed="true" stored="false"/>
|
||||
<field name="isgreen" type="boolean" multiValued="false" indexed="true" stored="false"/>
|
||||
<field name="openaccesscolor" type="string" multiValued="false" indexed="true" stored="false"/>
|
||||
<field name="isindiamondjournal" type="boolean" multiValued="false" indexed="true" stored="false"/>
|
||||
<field name="publiclyfunded" type="boolean" multiValued="false" indexed="true" stored="false"/>
|
||||
<field name="peerreviewed" type="boolean" multiValued="false" indexed="true" stored="false"/>
|
||||
|
||||
<copyField source="datasourceenglishname" dest="__all"/>
|
||||
<copyField source="datasourceoddescription" dest="__all"/>
|
||||
<copyField source="datasourceodsubjects" dest="__all"/>
|
||||
<copyField source="datasourceofficialname" dest="__all"/>
|
||||
<copyField source="datasourcesubject" dest="__all"/>
|
||||
<copyField source="externalreflabel" dest="__all"/>
|
||||
|
@ -382,26 +343,20 @@
|
|||
<copyField source="fundinglevel1_description" dest="__all"/>
|
||||
<copyField source="fundinglevel2_description" dest="__all"/>
|
||||
<copyField source="organizationalternativenames" dest="__all"/>
|
||||
<copyField source="organizationecenterprise" dest="__all"/>
|
||||
<copyField source="organizationecinternationalorganization" dest="__all"/>
|
||||
<copyField source="organizationecinternationalorganizationeurinterests" dest="__all"/>
|
||||
<copyField source="organizationeclegalbody" dest="__all"/>
|
||||
<copyField source="organizationeclegalperson" dest="__all"/>
|
||||
<copyField source="organizationecnonprofit" dest="__all"/>
|
||||
<copyField source="organizationecnutscode" dest="__all"/>
|
||||
<copyField source="organizationecresearchorganization" dest="__all"/>
|
||||
<copyField source="organizationecsmevalidated" dest="__all"/>
|
||||
<copyField source="organizationlegalname" dest="__all"/>
|
||||
<copyField source="organizationlegalshortname" dest="__all"/>
|
||||
<copyField source="projectacronym" dest="__all"/>
|
||||
<copyField source="projectcode" dest="__all"/>
|
||||
<copyField source="projectkeywords" dest="__all"/>
|
||||
<copyField source="projecttitle" dest="__all"/>
|
||||
<copyField source="projecttitle_alternative" dest="__all"/>
|
||||
<copyField source="relcontracttypename" dest="__all"/>
|
||||
<copyField source="relorganizationcountryname" dest="__all"/>
|
||||
<copyField source="relorganizationname" dest="__all"/>
|
||||
<copyField source="relorganizationshortname" dest="__all"/>
|
||||
<copyField source="relprojectname" dest="__all"/>
|
||||
<copyField source="relprojecttitle" dest="__all"/>
|
||||
<copyField source="resultacceptanceyear" dest="__all"/>
|
||||
<copyField source="resultauthor" dest="__all"/>
|
||||
<copyField source="resultdescription" dest="__all"/>
|
||||
<copyField source="resultidentifier" dest="__all"/>
|
||||
|
|
Binary file not shown.
Binary file not shown.
2
pom.xml
2
pom.xml
|
@ -888,7 +888,7 @@
|
|||
<mockito-core.version>3.3.3</mockito-core.version>
|
||||
<mongodb.driver.version>3.4.2</mongodb.driver.version>
|
||||
<vtd.version>[2.12,3.0)</vtd.version>
|
||||
<dhp-schemas.version>[4.17.2]</dhp-schemas.version>
|
||||
<dhp-schemas.version>[6.0.0-SNAPSHOT]</dhp-schemas.version>
|
||||
<dnet-actionmanager-api.version>[4.0.3]</dnet-actionmanager-api.version>
|
||||
<dnet-actionmanager-common.version>[6.0.5]</dnet-actionmanager-common.version>
|
||||
<dnet-openaire-broker-common.version>[3.1.6]</dnet-openaire-broker-common.version>
|
||||
|
|
Loading…
Reference in New Issue