fixed conflicts

This commit is contained in:
Miriam Baglioni 2024-01-10 11:03:42 +01:00
commit e711a05229
19 changed files with 382 additions and 175 deletions

View File

@ -23,6 +23,7 @@ import com.google.common.collect.Lists;
import com.google.common.collect.Sets;
import eu.dnetlib.dhp.common.vocabulary.VocabularyGroup;
import eu.dnetlib.dhp.common.vocabulary.VocabularyTerm;
import eu.dnetlib.dhp.schema.common.ModelConstants;
import eu.dnetlib.dhp.schema.common.ModelSupport;
import eu.dnetlib.dhp.schema.oaf.*;
@ -30,6 +31,10 @@ import me.xuender.unidecode.Unidecode;
public class GraphCleaningFunctions extends CleaningFunctions {
public static final String DNET_PUBLISHERS = "dnet:publishers";
public static final String DNET_LICENSES = "dnet:licenses";
public static final String ORCID_CLEANING_REGEX = ".*([0-9]{4}).*[-–—−=].*([0-9]{4}).*[-–—−=].*([0-9]{4}).*[-–—−=].*([0-9x]{4})";
public static final int ORCID_LEN = 19;
public static final String CLEANING_REGEX = "(?:\\n|\\r|\\t)";
@ -409,6 +414,14 @@ public class GraphCleaningFunctions extends CleaningFunctions {
.getPublisher()
.getValue()
.replaceAll(NAME_CLEANING_REGEX, " "));
if (vocs.vocabularyExists(DNET_PUBLISHERS)) {
vocs
.find(DNET_PUBLISHERS)
.map(voc -> voc.getTermBySynonym(r.getPublisher().getValue()))
.map(VocabularyTerm::getName)
.ifPresent(publisher -> r.getPublisher().setValue(publisher));
}
}
}
if (Objects.isNull(r.getLanguage()) || StringUtils.isBlank(r.getLanguage().getClassid())) {
@ -569,6 +582,14 @@ public class GraphCleaningFunctions extends CleaningFunctions {
i.setRefereed(qualifier("0000", "Unknown", ModelConstants.DNET_REVIEW_LEVELS));
}
if (Objects.nonNull(i.getLicense()) && Objects.nonNull(i.getLicense().getValue())) {
vocs
.find(DNET_LICENSES)
.map(voc -> voc.getTermBySynonym(i.getLicense().getValue()))
.map(VocabularyTerm::getId)
.ifPresent(license -> i.getLicense().setValue(license));
}
// from the script from Dimitris
if ("0000".equals(i.getRefereed().getClassid())) {
final boolean isFromCrossref = Optional
@ -668,6 +689,9 @@ public class GraphCleaningFunctions extends CleaningFunctions {
.filter(Objects::nonNull)
.filter(p -> Objects.nonNull(p.getQualifier()))
.filter(p -> StringUtils.isNotBlank(p.getValue()))
.filter(
p -> StringUtils
.contains(StringUtils.lowerCase(p.getQualifier().getClassid()), ORCID))
.map(p -> {
// hack to distinguish orcid from orcid_pending
String pidProvenance = getProvenance(p.getDataInfo());
@ -677,7 +701,8 @@ public class GraphCleaningFunctions extends CleaningFunctions {
.toLowerCase()
.contains(ModelConstants.ORCID)) {
if (pidProvenance
.equals(ModelConstants.SYSIMPORT_CROSSWALK_ENTITYREGISTRY)) {
.equals(ModelConstants.SYSIMPORT_CROSSWALK_ENTITYREGISTRY) ||
pidProvenance.equals("ORCID_ENRICHMENT")) {
p.getQualifier().setClassid(ModelConstants.ORCID);
} else {
p.getQualifier().setClassid(ModelConstants.ORCID_PENDING);

View File

@ -2,7 +2,7 @@
{"paramName":"mt", "paramLongName":"master", "paramDescription": "should be local or yarn", "paramRequired": true},
{"paramName":"i", "paramLongName":"isLookupUrl", "paramDescription": "isLookupUrl", "paramRequired": true},
{"paramName":"w", "paramLongName":"workingPath", "paramDescription": "the path of the sequencial file to read", "paramRequired": true},
{"paramName":"t", "paramLongName":"targetPath", "paramDescription": "the oaf path ", "paramRequired": true},
{"paramName":"mo", "paramLongName":"mdstoreOutputVersion", "paramDescription": "the oaf path ", "paramRequired": true},
{"paramName":"s", "paramLongName":"skipUpdate", "paramDescription": "skip update ", "paramRequired": false},
{"paramName":"h", "paramLongName":"hdfsServerUri", "paramDescription": "the working path ", "paramRequired": true}
]

View File

@ -1,4 +1,4 @@
<workflow-app name="Download_Transform_Pubmed_Workflow" xmlns="uri:oozie:workflow:0.5">
<workflow-app name="Download_Transform_Pubmed_Workflow" xmlns="uri:oozie:workflow:0.5">
<parameters>
<property>
<name>baselineWorkingPath</name>
@ -9,8 +9,12 @@
<description>The IS lookUp service endopoint</description>
</property>
<property>
<name>targetPath</name>
<description>The target path</description>
<name>mdStoreOutputId</name>
<description>the identifier of the cleaned MDStore</description>
</property>
<property>
<name>mdStoreManagerURI</name>
<description>the path of the cleaned mdstore</description>
</property>
<property>
<name>skipUpdate</name>
@ -19,12 +23,31 @@
</property>
</parameters>
<start to="ConvertDataset"/>
<start to="StartTransaction"/>
<kill name="Kill">
<message>Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}]</message>
</kill>
<action name="StartTransaction">
<java>
<configuration>
<property>
<name>oozie.launcher.mapreduce.user.classpath.first</name>
<value>true</value>
</property>
</configuration>
<main-class>eu.dnetlib.dhp.aggregation.mdstore.MDStoreActionNode</main-class>
<arg>--action</arg><arg>NEW_VERSION</arg>
<arg>--mdStoreID</arg><arg>${mdStoreOutputId}</arg>
<arg>--mdStoreManagerURI</arg><arg>${mdStoreManagerURI}</arg>
<capture-output/>
</java>
<ok to="ConvertDataset"/>
<error to="RollBack"/>
</action>
<action name="ConvertDataset">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn</master>
@ -43,16 +66,52 @@
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
</spark-opts>
<arg>--workingPath</arg><arg>${baselineWorkingPath}</arg>
<arg>--targetPath</arg><arg>${targetPath}</arg>
<arg>--mdstoreOutputVersion</arg><arg>${wf:actionData('StartTransaction')['mdStoreVersion']}</arg>
<arg>--master</arg><arg>yarn</arg>
<arg>--isLookupUrl</arg><arg>${isLookupUrl}</arg>
<arg>--hdfsServerUri</arg><arg>${nameNode}</arg>
<arg>--skipUpdate</arg><arg>${skipUpdate}</arg>
</spark>
<ok to="CommitVersion"/>
<error to="RollBack"/>
</action>
<action name="CommitVersion">
<java>
<configuration>
<property>
<name>oozie.launcher.mapreduce.user.classpath.first</name>
<value>true</value>
</property>
</configuration>
<main-class>eu.dnetlib.dhp.aggregation.mdstore.MDStoreActionNode</main-class>
<arg>--action</arg><arg>COMMIT</arg>
<arg>--namenode</arg><arg>${nameNode}</arg>
<arg>--mdStoreVersion</arg><arg>${wf:actionData('StartTransaction')['mdStoreVersion']}</arg>
<arg>--mdStoreManagerURI</arg><arg>${mdStoreManagerURI}</arg>
</java>
<ok to="End"/>
<error to="Kill"/>
</action>
<action name="RollBack">
<java>
<configuration>
<property>
<name>oozie.launcher.mapreduce.user.classpath.first</name>
<value>true</value>
</property>
</configuration>
<main-class>eu.dnetlib.dhp.aggregation.mdstore.MDStoreActionNode</main-class>
<arg>--action</arg><arg>ROLLBACK</arg>
<arg>--mdStoreVersion</arg><arg>${wf:actionData('StartTransaction')['mdStoreVersion']}</arg>
<arg>--mdStoreManagerURI</arg><arg>${mdStoreManagerURI}</arg>
</java>
<ok to="Kill"/>
<error to="Kill"/>
</action>
<end name="End"/>
</workflow-app>

View File

@ -2,9 +2,12 @@ package eu.dnetlib.dhp.sx.bio.ebi
import eu.dnetlib.dhp.application.ArgumentApplicationParser
import eu.dnetlib.dhp.collection.CollectionUtils
import eu.dnetlib.dhp.common.Constants.{MDSTORE_DATA_PATH, MDSTORE_SIZE_PATH}
import eu.dnetlib.dhp.common.vocabulary.VocabularyGroup
import eu.dnetlib.dhp.schema.mdstore.MDStoreVersion
import eu.dnetlib.dhp.schema.oaf.{Oaf, Result}
import eu.dnetlib.dhp.sx.bio.pubmed._
import eu.dnetlib.dhp.utils.DHPUtils.{MAPPER, writeHdfsFile}
import eu.dnetlib.dhp.utils.ISLookupClientFactory
import org.apache.commons.io.IOUtils
import org.apache.hadoop.conf.Configuration
@ -164,11 +167,15 @@ object SparkCreateBaselineDataFrame {
val workingPath = parser.get("workingPath")
log.info("workingPath: {}", workingPath)
val targetPath = parser.get("targetPath")
log.info("targetPath: {}", targetPath)
val mdstoreOutputVersion = parser.get("mdstoreOutputVersion")
log.info("mdstoreOutputVersion: {}", mdstoreOutputVersion)
val cleanedMdStoreVersion = MAPPER.readValue(mdstoreOutputVersion, classOf[MDStoreVersion])
val outputBasePath = cleanedMdStoreVersion.getHdfsPath
log.info("outputBasePath: {}", outputBasePath)
val hdfsServerUri = parser.get("hdfsServerUri")
log.info("hdfsServerUri: {}", targetPath)
log.info("hdfsServerUri: {}", hdfsServerUri)
val skipUpdate = parser.get("skipUpdate")
log.info("skipUpdate: {}", skipUpdate)
@ -216,8 +223,11 @@ object SparkCreateBaselineDataFrame {
.map(a => PubMedToOaf.convert(a, vocabularies))
.as[Oaf]
.filter(p => p != null),
targetPath
s"$outputBasePath/$MDSTORE_DATA_PATH"
)
val df = spark.read.text(s"$outputBasePath/$MDSTORE_DATA_PATH")
val mdStoreSize = df.count
writeHdfsFile(spark.sparkContext.hadoopConfiguration, s"$mdStoreSize", s"$outputBasePath/$MDSTORE_SIZE_PATH")
}
}

View File

@ -16,6 +16,10 @@
<name>filterInvisible</name>
<description>whether filter out invisible entities after merge</description>
</property>
<property>
<name>isLookupUrl</name>
<description>the URL address of the lookUp service</description>
</property>
<property>
<name>sparkDriverMemory</name>
<description>heap memory for driver process</description>
@ -128,6 +132,7 @@
<arg>--graphInputPath</arg><arg>${graphBasePath}</arg>
<arg>--checkpointPath</arg><arg>${workingPath}/grouped_entities</arg>
<arg>--outputPath</arg><arg>${graphOutputPath}</arg>
<arg>--isLookupUrl</arg><arg>${isLookupUrl}</arg>
<arg>--filterInvisible</arg><arg>${filterInvisible}</arg>
</spark>
<ok to="End"/>

View File

@ -34,18 +34,6 @@
<description>the MAG working path</description>
</property>
<!-- ORCID Parameters -->
<property>
<name>inputPathOrcid</name>
<description>the ORCID input path</description>
</property>
<property>
<name>workingPathOrcid</name>
<description>the ORCID working path</description>
</property>
</parameters>
<global>
@ -65,7 +53,6 @@
<switch>
<case to="ResetMagWorkingPath">${wf:conf('resumeFrom') eq 'ResetMagWorkingPath'}</case>
<case to="ConvertMagToDataset">${wf:conf('resumeFrom') eq 'ConvertMagToDataset'}</case>
<case to="PreProcessORCID">${wf:conf('resumeFrom') eq 'PreProcessORCID'}</case>
<default to="GenerateCrossrefDataset"/> <!-- first action to be done when downloadDump is to be performed -->
</switch>
</decision>

View File

@ -59,6 +59,7 @@ public class OafCleaner implements Serializable {
private static boolean isPrimitive(Object o) {
return Objects.isNull(o)
|| o.getClass().isPrimitive()
|| o.getClass().isEnum()
|| o instanceof Class
|| o instanceof Integer
|| o instanceof Double

View File

@ -38,16 +38,12 @@
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
</spark-opts>
<arg>--orcidPath</arg>
<arg>${orcidPath}</arg>
<arg>--targetPath</arg>
<arg>${targetPath}</arg>
<arg>--graphPath</arg>
<arg>${graphPath}</arg>
<arg>--master</arg>
<arg>yarn</arg>
<arg>--orcidPath</arg><arg>${orcidPath}</arg>
<arg>--targetPath</arg><arg>${targetPath}</arg>
<arg>--graphPath</arg><arg>${graphPath}</arg>
<arg>--master</arg><arg>yarn</arg>
</spark>
<ok to="End"/>
<ok to="copy_datasource"/>
<error to="Kill"/>
</action>

View File

@ -1,10 +1,10 @@
DROP VIEW IF EXISTS ${hiveDbName}.result;
CREATE VIEW IF NOT EXISTS ${hiveDbName}.result as
select id, originalid, dateofcollection, title, publisher, bestaccessright, datainfo, collectedfrom, pid, author, resulttype, language, country, subject, description, dateofacceptance, relevantdate, embargoenddate, resourcetype, context, externalreference, instance, measures, processingchargeamount, eoscifguidelines from ${hiveDbName}.publication p
select id, originalid, dateofcollection, title, publisher, bestaccessright, datainfo, collectedfrom, pid, author, resulttype, metaresourcetype, language, country, subject, description, dateofacceptance, relevantdate, embargoenddate, resourcetype, context, externalreference, instance, measures, processingchargeamount, eoscifguidelines from ${hiveDbName}.publication p
union all
select id, originalid, dateofcollection, title, publisher, bestaccessright, datainfo, collectedfrom, pid, author, resulttype, language, country, subject, description, dateofacceptance, relevantdate, embargoenddate, resourcetype, context, externalreference, instance, measures, processingchargeamount, eoscifguidelines from ${hiveDbName}.dataset d
select id, originalid, dateofcollection, title, publisher, bestaccessright, datainfo, collectedfrom, pid, author, resulttype, metaresourcetype, language, country, subject, description, dateofacceptance, relevantdate, embargoenddate, resourcetype, context, externalreference, instance, measures, processingchargeamount, eoscifguidelines from ${hiveDbName}.dataset d
union all
select id, originalid, dateofcollection, title, publisher, bestaccessright, datainfo, collectedfrom, pid, author, resulttype, language, country, subject, description, dateofacceptance, relevantdate, embargoenddate, resourcetype, context, externalreference, instance, measures, processingchargeamount, eoscifguidelines from ${hiveDbName}.software s
select id, originalid, dateofcollection, title, publisher, bestaccessright, datainfo, collectedfrom, pid, author, resulttype, metaresourcetype, language, country, subject, description, dateofacceptance, relevantdate, embargoenddate, resourcetype, context, externalreference, instance, measures, processingchargeamount, eoscifguidelines from ${hiveDbName}.software s
union all
select id, originalid, dateofcollection, title, publisher, bestaccessright, datainfo, collectedfrom, pid, author, resulttype, language, country, subject, description, dateofacceptance, relevantdate, embargoenddate, resourcetype, context, externalreference, instance, measures, processingchargeamount, eoscifguidelines from ${hiveDbName}.otherresearchproduct o;
select id, originalid, dateofcollection, title, publisher, bestaccessright, datainfo, collectedfrom, pid, author, resulttype, metaresourcetype, language, country, subject, description, dateofacceptance, relevantdate, embargoenddate, resourcetype, context, externalreference, instance, measures, processingchargeamount, eoscifguidelines from ${hiveDbName}.otherresearchproduct o;

View File

@ -690,6 +690,38 @@ public class CleanGraphSparkJobTest {
}
@Test
void testClean_ORP() throws Exception {
final String prefix = "gcube ";
new CleanGraphSparkJob(
args(
"/eu/dnetlib/dhp/oa/graph/input_clean_graph_parameters.json",
new String[] {
"--inputPath", graphInputPath + "/orp",
"--outputPath", graphOutputPath + "/orp",
"--isLookupUrl", "lookupurl",
"--graphTableClassName", OtherResearchProduct.class.getCanonicalName(),
"--deepClean", "true",
"--contextId", "sobigdata",
"--verifyParam", "gCube ",
"--masterDuplicatePath", dsMasterDuplicatePath,
"--country", "NL",
"--verifyCountryParam", "10.17632",
"--collectedfrom", "NARCIS",
"--hostedBy", Objects
.requireNonNull(
getClass()
.getResource("/eu/dnetlib/dhp/oa/graph/clean/hostedBy"))
.getPath()
})).run(false, isLookUpService);
Dataset<OtherResearchProduct> orp = read(spark, graphOutputPath + "/orp", OtherResearchProduct.class);
assertEquals(1, orp.count());
}
@Test
void testCleanCfHbSparkJob() throws Exception {

File diff suppressed because one or more lines are too long

View File

@ -434,6 +434,24 @@ public class XmlRecordFactory implements Serializable {
XmlSerializationUtils
.asXmlElement("processingchargecurrency", r.getProcessingchargecurrency().getValue()));
}
if (r.getIsGreen() != null) {
metadata
.add(XmlSerializationUtils.asXmlElement("isgreen", Boolean.toString(r.getIsGreen())));
}
if (r.getOpenAccessColor() != null) {
metadata
.add(XmlSerializationUtils.asXmlElement("openaccesscolor", r.getOpenAccessColor().toString()));
}
if (r.getIsInDiamondJournal() != null) {
metadata
.add(
XmlSerializationUtils
.asXmlElement("isindiamondjournal", Boolean.toString(r.getIsInDiamondJournal())));
}
if (r.getPubliclyFunded() != null) {
metadata
.add(XmlSerializationUtils.asXmlElement("publiclyfunded", Boolean.toString(r.getPubliclyFunded())));
}
}
switch (type) {

View File

@ -78,6 +78,22 @@ public class IndexRecordTransformerTest {
testRecordTransformation(record);
}
@Test
void testPeerReviewed() throws IOException, TransformerException {
final XmlRecordFactory xmlRecordFactory = new XmlRecordFactory(contextMapper, false,
XmlConverterJob.schemaLocation);
final Publication p = load("publication.json", Publication.class);
final JoinedEntity<Publication> je = new JoinedEntity<>(p);
final String record = xmlRecordFactory.build(je);
assertNotNull(record);
SolrInputDocument solrDoc = testRecordTransformation(record);
assertEquals("true", solrDoc.get("peerreviewed").getValue());
}
@Test
public void testRiunet() throws IOException, TransformerException {
@ -184,7 +200,7 @@ public class IndexRecordTransformerTest {
}
}
private void testRecordTransformation(final String record) throws IOException, TransformerException {
private SolrInputDocument testRecordTransformation(final String record) throws IOException, TransformerException {
final String fields = IOUtils.toString(getClass().getResourceAsStream("fields.xml"));
final String xslt = IOUtils.toString(getClass().getResourceAsStream("layoutToRecordTransformer.xsl"));
@ -200,6 +216,8 @@ public class IndexRecordTransformerTest {
Assertions.assertNotNull(xmlDoc);
System.out.println(xmlDoc);
return solrDoc;
}
private <T> T load(final String fileName, final Class<T> clazz) throws IOException {

View File

@ -1,9 +1,12 @@
package eu.dnetlib.dhp.oa.provision;
import static org.junit.jupiter.api.Assertions.assertEquals;
import java.io.IOException;
import java.io.StringReader;
import java.net.URI;
import java.util.Map;
import org.apache.commons.io.IOUtils;
import org.apache.hadoop.io.Text;
@ -11,6 +14,7 @@ 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;
@ -101,14 +105,38 @@ public class XmlIndexingJobTest extends SolrTest {
new XmlIndexingJob(spark, inputPath, FORMAT, batchSize, XmlIndexingJob.OutputFormat.SOLR, null)
.run(isLookupClient);
Assertions.assertEquals(0, miniCluster.getSolrClient().commit().getStatus());
assertEquals(0, miniCluster.getSolrClient().commit().getStatus());
QueryResponse rsp = miniCluster.getSolrClient().query(new SolrQuery().add(CommonParams.Q, "*:*"));
Assertions
.assertEquals(
nRecord, rsp.getResults().getNumFound(),
"the number of indexed records should be equal to the number of input records");
assertEquals(
nRecord, rsp.getResults().getNumFound(),
"the number of indexed records should be equal to the number of input records");
rsp = miniCluster.getSolrClient().query(new SolrQuery().add(CommonParams.Q, "isgreen:true"));
assertEquals(
0, rsp.getResults().getNumFound(),
"the number of indexed records having isgreen = true");
rsp = miniCluster.getSolrClient().query(new SolrQuery().add(CommonParams.Q, "openaccesscolor:bronze"));
assertEquals(
0, rsp.getResults().getNumFound(),
"the number of indexed records having openaccesscolor = bronze");
rsp = miniCluster.getSolrClient().query(new SolrQuery().add(CommonParams.Q, "isindiamondjournal:true"));
assertEquals(
0, rsp.getResults().getNumFound(),
"the number of indexed records having isindiamondjournal = true");
rsp = miniCluster.getSolrClient().query(new SolrQuery().add(CommonParams.Q, "publiclyfunded:true"));
assertEquals(
0, rsp.getResults().getNumFound(),
"the number of indexed records having publiclyfunded = true");
rsp = miniCluster.getSolrClient().query(new SolrQuery().add(CommonParams.Q, "peerreviewed:true"));
assertEquals(
0, rsp.getResults().getNumFound(),
"the number of indexed records having peerreviewed = true");
}
@Test
@ -126,7 +154,7 @@ public class XmlIndexingJobTest extends SolrTest {
.map(s -> new SAXReader().read(new StringReader(s)).valueOf(ID_XPATH))
.distinct()
.count();
Assertions.assertEquals(nRecord, xmlIdUnique, "IDs should be unique among input records");
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)
@ -142,7 +170,7 @@ public class XmlIndexingJobTest extends SolrTest {
}, Encoders.STRING())
.distinct()
.count();
Assertions.assertEquals(xmlIdUnique, docIdUnique, "IDs should be unique among the output records");
assertEquals(xmlIdUnique, docIdUnique, "IDs should be unique among the output records");
}

View File

@ -32,7 +32,7 @@ public class XmlRecordFactoryTest {
.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
@Test
public void testXMLRecordFactory() throws IOException, DocumentException {
void testXMLRecordFactory() throws IOException, DocumentException {
final ContextMapper contextMapper = new ContextMapper();
@ -93,6 +93,10 @@ public class XmlRecordFactoryTest {
"https://osf.io/preprints/socarxiv/7vgtu/download",
doc.valueOf("//*[local-name() = 'result']/fulltext[1]"));
assertEquals("true", doc.valueOf("//*[local-name() = 'result']/isgreen/text()"));
assertEquals("bronze", doc.valueOf("//*[local-name() = 'result']/openaccesscolor/text()"));
assertEquals("true", doc.valueOf("//*[local-name() = 'result']/isindiamondjournal/text()"));
assertEquals("true", doc.valueOf("//*[local-name() = 'result']/publiclyfunded/text()"));
}
@Test

View File

@ -1,116 +1,137 @@
<LAYOUT name="index">
<FIELDS>
<FIELD name="oaftype" indexable="true" value="local-name(//*[local-name()='entity']/*[local-name() != 'extraInfo'])" tokenizable="false" result="false" stat="false"/>
<FIELD name="objidentifier" stat="false" tokenizable="false" xpath="//header/dri:objIdentifier" result="false" indexable="true"/><!-- DATASOURCE FIELDS -->
<FIELD name="datasourceofficialname" stat="false" result="false" copy="true" indexable="true" xpath="//*[local-name()='entity']/*[local-name()='datasource']/officialname"/>
<FIELD name="datasourceenglishname" indexable="true" copy="true" result="false" stat="false" xpath="//*[local-name()='entity']/*[local-name()='datasource']/englishname"/>
<FIELD name="datasourceoddescription" stat="false" result="false" indexable="false" copy="true" xpath="//*[local-name()='entity']/*[local-name()='datasource']/oddescription"/>
<FIELD name="datasourceodsubjects" result="false" stat="false" copy="true" xpath="//*[local-name()='entity']/*[local-name()='datasource']/odsubjects" indexable="true" tokenizable="false"/>
<FIELD name="datasourceodlanguages" indexable="true" result="false" stat="false" xpath="//*[local-name()='entity']/*[local-name()='datasource']/odlanguages" tokenizable="false"/>
<FIELD name="datasourceodcontenttypes" result="false" stat="false" indexable="true" tokenizable="false" xpath="//*[local-name()='entity']/*[local-name()='datasource']/odcontenttypes"/>
<FIELD name="datasourcetypename" tokenizable="false" indexable="true" result="false" multivalued="false" stat="false" xpath="//*[local-name()='entity']/*[local-name()='datasource']/datasourcetype/@classname"/>
<FIELD name="datasourcetypeuiid" xpath="//*[local-name()='entity']/*[local-name()='datasource']/datasourcetypeui/@classid" result="false" indexable="true" multivalued="false" tokenizable="false" stat="false"/>
<FIELD name="datasourcetypeuiname" result="false" indexable="true" stat="false" xpath="//*[local-name()='entity']/*[local-name()='datasource']/datasourcetypeui/@classname" tokenizable="false" multivalued="false"/>
<FIELD name="datasourcecompatibilityid" result="false" xpath="//*[local-name()='entity']/*[local-name()='datasource']/openairecompatibility/@classid" multivalued="false" stat="false" indexable="true" tokenizable="false"/>
<FIELD name="datasourcecompatibilityname" indexable="true" multivalued="false" xpath="//*[local-name()='entity']/*[local-name()='datasource']/openairecompatibility/@classname" stat="false" tokenizable="false" result="false"/>
<FIELD name="datasourcesubject" type="ngramtext" multivalued="true" copy="true" result="false" xpath="//*[local-name()='entity']/*[local-name()='datasource']/subjects" indexable="true" stat="false"/><!-- datasource fields for EOSC -->
<FIELD name="datasourcejurisdiction" xpath="//*[local-name()='entity']/*[local-name()='datasource']/jurisdiction/@classname" indexable="true" tokenizable="false" result="false" stat="false"/>
<FIELD name="datasourcethematic" stat="false" xpath="//*[local-name()='entity']/*[local-name()='datasource']/thematic" result="false" tokenizable="false" indexable="true"/>
<FIELD name="eosctype" indexable="true" stat="false" tokenizable="false" result="false" xpath="//*[local-name()='entity']/*[local-name()='datasource']/eosctype/@classname"/>
<FIELD name="eoscdatasourcetype" result="false" indexable="true" xpath="//*[local-name()='entity']/*[local-name()='datasource']/eoscdatasourcetype/@classname" tokenizable="false" stat="false"/><!-- ORGANIZATION FIELDS -->
<FIELD name="organizationlegalshortname" copy="true" indexable="true" xpath="distinct-values(//*[local-name()='entity']/*[local-name()='organization']//legalshortname)" type="ngramtext" result="false" stat="false"/>
<FIELD name="organizationlegalname" indexable="true" stat="false" xpath="distinct-values(//*[local-name()='entity']/*[local-name()='organization']//legalname)" type="ngramtext" copy="true" result="false"/>
<FIELD name="organizationalternativenames" stat="false" copy="true" type="ngramtext" result="false" indexable="false" xpath="distinct-values(//*[local-name()='entity']/*[local-name()='organization']//alternativeNames)"/><!-- PROJECT FIELDS -->
<FIELD name="projectcode" result="false" xpath="//*[local-name()='entity']/*[local-name()='project']/code" copy="true" indexable="false" stat="false" type="ngramtext"/>
<FIELD name="projectcode_nt" indexable="true" tokenizable="false" result="false" xpath="//*[local-name()='entity']/*[local-name()='project']/code" stat="false"/>
<FIELD name="projectacronym" indexable="true" xpath="//*[local-name()='entity']/*[local-name()='project']/acronym" copy="true" result="false" stat="false" type="ngramtext"/>
<FIELD name="projecttitle" copy="true" stat="false" result="false" indexable="true" type="ngramtext" xpath="//*[local-name()='entity']/*[local-name()='project']/title"/>
<FIELD name="projectstartdate" indexable="true" type="date" result="false" multivalued="false" value="//*[local-name()='entity']/*[local-name()='project']/startdate" stat="false"/>
<FIELD name="projectstartyear" indexable="true" value="dnet:extractYear(//*[local-name()='entity']/*[local-name()='project']/startdate)" tokenizable="false" multivalued="false" result="false" stat="false"/>
<FIELD name="projectenddate" stat="false" multivalued="false" result="false" type="date" value="//*[local-name()='entity']/*[local-name()='project']/enddate" indexable="true"/>
<FIELD name="projectendyear" result="false" tokenizable="false" multivalued="false" indexable="true" stat="false" value="dnet:extractYear(//*[local-name()='entity']/*[local-name()='project']/enddate)"/>
<FIELD name="projectcallidentifier" indexable="true" xpath="//*[local-name()='entity']/*[local-name()='project']/callidentifier" stat="false" tokenizable="false" multivalued="false" result="false"/>
<FIELD name="projectkeywords" result="false" xpath="//*[local-name()='entity']/*[local-name()='project']/keywords" copy="true" stat="false" indexable="false"/>
<FIELD name="projectduration" multivalued="false" stat="false" xpath="//*[local-name()='entity']/*[local-name()='project']/duration" indexable="true" tokenizable="false" result="false"/>
<FIELD name="projectecsc39" multivalued="false" result="false" stat="false" tokenizable="false" xpath="distinct-values(//*[local-name()='entity']/*[local-name()='project']/ecsc39)" indexable="true"/>
<FIELD name="projectoamandatepublications" result="false" stat="false" tokenizable="false" xpath="//*[local-name()='entity']/*[local-name()='project']/oamandatepublications" indexable="true" multivalued="false"/>
<FIELD name="fundinglevel0_id" indexable="true" tokenizable="false" result="false" xpath="//*[local-name()='entity']/*[local-name()='project']/fundingtree//funding_level_0/id" stat="false"/>
<FIELD name="fundinglevel0_name" xpath="//*[local-name()='entity']/*[local-name()='project']/fundingtree//funding_level_0/name" indexable="true" tokenizable="false" result="false" stat="false"/>
<FIELD name="fundinglevel0_description" indexable="false" xpath="//*[local-name()='entity']/*[local-name()='project']/fundingtree//funding_level_0/description" result="false" stat="false" copy="true"/>
<FIELD name="fundinglevel1_id" stat="false" tokenizable="false" result="false" xpath="//*[local-name()='entity']/*[local-name()='project']/fundingtree//funding_level_1/id" indexable="true"/>
<FIELD name="fundinglevel1_name" stat="false" result="false" indexable="true" xpath="//*[local-name()='entity']/*[local-name()='project']/fundingtree//funding_level_1/name" tokenizable="false"/>
<FIELD name="fundinglevel1_description" result="false" xpath="//*[local-name()='entity']/*[local-name()='project']/fundingtree//funding_level_1/description" stat="false" copy="true" indexable="false"/>
<FIELD name="fundinglevel2_id" indexable="true" result="false" stat="false" xpath="//*[local-name()='entity']/*[local-name()='project']/fundingtree//funding_level_2/id" tokenizable="false"/>
<FIELD name="fundinglevel2_name" tokenizable="false" xpath="//*[local-name()='entity']/*[local-name()='project']/fundingtree//funding_level_2/name" result="false" indexable="true" stat="false"/>
<FIELD name="fundinglevel2_description" indexable="false" copy="true" result="false" xpath="//*[local-name()='entity']/*[local-name()='project']/fundingtree//funding_level_2/description" stat="false"/><!-- PROJECTS' FUNDER FIELDS: indexable only with the new funding path/context handling -->
<FIELD name="funder" result="false" value="concat(./id/text(), '||', ./name/text(), '||', ./shortname/text())" xpath="//*[local-name()='entity']/*[local-name()='project']/fundingtree/funder" indexable="true" stat="false" tokenizable="false"/>
<FIELD name="fundershortname" stat="false" tokenizable="false" indexable="true" xpath="//*[local-name()='entity']/*[local-name()='project']/fundingtree/funder/shortname" result="false"/>
<FIELD name="funderid" stat="false" indexable="true" xpath="//*[local-name()='entity']/*[local-name()='project']/fundingtree/funder/id" result="false" tokenizable="false"/><!-- RESULT FIELDS -->
<FIELD name="resulttitle" indexable="true" xpath="//*[local-name() = 'entity']/*[local-name() ='result']/title | //*[local-name()='entity']/*[local-name()='result']/children/result/title" stat="false" type="text_en" copy="true" result="false"/>
<FIELD name="resultsubject" stat="false" type="text_en" indexable="true" result="false" xpath="distinct-values(//*[local-name()='entity']/*[local-name()='result']/subject)"/>
<FIELD name="resultembargoenddate" type="date" multivalued="false" result="false" value="//*[local-name()='entity']/*[local-name()='result']/embargoenddate" indexable="true" stat="false"/>
<FIELD name="resultembargoendyear" multivalued="false" tokenizable="false" stat="false" value="dnet:extractYear(//*[local-name()='entity']/*[local-name()='result']/embargoenddate)" indexable="true" result="false"/>
<FIELD name="resulttypeid" tokenizable="false" xpath="//*[local-name()='entity']/*[local-name()='result']/resulttype/@classid" stat="false" multivalued="false" indexable="true" result="false"/>
<FIELD name="resultlanguagename" result="false" indexable="true" tokenizable="false" stat="false" xpath="//*[local-name()='entity']/*[local-name()='result']/language/@classname" multivalued="false"/>
<FIELD name="resultpublisher" stat="false" copy="true" xpath="//*[local-name()='entity']/*[local-name()='result']/*[local-name()='publisher']" result="false" indexable="true"/>
<FIELD name="resultdescription" result="false" xpath="//*[local-name()='entity']/*[local-name()='result']//*[local-name()='description']" copy="true" type="text_en" indexable="true" stat="false"/>
<FIELD name="resultbestaccessright" xpath="distinct-values(//*[local-name()='entity']/*[local-name()='result']/bestaccessright/@classname)" stat="false" indexable="true" tokenizable="false" result="false" multivalued="false"/>
<FIELD name="resultdateofacceptance" type="date" multivalued="false" stat="false" result="false" indexable="true" value="//*[local-name()='entity']/*[local-name()='result']/dateofacceptance"/>
<FIELD name="resultacceptanceyear" result="false" copy="true" indexable="true" multivalued="false" value="dnet:extractYear(//*[local-name()='entity']/*[local-name()='result']/dateofacceptance)" stat="false" tokenizable="false"/>
<FIELD name="resultauthor" copy="true" indexable="true" multivalued="true" stat="false" xpath="//*[local-name()='entity']/*[local-name()='result']/creator" result="false"/>
<FIELD name="authorid" indexable="true" result="false" stat="false" type="string_ci" multivalued="true" xpath="//*[local-name()='entity']/*[local-name()='result']/creator/@*[local-name() != 'rank' and local-name() != 'name' and local-name() != 'surname']"/>
<FIELD name="orcidtypevalue" xpath="//*[local-name()='entity']/*[local-name()='result']/creator" type="string_ci" result="false" value="string-join((./@*[local-name() = 'orcid' or local-name() = 'orcid_pending'], ./@*[local-name() = 'orcid' or local-name() = 'orcid_pending']/local-name()), '||' )" stat="false" multivalued="true" indexable="true"/>
<FIELD result="false" xpath="//*[local-name()='entity']/*[local-name()='result']/children/instance/*[local-name()='hostedby']" stat="false" indexable="true" name="resulthostingdatasource" tokenizable="false" value="distinct-values(concat(./@id, '||', ./@name))"/>
<FIELD name="resulthostingdatasourceid" indexable="true" xpath="distinct-values(//*[local-name()='entity']/*[local-name()='result']/children/instance/*[local-name()='hostedby']/@id)" result="false" tokenizable="false" stat="false"/>
<FIELD name="instancetypename" indexable="true" tokenizable="false" stat="false" result="false" xpath="distinct-values(//*[local-name()='entity']/*[local-name()='result']/children/instance/*[local-name()='instancetype']/@classname)"/>
<FIELD name="resultdupid" indexable="true" stat="false" xpath="//*[local-name()='entity']/*//children/result/@objidentifier" result="false" tokenizable="false"/>
<FIELD name="organizationdupid" stat="false" result="false" tokenizable="false" indexable="true" xpath="//*[local-name()='entity']/*//children/organization/@objidentifier"/>
<FIELD name="externalreflabel" stat="false" tokenizable="true" result="false" indexable="false" copy="true" xpath="distinct-values(//*[local-name()='entity']/*//children/externalreference/label)"/>
<FIELD name="resultidentifier" xpath="distinct-values(//*[local-name()='entity']/*[local-name()='result']/children/instance/webresource/*[local-name()='url'])" copy="true" type="string_ci" stat="false" indexable="true" result="false"/>
<FIELD name="resultsource" result="false" xpath="distinct-values(//*[local-name()='entity']/*[local-name()='result']/source)" indexable="false" copy="true" stat="false"/>
<FIELD name="eoscifguidelines" stat="false" indexable="true" tokenizable="false" result="false" xpath="distinct-values(//*[local-name() = 'result']/eoscifguidelines/@code)"/><!-- FOS and SDGs non tokenizable for faceted search-->
<FIELD name="fos" xpath="distinct-values(//*[local-name()='entity']/*[local-name()='result']/subject[@classid='FOS'])" indexable="true" stat="false" tokenizable="false" result="false"/>
<FIELD name="sdg" tokenizable="false" indexable="true" xpath="distinct-values(//*[local-name()='entity']/*[local-name()='result']/subject[@classid='SDG'])" stat="false" result="false"/><!-- REL FIELDS -->
<FIELD name="reldatasourcecompatibilityid" result="false" xpath="distinct-values(//*[local-name()='entity']/*//rel[./to/@type='datasource']/openairecompatibility/@classid)" indexable="true" stat="false" tokenizable="false"/>
<FIELD name="relproject" value="distinct-values(concat(./text(), '||', dnet:pickFirst(../acronym/text(), ../title/text())))" result="false" xpath="//*[local-name()='entity']/*//rel/to[@type='project']" stat="false" tokenizable="false" indexable="true"/>
<FIELD name="relprojectid" xpath="distinct-values(//*[local-name()='entity']/*//rel/to[@type='project'])" indexable="true" tokenizable="false" result="false" stat="false"/>
<FIELD name="relprojectcode" tokenizable="false" indexable="true" xpath="distinct-values(//*[local-name()='entity']/*//rel[./to/@type='project']/code)" stat="false" result="false"/>
<FIELD name="relprojectname" stat="false" result="false" indexable="true" copy="true" xpath="distinct-values(//*[local-name()='entity']/*//rel[./to/@type='project']/acronym)" tokenizable="false"/>
<FIELD name="relprojecttitle" indexable="false" stat="false" copy="true" result="false" xpath="distinct-values(//*[local-name()='entity']/*//rel[./to/@type='project']/title)"/>
<FIELD name="relcontracttypename" stat="false" indexable="false" copy="true" result="false" xpath="distinct-values(//*[local-name()='entity']/*//rel[./to/@type='project']/contracttype/@classname)"/>
<FIELD name="relorganizationcountryid" stat="false" result="false" indexable="true" tokenizable="false" xpath="distinct-values(//*[local-name()='entity']/*//rel[./to/@type='organization']/country/@classid)"/>
<FIELD name="relorganizationcountryname" result="false" copy="true" xpath="distinct-values(//*[local-name()='entity']/*//rel[./to/@type='organization']/country/@classname)" indexable="false" stat="false"/>
<FIELD name="relorganizationid" result="false" indexable="true" stat="false" tokenizable="false" xpath="distinct-values(//*[local-name()='entity']/*//rel/to[@type='organization'])"/>
<FIELD name="relorganizationname" xpath="distinct-values(//*[local-name()='entity']/*//rel[./to/@type='organization']/legalname)" result="false" stat="false" indexable="true" copy="true"/>
<FIELD name="relorganizationshortname" xpath="distinct-values(//*[local-name()='entity']/*//rel[./to/@type='organization']/legalshortname)" indexable="true" result="false" copy="true" stat="false"/>
<FIELD name="relresulttype" indexable="true" xpath="distinct-values(//*[local-name()='entity']/*//rel/to/@type)" tokenizable="false" stat="false" result="false"/>
<FIELD name="relclass" stat="false" tokenizable="false" xpath="distinct-values(//*[local-name()='entity']/*//rel/to/@class)" result="false" indexable="true"/>
<FIELD name="relfundinglevel0_id" stat="false" indexable="true" result="false" xpath="//*[local-name()='entity']//rel/funding/funding_level_0" tokenizable="false"/>
<FIELD name="relfundinglevel0_name" indexable="true" tokenizable="false" xpath="//*[local-name()='entity']//rel/funding/funding_level_0/@name/string()" result="false" stat="false"/>
<FIELD name="relfundinglevel1_id" indexable="true" stat="false" tokenizable="false" result="false" xpath="//*[local-name()='entity']//rel/funding/funding_level_1"/>
<FIELD name="relfundinglevel1_name" xpath="//*[local-name()='entity']//rel/funding/funding_level_1/@name/string()" result="false" indexable="true" stat="false" tokenizable="false"/>
<FIELD name="relfundinglevel2_id" result="false" xpath="//*[local-name()='entity']//rel/funding/funding_level_2" indexable="true" tokenizable="false" stat="false"/>
<FIELD name="relfundinglevel2_name" xpath="//*[local-name()='entity']//rel/funding/funding_level_2/@name/string()" tokenizable="false" result="false" stat="false" indexable="true"/><!-- PROJECTS' FUNDER FIELDS: indexable only with the new funding path/context handling -->
<FIELD name="relfunder" indexable="true" tokenizable="false" stat="false" result="false" value="distinct-values(concat(@id, '||', @name, '||', @shortname))" xpath="//*[local-name()='entity']//rel/funding/funder"/>
<FIELD name="relfunderid" xpath="distinct-values(//*[local-name()='entity']//rel/funding/funder/@id)" stat="false" tokenizable="false" result="false" indexable="true"/>
<FIELD name="relfundershortname" indexable="true" xpath="distinct-values(//*[local-name()='entity']//rel/funding/funder/@shortname)" result="false" stat="false" tokenizable="false"/>
<FIELD name="semrelid" stat="false" tokenizable="false" value="concat(./to/text(), '||', ./to/@class/string())" indexable="true" result="false" xpath="//*[local-name()='entity']//rel"/><!-- COMMON FIELDS -->
<FIELD name="dateofcollection" stat="false" type="date" indexable="true" result="false" multivalued="false" value="//header/*[local-name()='dateOfCollection']"/>
<FIELD name="status" type="string_ci" tokenizable="false" stat="false" indexable="true" xpath="//header/*[local-name()='status']" result="false"/>
<FIELD name="collectedfromdatasourceid" indexable="true" result="false" xpath="distinct-values(//*[local-name()='entity']/*/*[local-name()='collectedfrom']/@id | //*[local-name()='entity']/*//*[local-name() = 'instance']/*[local-name()='collectedfrom']/@id)" tokenizable="false" stat="false"/>
<FIELD name="collectedfromname" stat="false" xpath="distinct-values(//*[local-name()='entity']/*/*[local-name()='collectedfrom']/@name | //*[local-name()='entity']/*//*[local-name() = 'instance']/*[local-name()='collectedfrom']/@name)" result="false" tokenizable="false" indexable="true"/>
<FIELD name="originalid" result="false" indexable="true" type="string_ci" xpath="//*[local-name()='entity']/*/*[local-name()='originalId']" tokenizable="false" stat="false"/>
<FIELD name="pid" type="string_ci" tokenizable="false" indexable="true" result="false" xpath="//*[local-name()='entity']/*/pid/text()" stat="false"/>
<FIELD name="pidclassid" result="false" xpath="distinct-values(//*[local-name()='entity']/*/pid/@classid)" stat="false" indexable="true" tokenizable="false"/>
<FIELD name="deletedbyinference" indexable="true" stat="false" xpath="//*[local-name()='entity']//datainfo/deletedbyinference" result="false" tokenizable="false"/>
<FIELD name="provenanceactionclassid" result="false" stat="false" xpath="//*[local-name()='entity']//datainfo/provenanceaction/@classid" tokenizable="false" indexable="true"/>
<FIELD name="contextid" xpath="distinct-values(//*[local-name()='entity']/*[local-name()='result']/context/@id)" tokenizable="false" indexable="true" stat="false" result="false"/>
<FIELD name="contextname" indexable="true" result="false" stat="false" tokenizable="false" xpath="distinct-values(//*[local-name()='entity']/*[local-name()='result']/context/@label)"/><!-- Need special fields for community (research initiative) context in order to exclude funders from the context browse -->
<FIELD name="community" tokenizable="false" result="false" xpath="//*[local-name()='entity']/*[local-name()='result']/context[@type='community' or @type='ri']" value="distinct-values(concat(@id, '||', @label))" stat="false" indexable="true"/>
<FIELD name="communityid" xpath="distinct-values(//*[local-name()='entity']/*[local-name()='result']/context[@type='community' or @type='ri']/@id)" stat="false" tokenizable="false" result="false" indexable="true"/>
<FIELD name="categoryid" xpath="distinct-values(//*[local-name()='entity']/*[local-name()='result']/context/category/@id)" stat="false" indexable="true" result="false" tokenizable="false"/>
<FIELD name="conceptname" stat="false" indexable="true" xpath="distinct-values(//*[local-name()='entity']/*[local-name()='result']/context/category//concept/@label)" result="false" tokenizable="false"/><!-- new index field for country info from different xpaths for any type of entity -->
<FIELD name="country" result="false" stat="false" xpath="distinct-values(//*[local-name()='entity']/*/country/@classid | //*[local-name()='entity']/*//rel[./to/@type='organization']/country/@classid | //*[local-name()='entity']//funder/@jurisdiction)" tokenizable="false" indexable="true"/>
<FIELD name="oafentity" indexable="false" result="true" tokenizable="false" stat="false" xpath="//*[local-name() = 'entity']"/>
<FIELD indexable="true" name="oaftype" result="false" stat="false" tokenizable="false" value="local-name(//*[local-name()='entity']/*[local-name() != 'extraInfo'])"/>
<FIELD indexable="true" name="objidentifier" result="false" stat="false" tokenizable="false" xpath="//header/dri:objIdentifier"/><!-- DATASOURCE FIELDS -->
<FIELD copy="true" indexable="true" name="datasourceofficialname" result="false" stat="false" xpath="//*[local-name()='entity']/*[local-name()='datasource']/officialname"/>
<FIELD copy="true" indexable="true" name="datasourceenglishname" result="false" stat="false" xpath="//*[local-name()='entity']/*[local-name()='datasource']/englishname"/>
<FIELD copy="true" indexable="false" name="datasourceoddescription" result="false" stat="false" xpath="//*[local-name()='entity']/*[local-name()='datasource']/oddescription"/>
<FIELD copy="true" indexable="true" name="datasourceodsubjects" result="false" stat="false" tokenizable="false" xpath="//*[local-name()='entity']/*[local-name()='datasource']/odsubjects"/>
<FIELD indexable="true" name="datasourceodlanguages" result="false" stat="false" tokenizable="false" xpath="//*[local-name()='entity']/*[local-name()='datasource']/odlanguages"/>
<FIELD indexable="true" name="datasourceodcontenttypes" result="false" stat="false" tokenizable="false" xpath="//*[local-name()='entity']/*[local-name()='datasource']/odcontenttypes"/>
<FIELD indexable="true" multivalued="false" name="datasourcetypename" result="false" stat="false" tokenizable="false" xpath="//*[local-name()='entity']/*[local-name()='datasource']/datasourcetype/@classname"/>
<FIELD indexable="true" multivalued="false" name="datasourcetypeuiid" result="false" stat="false" tokenizable="false" xpath="//*[local-name()='entity']/*[local-name()='datasource']/datasourcetypeui/@classid"/>
<FIELD indexable="true" multivalued="false" name="datasourcetypeuiname" result="false" stat="false" tokenizable="false" xpath="//*[local-name()='entity']/*[local-name()='datasource']/datasourcetypeui/@classname"/>
<FIELD indexable="true" multivalued="false" name="datasourcecompatibilityid" result="false" stat="false" tokenizable="false" xpath="//*[local-name()='entity']/*[local-name()='datasource']/openairecompatibility/@classid"/>
<FIELD indexable="true" multivalued="false" name="datasourcecompatibilityname" result="false" stat="false" tokenizable="false" xpath="//*[local-name()='entity']/*[local-name()='datasource']/openairecompatibility/@classname"/>
<FIELD copy="true" indexable="true" multivalued="true" name="datasourcesubject" result="false" stat="false" type="ngramtext" xpath="//*[local-name()='entity']/*[local-name()='datasource']/subjects"/><!-- datasource fields for EOSC -->
<FIELD indexable="true" name="datasourcejurisdiction" result="false" stat="false" tokenizable="false" xpath="//*[local-name()='entity']/*[local-name()='datasource']/jurisdiction/@classname"/>
<FIELD indexable="true" name="datasourcethematic" result="false" stat="false" tokenizable="false" xpath="//*[local-name()='entity']/*[local-name()='datasource']/thematic"/>
<FIELD indexable="true" name="eosctype" result="false" stat="false" tokenizable="false" xpath="//*[local-name()='entity']/*[local-name()='datasource']/eosctype/@classname"/>
<FIELD indexable="true" name="eoscdatasourcetype" result="false" stat="false" tokenizable="false" xpath="//*[local-name()='entity']/*[local-name()='datasource']/eoscdatasourcetype/@classname"/><!-- ORGANIZATION FIELDS -->
<FIELD copy="true" indexable="true" name="organizationlegalshortname" result="false" stat="false" type="ngramtext" xpath="distinct-values(//*[local-name()='entity']/*[local-name()='organization']//legalshortname)"/>
<FIELD copy="true" indexable="true" name="organizationlegalname" result="false" stat="false" type="ngramtext" xpath="distinct-values(//*[local-name()='entity']/*[local-name()='organization']//legalname)"/>
<FIELD copy="true" indexable="false" name="organizationalternativenames" result="false" stat="false" type="ngramtext" xpath="distinct-values(//*[local-name()='entity']/*[local-name()='organization']//alternativeNames)"/><!-- PROJECT FIELDS -->
<FIELD copy="true" indexable="false" name="projectcode" result="false" stat="false" type="ngramtext" xpath="//*[local-name()='entity']/*[local-name()='project']/code"/>
<FIELD indexable="true" name="projectcode_nt" result="false" stat="false" tokenizable="false" xpath="//*[local-name()='entity']/*[local-name()='project']/code"/>
<FIELD copy="true" indexable="true" name="projectacronym" result="false" stat="false" type="ngramtext" xpath="//*[local-name()='entity']/*[local-name()='project']/acronym"/>
<FIELD copy="true" indexable="true" name="projecttitle" result="false" stat="false" type="ngramtext" xpath="//*[local-name()='entity']/*[local-name()='project']/title"/>
<FIELD indexable="true" multivalued="false" name="projectstartdate" result="false" stat="false" type="date" value="//*[local-name()='entity']/*[local-name()='project']/startdate"/>
<FIELD indexable="true" multivalued="false" name="projectstartyear" result="false" stat="false" tokenizable="false" value="dnet:extractYear(//*[local-name()='entity']/*[local-name()='project']/startdate)"/>
<FIELD indexable="true" multivalued="false" name="projectenddate" result="false" stat="false" type="date" value="//*[local-name()='entity']/*[local-name()='project']/enddate"/>
<FIELD indexable="true" multivalued="false" name="projectendyear" result="false" stat="false" tokenizable="false" value="dnet:extractYear(//*[local-name()='entity']/*[local-name()='project']/enddate)"/>
<FIELD indexable="true" multivalued="false" name="projectcallidentifier" result="false" stat="false" tokenizable="false" xpath="//*[local-name()='entity']/*[local-name()='project']/callidentifier"/>
<FIELD copy="true" indexable="false" name="projectkeywords" result="false" stat="false" xpath="//*[local-name()='entity']/*[local-name()='project']/keywords"/>
<FIELD indexable="true" multivalued="false" name="projectduration" result="false" stat="false" tokenizable="false" xpath="//*[local-name()='entity']/*[local-name()='project']/duration"/>
<FIELD indexable="true" multivalued="false" name="projectecsc39" result="false" stat="false" tokenizable="false" xpath="distinct-values(//*[local-name()='entity']/*[local-name()='project']/ecsc39)"/>
<FIELD indexable="true" multivalued="false" name="projectoamandatepublications" result="false" stat="false" tokenizable="false" xpath="//*[local-name()='entity']/*[local-name()='project']/oamandatepublications"/>
<FIELD indexable="true" name="fundinglevel0_id" result="false" stat="false" tokenizable="false" xpath="//*[local-name()='entity']/*[local-name()='project']/fundingtree//funding_level_0/id"/>
<FIELD indexable="true" name="fundinglevel0_name" result="false" stat="false" tokenizable="false" xpath="//*[local-name()='entity']/*[local-name()='project']/fundingtree//funding_level_0/name"/>
<FIELD copy="true" indexable="false" name="fundinglevel0_description" result="false" stat="false" xpath="//*[local-name()='entity']/*[local-name()='project']/fundingtree//funding_level_0/description"/>
<FIELD indexable="true" name="fundinglevel1_id" result="false" stat="false" tokenizable="false" xpath="//*[local-name()='entity']/*[local-name()='project']/fundingtree//funding_level_1/id"/>
<FIELD indexable="true" name="fundinglevel1_name" result="false" stat="false" tokenizable="false" xpath="//*[local-name()='entity']/*[local-name()='project']/fundingtree//funding_level_1/name"/>
<FIELD copy="true" indexable="false" name="fundinglevel1_description" result="false" stat="false" xpath="//*[local-name()='entity']/*[local-name()='project']/fundingtree//funding_level_1/description"/>
<FIELD indexable="true" name="fundinglevel2_id" result="false" stat="false" tokenizable="false" xpath="//*[local-name()='entity']/*[local-name()='project']/fundingtree//funding_level_2/id"/>
<FIELD indexable="true" name="fundinglevel2_name" result="false" stat="false" tokenizable="false" xpath="//*[local-name()='entity']/*[local-name()='project']/fundingtree//funding_level_2/name"/>
<FIELD copy="true" indexable="false" name="fundinglevel2_description" result="false" stat="false" xpath="//*[local-name()='entity']/*[local-name()='project']/fundingtree//funding_level_2/description"/><!-- PROJECTS' FUNDER FIELDS: indexable only with the new funding path/context handling -->
<FIELD indexable="true" name="funder" result="false" stat="false" tokenizable="false" value="concat(./id/text(), '||', ./name/text(), '||', ./shortname/text())" xpath="//*[local-name()='entity']/*[local-name()='project']/fundingtree/funder"/>
<FIELD indexable="true" name="fundershortname" result="false" stat="false" tokenizable="false" xpath="//*[local-name()='entity']/*[local-name()='project']/fundingtree/funder/shortname"/>
<FIELD indexable="true" name="funderid" result="false" stat="false" tokenizable="false" xpath="//*[local-name()='entity']/*[local-name()='project']/fundingtree/funder/id"/><!-- RESULT FIELDS -->
<FIELD copy="true" indexable="true" name="resulttitle" result="false" stat="false" type="text_en" xpath="//*[local-name() = 'entity']/*[local-name() ='result']/title | //*[local-name()='entity']/*[local-name()='result']/children/result/title"/>
<FIELD indexable="true" name="resultsubject" result="false" stat="false" type="text_en" xpath="distinct-values(//*[local-name()='entity']/*[local-name()='result']/subject)"/>
<FIELD indexable="true" multivalued="false" name="resultembargoenddate" result="false" stat="false" type="date" value="//*[local-name()='entity']/*[local-name()='result']/embargoenddate"/>
<FIELD indexable="true" multivalued="false" name="resultembargoendyear" result="false" stat="false" tokenizable="false" value="dnet:extractYear(//*[local-name()='entity']/*[local-name()='result']/embargoenddate)"/>
<FIELD indexable="true" multivalued="false" name="resulttypeid" result="false" stat="false" tokenizable="false" xpath="//*[local-name()='entity']/*[local-name()='result']/resulttype/@classid"/>
<FIELD indexable="true" multivalued="false" name="resultlanguagename" result="false" stat="false" tokenizable="false" xpath="//*[local-name()='entity']/*[local-name()='result']/language/@classname"/>
<FIELD copy="true" indexable="true" name="resultpublisher" result="false" stat="false" xpath="//*[local-name()='entity']/*[local-name()='result']/*[local-name()='publisher']"/>
<FIELD copy="true" indexable="true" name="resultdescription" result="false" stat="false" type="text_en" xpath="//*[local-name()='entity']/*[local-name()='result']//*[local-name()='description']"/>
<FIELD indexable="true" multivalued="false" name="resultbestaccessright" result="false" stat="false" tokenizable="false" xpath="distinct-values(//*[local-name()='entity']/*[local-name()='result']/bestaccessright/@classname)"/>
<FIELD indexable="true" multivalued="false" name="resultdateofacceptance" result="false" stat="false" type="date" value="//*[local-name()='entity']/*[local-name()='result']/dateofacceptance"/>
<FIELD copy="true" indexable="true" multivalued="false" name="resultacceptanceyear" result="false" stat="false" tokenizable="false" value="dnet:extractYear(//*[local-name()='entity']/*[local-name()='result']/dateofacceptance)"/>
<FIELD copy="true" indexable="true" multivalued="true" name="resultauthor" result="false" stat="false" xpath="//*[local-name()='entity']/*[local-name()='result']/creator"/>
<FIELD indexable="true" multivalued="true" name="authorid" result="false" stat="false" type="string_ci" xpath="//*[local-name()='entity']/*[local-name()='result']/creator/@*[local-name() != 'rank' and local-name() != 'name' and local-name() != 'surname']"/>
<FIELD indexable="true" multivalued="true" name="orcidtypevalue" result="false" stat="false" type="string_ci" value="string-join((./@*[local-name() = 'orcid' or local-name() = 'orcid_pending'], ./@*[local-name() = 'orcid' or local-name() = 'orcid_pending']/local-name()), '||' )" xpath="//*[local-name()='entity']/*[local-name()='result']/creator"/>
<FIELD indexable="true" name="resulthostingdatasource" result="false" stat="false" tokenizable="false" value="distinct-values(concat(./@id, '||', ./@name))" xpath="//*[local-name()='entity']/*[local-name()='result']/children/instance/*[local-name()='hostedby']"/>
<FIELD indexable="true" name="resulthostingdatasourceid" result="false" stat="false" tokenizable="false" xpath="distinct-values(//*[local-name()='entity']/*[local-name()='result']/children/instance/*[local-name()='hostedby']/@id)"/>
<FIELD indexable="true" name="instancetypename" result="false" stat="false" tokenizable="false" xpath="distinct-values(//*[local-name()='entity']/*[local-name()='result']/children/instance/*[local-name()='instancetype']/@classname)"/>
<FIELD indexable="true" name="resultdupid" result="false" stat="false" tokenizable="false" xpath="//*[local-name()='entity']/*//children/result/@objidentifier"/>
<FIELD indexable="true" name="organizationdupid" result="false" stat="false" tokenizable="false" xpath="//*[local-name()='entity']/*//children/organization/@objidentifier"/>
<FIELD copy="true" indexable="false" name="externalreflabel" result="false" stat="false" tokenizable="true" xpath="distinct-values(//*[local-name()='entity']/*//children/externalreference/label)"/>
<FIELD copy="true" indexable="true" name="resultidentifier" result="false" stat="false" type="string_ci" xpath="distinct-values(//*[local-name()='entity']/*[local-name()='result']/children/instance/webresource/*[local-name()='url'])"/>
<FIELD copy="true" indexable="false" name="resultsource" result="false" stat="false" xpath="distinct-values(//*[local-name()='entity']/*[local-name()='result']/source)"/>
<FIELD indexable="true" multivalued="false" name="isgreen" result="false" stat="false" type="boolean" value="//*[local-name()='entity']/*[local-name()='result']/isgreen"/>
<FIELD indexable="true" multivalued="false" name="openaccesscolor" result="false" stat="false" tokenizable="false" value="//*[local-name()='entity']/*[local-name()='result']/openaccesscolor"/>
<FIELD indexable="true" multivalued="false" name="isindiamondjournal" result="false" stat="false" type="boolean" value="//*[local-name()='entity']/*[local-name()='result']/isindiamondjournal"/>
<FIELD indexable="true" multivalued="false" name="publiclyfunded" result="false" stat="false" type="boolean" value="//*[local-name()='entity']/*[local-name()='result']/publiclyfunded"/>
<FIELD indexable="true" multivalued="false" name="peerreviewed" result="false" stat="false" type="boolean" value="some $refereed in //*[local-name()='entity']/*[local-name()='result']/children/instance/*[local-name()='refereed']/@classid satisfies ($refereed = '0001')"/>
<FIELD indexable="true" name="eoscifguidelines" result="false" stat="false" tokenizable="false" xpath="distinct-values(//*[local-name() = 'result']/eoscifguidelines/@code)"/><!-- FOS and SDGs non tokenizable for faceted search-->
<FIELD indexable="true" name="fos" result="false" stat="false" tokenizable="false" xpath="distinct-values(//*[local-name()='entity']/*[local-name()='result']/subject[@classid='FOS'])"/>
<FIELD indexable="true" name="foslabel" result="false" stat="false" tokenizable="false" value="concat(./text(), '||', replace(./text(), '^\d+\s', ''))" xpath="//*[local-name()='entity']/*[local-name()='result']/subject[@classid='FOS']"/>
<FIELD indexable="true" name="sdg" result="false" stat="false" tokenizable="false" xpath="distinct-values(//*[local-name()='entity']/*[local-name()='result']/subject[@classid='SDG'])"/><!-- REL FIELDS -->
<FIELD indexable="true" name="reldatasourcecompatibilityid" result="false" stat="false" tokenizable="false" xpath="distinct-values(//*[local-name()='entity']/*//rel[./to/@type='datasource']/openairecompatibility/@classid)"/>
<FIELD indexable="true" name="relproject" result="false" stat="false" tokenizable="false" value="distinct-values(concat(./text(), '||', dnet:pickFirst(../acronym/text(), ../title/text())))" xpath="//*[local-name()='entity']/*//rel/to[@type='project']"/>
<FIELD indexable="true" name="relprojectid" result="false" stat="false" tokenizable="false" xpath="distinct-values(//*[local-name()='entity']/*//rel/to[@type='project'])"/>
<FIELD indexable="true" name="relprojectcode" result="false" stat="false" tokenizable="false" xpath="distinct-values(//*[local-name()='entity']/*//rel[./to/@type='project']/code)"/>
<FIELD copy="true" indexable="true" name="relprojectname" result="false" stat="false" tokenizable="false" xpath="distinct-values(//*[local-name()='entity']/*//rel[./to/@type='project']/acronym)"/>
<FIELD copy="true" indexable="false" name="relprojecttitle" result="false" stat="false" xpath="distinct-values(//*[local-name()='entity']/*//rel[./to/@type='project']/title)"/>
<FIELD copy="true" indexable="false" name="relcontracttypename" result="false" stat="false" xpath="distinct-values(//*[local-name()='entity']/*//rel[./to/@type='project']/contracttype/@classname)"/>
<FIELD indexable="true" name="relorganizationcountryid" result="false" stat="false" tokenizable="false" xpath="distinct-values(//*[local-name()='entity']/*//rel[./to/@type='organization']/country/@classid)"/>
<FIELD copy="true" indexable="false" name="relorganizationcountryname" result="false" stat="false" xpath="distinct-values(//*[local-name()='entity']/*//rel[./to/@type='organization']/country/@classname)"/>
<FIELD indexable="true" name="relorganizationid" result="false" stat="false" tokenizable="false" xpath="distinct-values(//*[local-name()='entity']/*//rel/to[@type='organization'])"/>
<FIELD copy="true" indexable="true" name="relorganizationname" result="false" stat="false" xpath="distinct-values(//*[local-name()='entity']/*//rel[./to/@type='organization']/legalname)"/>
<FIELD copy="true" indexable="true" name="relorganizationshortname" result="false" stat="false" xpath="distinct-values(//*[local-name()='entity']/*//rel[./to/@type='organization']/legalshortname)"/>
<FIELD indexable="true" name="relresultid" result="false" stat="false" tokenizable="false" xpath="distinct-values(//*[local-name()='entity']/*//rel/to[@type='publication' or @type='dataset' or @type='software' or @type='otherresearchproduct'])"/>
<FIELD indexable="true" name="relresulttype" result="false" stat="false" tokenizable="false" xpath="distinct-values(//*[local-name()='entity']/*//rel/to/@type)"/>
<FIELD indexable="true" name="relclass" result="false" stat="false" tokenizable="false" xpath="distinct-values(//*[local-name()='entity']/*//rel/to/@class)"/>
<FIELD indexable="true" name="relfundinglevel0_id" result="false" stat="false" tokenizable="false" xpath="//*[local-name()='entity']//rel/funding/funding_level_0"/>
<FIELD indexable="true" name="relfundinglevel0_name" result="false" stat="false" tokenizable="false" xpath="//*[local-name()='entity']//rel/funding/funding_level_0/@name/string()"/>
<FIELD indexable="true" name="relfundinglevel1_id" result="false" stat="false" tokenizable="false" xpath="//*[local-name()='entity']//rel/funding/funding_level_1"/>
<FIELD indexable="true" name="relfundinglevel1_name" result="false" stat="false" tokenizable="false" xpath="//*[local-name()='entity']//rel/funding/funding_level_1/@name/string()"/>
<FIELD indexable="true" name="relfundinglevel2_id" result="false" stat="false" tokenizable="false" xpath="//*[local-name()='entity']//rel/funding/funding_level_2"/>
<FIELD indexable="true" name="relfundinglevel2_name" result="false" stat="false" tokenizable="false" xpath="//*[local-name()='entity']//rel/funding/funding_level_2/@name/string()"/><!-- PROJECTS' FUNDER FIELDS: indexable only with the new funding path/context handling -->
<FIELD indexable="true" name="relfunder" result="false" stat="false" tokenizable="false" value="distinct-values(concat(@id, '||', @name, '||', @shortname))" xpath="//*[local-name()='entity']//rel/funding/funder"/>
<FIELD indexable="true" name="relfunderid" result="false" stat="false" tokenizable="false" xpath="distinct-values(//*[local-name()='entity']//rel/funding/funder/@id)"/>
<FIELD indexable="true" name="relfundershortname" result="false" stat="false" tokenizable="false" xpath="distinct-values(//*[local-name()='entity']//rel/funding/funder/@shortname)"/>
<FIELD indexable="true" name="semrelid" result="false" stat="false" tokenizable="false" value="concat(./to/text(), '||', ./to/@class/string())" xpath="//*[local-name()='entity']//rel"/><!-- COMMON FIELDS -->
<FIELD indexable="true" multivalued="false" name="dateofcollection" result="false" stat="false" type="date" value="//header/*[local-name()='dateOfCollection']"/>
<FIELD indexable="true" name="status" result="false" stat="false" tokenizable="false" type="string_ci" xpath="//header/*[local-name()='status']"/>
<FIELD indexable="true" name="collectedfromdatasourceid" result="false" stat="false" tokenizable="false" xpath="distinct-values(//*[local-name()='entity']/*/*[local-name()='collectedfrom']/@id | //*[local-name()='entity']/*//*[local-name() = 'instance']/*[local-name()='collectedfrom']/@id)"/>
<FIELD indexable="true" name="collectedfromname" result="false" stat="false" tokenizable="false" xpath="distinct-values(//*[local-name()='entity']/*/*[local-name()='collectedfrom']/@name | //*[local-name()='entity']/*//*[local-name() = 'instance']/*[local-name()='collectedfrom']/@name)"/>
<FIELD indexable="true" name="originalid" result="false" stat="false" tokenizable="false" type="string_ci" xpath="//*[local-name()='entity']/*/*[local-name()='originalId']"/>
<FIELD indexable="true" name="pid" result="false" stat="false" tokenizable="false" type="string_ci" xpath="//*[local-name()='entity']/*/pid/text()"/>
<FIELD indexable="true" name="pidclassid" result="false" stat="false" tokenizable="false" xpath="distinct-values(//*[local-name()='entity']/*/pid/@classid)"/>
<FIELD indexable="true" name="deletedbyinference" result="false" stat="false" tokenizable="false" xpath="//*[local-name()='entity']//datainfo/deletedbyinference"/>
<FIELD indexable="true" name="provenanceactionclassid" result="false" stat="false" tokenizable="false" xpath="//*[local-name()='entity']//datainfo/provenanceaction/@classid"/>
<FIELD indexable="true" name="contextid" result="false" stat="false" tokenizable="false" xpath="distinct-values(//*[local-name()='entity']/*[local-name()='result']/context/@id)"/>
<FIELD indexable="true" name="contextname" result="false" stat="false" tokenizable="false" xpath="distinct-values(//*[local-name()='entity']/*[local-name()='result']/context/@label)"/><!-- Need special fields for community (research initiative) context in order to exclude funders from the context browse -->
<FIELD indexable="true" name="community" result="false" stat="false" tokenizable="false" value="distinct-values(concat(@id, '||', @label))" xpath="//*[local-name()='entity']/*[local-name()='result']/context[@type='community' or @type='ri']"/>
<FIELD indexable="true" name="communityid" result="false" stat="false" tokenizable="false" xpath="distinct-values(//*[local-name()='entity']/*[local-name()='result']/context[@type='community' or @type='ri']/@id)"/>
<FIELD indexable="true" name="categoryid" result="false" stat="false" tokenizable="false" xpath="distinct-values(//*[local-name()='entity']/*[local-name()='result']/context/category/@id)"/>
<FIELD indexable="true" name="conceptname" result="false" stat="false" tokenizable="false" xpath="distinct-values(//*[local-name()='entity']/*[local-name()='result']/context/category//concept/@label)"/><!-- new index field for country info from different xpaths for any type of entity -->
<FIELD indexable="true" name="country" result="false" stat="false" tokenizable="false" xpath="distinct-values(//*[local-name()='entity']/*/country/@classid | //*[local-name()='entity']/*//rel[./to/@type='organization']/country/@classid | //*[local-name()='entity']//funder/@jurisdiction)"/>
<FIELD indexable="false" name="oafentity" result="true" stat="false" tokenizable="false" xpath="//*[local-name() = 'entity']"/><!-- impact indicators -->
<FIELD copy="false" indexable="true" multivalued="false" name="influence" result="false" stat="false" type="pfloat" xpath="//measure[@id='influence']/@score/number()"/>
<FIELD copy="false" indexable="true" multivalued="false" name="influence_class" result="false" stat="false" type="string" xpath="//measure[@id='influence']/@class/string()"/>
<FIELD copy="false" indexable="true" multivalued="false" name="popularity" result="false" stat="false" type="pfloat" xpath="//measure[@id='popularity']/@score/number()"/>
<FIELD copy="false" indexable="true" multivalued="false" name="popularity_class" result="false" stat="false" type="string" xpath="//measure[@id='popularity']/@class/string()"/>
<FIELD copy="false" indexable="true" multivalued="false" name="citation_count" result="false" stat="false" type="pint" xpath="//measure[@id='influence_alt']/@score/number()"/>
<FIELD copy="false" indexable="true" multivalued="false" name="citation_count_class" result="false" stat="false" type="string" xpath="//measure[@id='influence_alt']/@class/string()"/>
<FIELD copy="false" indexable="true" multivalued="false" name="popularity_alt" result="false" stat="false" type="pfloat" xpath="//measure[@id='popularity_alt']/@score/number()"/>
<FIELD copy="false" indexable="true" multivalued="false" name="popularity_alt_class" result="false" stat="false" type="string" xpath="//measure[@id='popularity_alt']/@class/string()"/>
<FIELD copy="false" indexable="true" multivalued="false" name="impulse" result="false" stat="false" type="pint" xpath="//measure[@id='impulse']/@score/number()"/>
<FIELD copy="false" indexable="true" multivalued="false" name="impulse_class" result="false" stat="false" type="string" xpath="//measure[@id='impulse']/@class/string()"/>
</FIELDS>
</LAYOUT>

View File

@ -1048,8 +1048,8 @@
"schemename": "dnet:dataCite_resource"
},
"refereed": {
"classid": "0001",
"classname": "peerReviewed",
"classid": "0002",
"classname": "nonPeerReviewed",
"schemeid": "dnet:review_levels",
"schemename": "dnet:review_levels"
},
@ -1949,5 +1949,9 @@
"schemename": "dnet:provenanceActions"
}
}
}
},
"isGreen": true,
"openAccessColor": "bronze",
"isInDiamondJournal": true,
"publiclyFunded": true
}

View File

@ -219,20 +219,6 @@
<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"/>
@ -368,6 +354,11 @@
<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"/>
<!-- Copy field definitions follow: -->

View File

@ -365,7 +365,14 @@
<field name="semrelid" type="string" 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="datasourceofficialname" dest="__all"/>