diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/project/utils/ReadCSV.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/project/utils/ReadCSV.java index dc6f46771..ca1c10611 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/project/utils/ReadCSV.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/project/utils/ReadCSV.java @@ -17,8 +17,8 @@ import org.apache.hadoop.fs.Path; import com.fasterxml.jackson.databind.ObjectMapper; -import eu.dnetlib.dhp.collection.worker.utils.HttpConnector; import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.collection.worker.utils.HttpConnector; /** * Applies the parsing of a csv file and writes the Serialization of it in hdfs diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/project/utils/ReadExcel.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/project/utils/ReadExcel.java index e665bc704..585a408f3 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/project/utils/ReadExcel.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/project/utils/ReadExcel.java @@ -4,7 +4,6 @@ package eu.dnetlib.dhp.actionmanager.project.utils; import java.io.*; import java.nio.charset.StandardCharsets; -import eu.dnetlib.dhp.collection.worker.utils.HttpConnector; import org.apache.commons.io.IOUtils; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -15,8 +14,8 @@ import org.apache.hadoop.fs.Path; import com.fasterxml.jackson.databind.ObjectMapper; - import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.collection.worker.utils.HttpConnector; /** * Applies the parsing of an excel file and writes the Serialization of it in hdfs diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/aggregation/common/AggregationCounter.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/aggregation/common/AggregationCounter.java new file mode 100644 index 000000000..bf2fd22cb --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/aggregation/common/AggregationCounter.java @@ -0,0 +1,45 @@ + +package eu.dnetlib.dhp.aggregation.common; + +import java.io.Serializable; + +import org.apache.spark.util.LongAccumulator; + +public class AggregationCounter implements Serializable { + private LongAccumulator totalItems; + private LongAccumulator errorItems; + private LongAccumulator processedItems; + + public AggregationCounter() { + } + + public AggregationCounter(LongAccumulator totalItems, LongAccumulator errorItems, LongAccumulator processedItems) { + this.totalItems = totalItems; + this.errorItems = errorItems; + this.processedItems = processedItems; + } + + public LongAccumulator getTotalItems() { + return totalItems; + } + + public void setTotalItems(LongAccumulator totalItems) { + this.totalItems = totalItems; + } + + public LongAccumulator getErrorItems() { + return errorItems; + } + + public void setErrorItems(LongAccumulator errorItems) { + this.errorItems = errorItems; + } + + public LongAccumulator getProcessedItems() { + return processedItems; + } + + public void setProcessedItems(LongAccumulator processedItems) { + this.processedItems = processedItems; + } +} diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/GenerateNativeStoreSparkJob.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/GenerateNativeStoreSparkJob.java index c0bd4c940..c9c29b4ea 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/GenerateNativeStoreSparkJob.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/GenerateNativeStoreSparkJob.java @@ -5,12 +5,9 @@ import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession; import java.io.ByteArrayInputStream; import java.nio.charset.StandardCharsets; -import java.util.HashMap; -import java.util.Map; import java.util.Objects; import java.util.Optional; -import org.apache.commons.cli.*; import org.apache.commons.io.IOUtils; import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.io.IntWritable; @@ -22,7 +19,6 @@ import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Encoder; import org.apache.spark.sql.Encoders; -import org.apache.spark.sql.SparkSession; import org.apache.spark.util.LongAccumulator; import org.dom4j.Document; import org.dom4j.Node; @@ -35,9 +31,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import eu.dnetlib.dhp.application.ArgumentApplicationParser; import eu.dnetlib.dhp.model.mdstore.MetadataRecord; import eu.dnetlib.dhp.model.mdstore.Provenance; -import eu.dnetlib.message.Message; import eu.dnetlib.message.MessageManager; -import eu.dnetlib.message.MessageType; public class GenerateNativeStoreSparkJob { @@ -46,100 +40,62 @@ public class GenerateNativeStoreSparkJob { public static void main(String[] args) throws Exception { final ArgumentApplicationParser parser = new ArgumentApplicationParser( - IOUtils - .toString( - GenerateNativeStoreSparkJob.class - .getResourceAsStream( - "/eu/dnetlib/dhp/collection/collection_input_parameters.json"))); + IOUtils + .toString( + GenerateNativeStoreSparkJob.class + .getResourceAsStream( + "/eu/dnetlib/dhp/collection/collection_input_parameters.json"))); parser.parseArgument(args); final ObjectMapper jsonMapper = new ObjectMapper(); - final Provenance provenance = jsonMapper.readValue(parser.get("provenance"), Provenance.class); - final long dateOfCollection = new Long(parser.get("dateOfCollection")); + final String provenanceArgument = parser.get("provenance"); + log.info("Provenance is {}", provenanceArgument); + final Provenance provenance = jsonMapper.readValue(provenanceArgument, Provenance.class); + final String dateOfCollectionArgs = parser.get("dateOfCollection"); + log.info("dateOfCollection is {}", dateOfCollectionArgs); + final long dateOfCollection = new Long(dateOfCollectionArgs); + final String sequenceFileInputPath = parser.get("input"); + log.info("sequenceFileInputPath is {}", dateOfCollectionArgs); Boolean isSparkSessionManaged = Optional - .ofNullable(parser.get("isSparkSessionManaged")) - .map(Boolean::valueOf) - .orElse(Boolean.TRUE); + .ofNullable(parser.get("isSparkSessionManaged")) + .map(Boolean::valueOf) + .orElse(Boolean.TRUE); log.info("isSparkSessionManaged: {}", isSparkSessionManaged); - final Map ongoingMap = new HashMap<>(); - final Map reportMap = new HashMap<>(); - - final boolean test = parser.get("isTest") == null ? false : Boolean.valueOf(parser.get("isTest")); - SparkConf conf = new SparkConf(); runWithSparkSession( - conf, - isSparkSessionManaged, - spark -> { - final JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext()); + conf, + isSparkSessionManaged, + spark -> { + final JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext()); - final JavaPairRDD inputRDD = sc - .sequenceFile(parser.get("input"), IntWritable.class, Text.class); + final JavaPairRDD inputRDD = sc + .sequenceFile(sequenceFileInputPath, IntWritable.class, Text.class); - final LongAccumulator totalItems = sc.sc().longAccumulator("TotalItems"); - final LongAccumulator invalidRecords = sc.sc().longAccumulator("InvalidRecords"); + final LongAccumulator totalItems = sc.sc().longAccumulator("TotalItems"); + final LongAccumulator invalidRecords = sc.sc().longAccumulator("InvalidRecords"); - final MessageManager manager = new MessageManager( - parser.get("rabbitHost"), - parser.get("rabbitUser"), - parser.get("rabbitPassword"), - false, - false, - null); + final JavaRDD nativeStore = inputRDD + .map( + item -> parseRecord( + item._2().toString(), + parser.get("xpath"), + parser.get("encoding"), + provenance, + dateOfCollection, + totalItems, + invalidRecords)) + .filter(Objects::nonNull) + .distinct(); - final JavaRDD mappeRDD = inputRDD - .map( - item -> parseRecord( - item._2().toString(), - parser.get("xpath"), - parser.get("encoding"), - provenance, - dateOfCollection, - totalItems, - invalidRecords)) - .filter(Objects::nonNull) - .distinct(); + final Encoder encoder = Encoders.bean(MetadataRecord.class); + final Dataset mdstore = spark.createDataset(nativeStore.rdd(), encoder); + final LongAccumulator mdStoreRecords = sc.sc().longAccumulator("MDStoreRecords"); + mdStoreRecords.add(mdstore.count()); - ongoingMap.put("ongoing", "0"); - if (!test) { - manager - .sendMessage( - new Message( - parser.get("workflowId"), "DataFrameCreation", MessageType.ONGOING, ongoingMap), - parser.get("rabbitOngoingQueue"), - true, - false); - } + mdstore.write().format("parquet").save(parser.get("output")); - final Encoder encoder = Encoders.bean(MetadataRecord.class); - final Dataset mdstore = spark.createDataset(mappeRDD.rdd(), encoder); - final LongAccumulator mdStoreRecords = sc.sc().longAccumulator("MDStoreRecords"); - mdStoreRecords.add(mdstore.count()); - ongoingMap.put("ongoing", "" + totalItems.value()); - if (!test) { - manager - .sendMessage( - new Message( - parser.get("workflowId"), "DataFrameCreation", MessageType.ONGOING, ongoingMap), - parser.get("rabbitOngoingQueue"), - true, - false); - } - mdstore.write().format("parquet").save(parser.get("output")); - reportMap.put("inputItem", "" + totalItems.value()); - reportMap.put("invalidRecords", "" + invalidRecords.value()); - reportMap.put("mdStoreSize", "" + mdStoreRecords.value()); - if (!test) { - manager - .sendMessage( - new Message(parser.get("workflowId"), "Collection", MessageType.REPORT, reportMap), - parser.get("rabbitReportQueue"), - true, - false); - manager.close(); - } - }); + }); } @@ -166,12 +122,9 @@ public class GenerateNativeStoreSparkJob { } return new MetadataRecord(originalIdentifier, encoding, provenance, input, dateOfCollection); } catch (Throwable e) { - if (invalidRecords != null) - invalidRecords.add(1); - e.printStackTrace(); + invalidRecords.add(1); return null; } } - } diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/CollectorPlugin.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/CollectorPlugin.java index 4a0c70c45..7146e610e 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/CollectorPlugin.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/CollectorPlugin.java @@ -4,9 +4,9 @@ package eu.dnetlib.dhp.collection.plugin; import java.util.stream.Stream; import eu.dnetlib.collector.worker.model.ApiDescriptor; -import eu.dnetlib.dhp.collection.worker.DnetCollectorException; +import eu.dnetlib.dhp.collection.worker.CollectorException; public interface CollectorPlugin { - Stream collect(ApiDescriptor api) throws DnetCollectorException; + Stream collect(ApiDescriptor api) throws CollectorException; } diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/oai/OaiCollectorPlugin.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/oai/OaiCollectorPlugin.java index 7f71f401d..c4c52271a 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/oai/OaiCollectorPlugin.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/oai/OaiCollectorPlugin.java @@ -15,7 +15,7 @@ import com.google.common.collect.Lists; import eu.dnetlib.collector.worker.model.ApiDescriptor; import eu.dnetlib.dhp.collection.plugin.CollectorPlugin; -import eu.dnetlib.dhp.collection.worker.DnetCollectorException; +import eu.dnetlib.dhp.collection.worker.CollectorException; public class OaiCollectorPlugin implements CollectorPlugin { @@ -27,7 +27,7 @@ public class OaiCollectorPlugin implements CollectorPlugin { private OaiIteratorFactory oaiIteratorFactory; @Override - public Stream collect(final ApiDescriptor api) throws DnetCollectorException { + public Stream collect(final ApiDescriptor api) throws CollectorException { final String baseUrl = api.getBaseUrl(); final String mdFormat = api.getParams().get(FORMAT_PARAM); final String setParam = api.getParams().get(OAI_SET_PARAM); @@ -46,19 +46,19 @@ public class OaiCollectorPlugin implements CollectorPlugin { } if (baseUrl == null || baseUrl.isEmpty()) { - throw new DnetCollectorException("Param 'baseurl' is null or empty"); + throw new CollectorException("Param 'baseurl' is null or empty"); } if (mdFormat == null || mdFormat.isEmpty()) { - throw new DnetCollectorException("Param 'mdFormat' is null or empty"); + throw new CollectorException("Param 'mdFormat' is null or empty"); } if (fromDate != null && !fromDate.matches("\\d{4}-\\d{2}-\\d{2}")) { - throw new DnetCollectorException("Invalid date (YYYY-MM-DD): " + fromDate); + throw new CollectorException("Invalid date (YYYY-MM-DD): " + fromDate); } if (untilDate != null && !untilDate.matches("\\d{4}-\\d{2}-\\d{2}")) { - throw new DnetCollectorException("Invalid date (YYYY-MM-DD): " + untilDate); + throw new CollectorException("Invalid date (YYYY-MM-DD): " + untilDate); } final Iterator> iters = sets diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/oai/OaiIterator.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/oai/OaiIterator.java index d61f13fb5..e54bae67d 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/oai/OaiIterator.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/oai/OaiIterator.java @@ -16,7 +16,7 @@ import org.dom4j.DocumentException; import org.dom4j.Node; import org.dom4j.io.SAXReader; -import eu.dnetlib.dhp.collection.worker.DnetCollectorException; +import eu.dnetlib.dhp.collection.worker.CollectorException; import eu.dnetlib.dhp.collection.worker.utils.HttpConnector; import eu.dnetlib.dhp.collection.worker.utils.XmlCleaner; @@ -58,7 +58,7 @@ public class OaiIterator implements Iterator { this.started = true; try { this.token = firstPage(); - } catch (final DnetCollectorException e) { + } catch (final CollectorException e) { throw new RuntimeException(e); } } @@ -80,7 +80,7 @@ public class OaiIterator implements Iterator { while (queue.isEmpty() && token != null && !token.isEmpty()) { try { token = otherPages(token); - } catch (final DnetCollectorException e) { + } catch (final CollectorException e) { throw new RuntimeException(e); } } @@ -92,7 +92,7 @@ public class OaiIterator implements Iterator { public void remove() { } - private String firstPage() throws DnetCollectorException { + private String firstPage() throws CollectorException { try { String url = baseUrl + "?verb=ListRecords&metadataPrefix=" + URLEncoder.encode(mdFormat, "UTF-8"); if (set != null && !set.isEmpty()) { @@ -108,7 +108,7 @@ public class OaiIterator implements Iterator { return downloadPage(url); } catch (final UnsupportedEncodingException e) { - throw new DnetCollectorException(e); + throw new CollectorException(e); } } @@ -126,18 +126,18 @@ public class OaiIterator implements Iterator { return result.trim(); } - private String otherPages(final String resumptionToken) throws DnetCollectorException { + private String otherPages(final String resumptionToken) throws CollectorException { try { return downloadPage( baseUrl + "?verb=ListRecords&resumptionToken=" + URLEncoder.encode(resumptionToken, "UTF-8")); } catch (final UnsupportedEncodingException e) { - throw new DnetCollectorException(e); + throw new CollectorException(e); } } - private String downloadPage(final String url) throws DnetCollectorException { + private String downloadPage(final String url) throws CollectorException { final String xml = httpConnector.getInputSource(url); Document doc; @@ -151,7 +151,7 @@ public class OaiIterator implements Iterator { } catch (final DocumentException e1) { final String resumptionToken = extractResumptionToken(xml); if (resumptionToken == null) { - throw new DnetCollectorException("Error parsing cleaned document:" + cleaned, e1); + throw new CollectorException("Error parsing cleaned document:" + cleaned, e1); } return resumptionToken; } @@ -164,7 +164,7 @@ public class OaiIterator implements Iterator { log.warn("noRecordsMatch for oai call: " + url); return null; } else { - throw new DnetCollectorException(code + " - " + errorNode.getText()); + throw new CollectorException(code + " - " + errorNode.getText()); } } diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/worker/DnetCollectorException.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/worker/CollectorException.java similarity index 56% rename from dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/worker/DnetCollectorException.java rename to dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/worker/CollectorException.java index f40962c21..71d225f13 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/worker/DnetCollectorException.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/worker/CollectorException.java @@ -1,16 +1,16 @@ package eu.dnetlib.dhp.collection.worker; -public class DnetCollectorException extends Exception { +public class CollectorException extends Exception { /** */ private static final long serialVersionUID = -290723075076039757L; - public DnetCollectorException() { + public CollectorException() { super(); } - public DnetCollectorException( + public CollectorException( final String message, final Throwable cause, final boolean enableSuppression, @@ -18,15 +18,15 @@ public class DnetCollectorException extends Exception { super(message, cause, enableSuppression, writableStackTrace); } - public DnetCollectorException(final String message, final Throwable cause) { + public CollectorException(final String message, final Throwable cause) { super(message, cause); } - public DnetCollectorException(final String message) { + public CollectorException(final String message) { super(message); } - public DnetCollectorException(final Throwable cause) { + public CollectorException(final Throwable cause) { super(cause); } } diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/worker/CollectorWorker.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/worker/CollectorWorker.java new file mode 100644 index 000000000..380db641a --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/worker/CollectorWorker.java @@ -0,0 +1,93 @@ + +package eu.dnetlib.dhp.collection.worker; + +import java.io.IOException; +import java.net.URI; +import java.util.concurrent.atomic.AtomicInteger; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.io.IntWritable; +import org.apache.hadoop.io.SequenceFile; +import org.apache.hadoop.io.Text; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.fasterxml.jackson.databind.ObjectMapper; + +import eu.dnetlib.collector.worker.model.ApiDescriptor; +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.collection.plugin.CollectorPlugin; +import eu.dnetlib.dhp.collection.worker.utils.CollectorPluginFactory; + +public class CollectorWorker { + + private static final Logger log = LoggerFactory.getLogger(CollectorWorker.class); + + private final CollectorPluginFactory collectorPluginFactory; + + private final ApiDescriptor api; + + private final String hdfsuri; + + private final String hdfsPath; + + public CollectorWorker( + final CollectorPluginFactory collectorPluginFactory, + final ApiDescriptor api, + final String hdfsuri, + final String hdfsPath) { + this.collectorPluginFactory = collectorPluginFactory; + this.api = api; + this.hdfsuri = hdfsuri; + this.hdfsPath = hdfsPath; + + } + + public void collect() throws CollectorException { + try { + final CollectorPlugin plugin = collectorPluginFactory.getPluginByProtocol(api.getProtocol()); + + // ====== Init HDFS File System Object + Configuration conf = new Configuration(); + // Set FileSystem URI + conf.set("fs.defaultFS", hdfsuri); + // Because of Maven + conf.set("fs.hdfs.impl", org.apache.hadoop.hdfs.DistributedFileSystem.class.getName()); + conf.set("fs.file.impl", org.apache.hadoop.fs.LocalFileSystem.class.getName()); + + System.setProperty("hadoop.home.dir", "/"); + // Get the filesystem - HDFS + FileSystem.get(URI.create(hdfsuri), conf); + Path hdfswritepath = new Path(hdfsPath); + + log.info("Created path " + hdfswritepath.toString()); + + final AtomicInteger counter = new AtomicInteger(0); + try (SequenceFile.Writer writer = SequenceFile + .createWriter( + conf, + SequenceFile.Writer.file(hdfswritepath), + SequenceFile.Writer.keyClass(IntWritable.class), + SequenceFile.Writer.valueClass(Text.class))) { + final IntWritable key = new IntWritable(counter.get()); + final Text value = new Text(); + plugin + .collect(api) + .forEach( + content -> { + key.set(counter.getAndIncrement()); + value.set(content); + try { + writer.append(key, value); + } catch (IOException e) { + throw new RuntimeException(e); + } + }); + } + } catch (Throwable e) { + throw new CollectorException("Error on collecting ", e); + } + } +} diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/worker/CollectorWorkerApplication.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/worker/CollectorWorkerApplication.java new file mode 100644 index 000000000..5e8d0f9c2 --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/worker/CollectorWorkerApplication.java @@ -0,0 +1,55 @@ + +package eu.dnetlib.dhp.collection.worker; + +import org.apache.commons.io.IOUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.fasterxml.jackson.databind.ObjectMapper; + +import eu.dnetlib.collector.worker.model.ApiDescriptor; +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.collection.worker.utils.CollectorPluginFactory; + +/** + * DnetCollectortWorkerApplication is the main class responsible to start the Dnet Collection into HDFS. This module + * will be executed on the hadoop cluster and taking in input some parameters that tells it which is the right collector + * plugin to use and where store the data into HDFS path + * + * @author Sandro La Bruzzo + */ +public class CollectorWorkerApplication { + + private static final Logger log = LoggerFactory.getLogger(CollectorWorkerApplication.class); + + private static final CollectorPluginFactory collectorPluginFactory = new CollectorPluginFactory(); + + /** + * @param args + */ + public static void main(final String[] args) throws Exception { + + final ArgumentApplicationParser argumentParser = new ArgumentApplicationParser( + IOUtils + .toString( + CollectorWorker.class + .getResourceAsStream( + "/eu/dnetlib/dhp/collection/collector_parameter.json"))); + argumentParser.parseArgument(args); + + final String hdfsuri = argumentParser.get("namenode"); + + log.info("hdfsURI is {}", hdfsuri); + final String hdfsPath = argumentParser.get("hdfsPath"); + log.info("hdfsPath is {}" + hdfsPath); + final String apiDescriptor = argumentParser.get("apidescriptor"); + log.info("apiDescriptor is {}" + apiDescriptor); + + final ObjectMapper jsonMapper = new ObjectMapper(); + + final ApiDescriptor api = jsonMapper.readValue(apiDescriptor, ApiDescriptor.class); + + final CollectorWorker worker = new CollectorWorker(collectorPluginFactory, api, hdfsuri, hdfsPath); + worker.collect(); + } +} diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/worker/DnetCollectorWorker.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/worker/DnetCollectorWorker.java deleted file mode 100644 index e686ad518..000000000 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/worker/DnetCollectorWorker.java +++ /dev/null @@ -1,139 +0,0 @@ - -package eu.dnetlib.dhp.collection.worker; - -import java.io.IOException; -import java.net.URI; -import java.util.HashMap; -import java.util.Map; -import java.util.concurrent.atomic.AtomicInteger; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.io.IntWritable; -import org.apache.hadoop.io.SequenceFile; -import org.apache.hadoop.io.Text; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import com.fasterxml.jackson.databind.ObjectMapper; - -import eu.dnetlib.collector.worker.model.ApiDescriptor; -import eu.dnetlib.dhp.application.ArgumentApplicationParser; -import eu.dnetlib.dhp.collection.plugin.CollectorPlugin; -import eu.dnetlib.dhp.collection.worker.utils.CollectorPluginFactory; -import eu.dnetlib.message.Message; -import eu.dnetlib.message.MessageManager; -import eu.dnetlib.message.MessageType; - -public class DnetCollectorWorker { - - private static final Logger log = LoggerFactory.getLogger(DnetCollectorWorker.class); - - private final CollectorPluginFactory collectorPluginFactory; - - private final ArgumentApplicationParser argumentParser; - - private final MessageManager manager; - - public DnetCollectorWorker( - final CollectorPluginFactory collectorPluginFactory, - final ArgumentApplicationParser argumentParser, - final MessageManager manager) - throws DnetCollectorException { - this.collectorPluginFactory = collectorPluginFactory; - this.argumentParser = argumentParser; - this.manager = manager; - } - - public void collect() throws DnetCollectorException { - try { - final ObjectMapper jsonMapper = new ObjectMapper(); - final ApiDescriptor api = jsonMapper.readValue(argumentParser.get("apidescriptor"), ApiDescriptor.class); - - final CollectorPlugin plugin = collectorPluginFactory.getPluginByProtocol(api.getProtocol()); - - final String hdfsuri = argumentParser.get("namenode"); - - // ====== Init HDFS File System Object - Configuration conf = new Configuration(); - // Set FileSystem URI - conf.set("fs.defaultFS", hdfsuri); - // Because of Maven - conf.set("fs.hdfs.impl", org.apache.hadoop.hdfs.DistributedFileSystem.class.getName()); - conf.set("fs.file.impl", org.apache.hadoop.fs.LocalFileSystem.class.getName()); - - System.setProperty("HADOOP_USER_NAME", argumentParser.get("userHDFS")); - System.setProperty("hadoop.home.dir", "/"); - // Get the filesystem - HDFS - FileSystem.get(URI.create(hdfsuri), conf); - Path hdfswritepath = new Path(argumentParser.get("hdfsPath")); - - log.info("Created path " + hdfswritepath.toString()); - - final Map ongoingMap = new HashMap<>(); - final Map reportMap = new HashMap<>(); - final AtomicInteger counter = new AtomicInteger(0); - try (SequenceFile.Writer writer = SequenceFile - .createWriter( - conf, - SequenceFile.Writer.file(hdfswritepath), - SequenceFile.Writer.keyClass(IntWritable.class), - SequenceFile.Writer.valueClass(Text.class))) { - final IntWritable key = new IntWritable(counter.get()); - final Text value = new Text(); - plugin - .collect(api) - .forEach( - content -> { - key.set(counter.getAndIncrement()); - value.set(content); - if (counter.get() % 10 == 0) { - try { - ongoingMap.put("ongoing", "" + counter.get()); - log - .debug( - "Sending message: " - + manager - .sendMessage( - new Message( - argumentParser.get("workflowId"), - "Collection", - MessageType.ONGOING, - ongoingMap), - argumentParser.get("rabbitOngoingQueue"), - true, - false)); - } catch (Exception e) { - log.error("Error on sending message ", e); - } - } - try { - writer.append(key, value); - } catch (IOException e) { - throw new RuntimeException(e); - } - }); - } - ongoingMap.put("ongoing", "" + counter.get()); - manager - .sendMessage( - new Message( - argumentParser.get("workflowId"), "Collection", MessageType.ONGOING, ongoingMap), - argumentParser.get("rabbitOngoingQueue"), - true, - false); - reportMap.put("collected", "" + counter.get()); - manager - .sendMessage( - new Message( - argumentParser.get("workflowId"), "Collection", MessageType.REPORT, reportMap), - argumentParser.get("rabbitOngoingQueue"), - true, - false); - manager.close(); - } catch (Throwable e) { - throw new DnetCollectorException("Error on collecting ", e); - } - } -} diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/worker/DnetCollectorWorkerApplication.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/worker/DnetCollectorWorkerApplication.java deleted file mode 100644 index da30e8793..000000000 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/worker/DnetCollectorWorkerApplication.java +++ /dev/null @@ -1,49 +0,0 @@ - -package eu.dnetlib.dhp.collection.worker; - -import org.apache.commons.io.IOUtils; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import eu.dnetlib.dhp.application.ArgumentApplicationParser; -import eu.dnetlib.dhp.collection.worker.utils.CollectorPluginFactory; -import eu.dnetlib.message.MessageManager; - -/** - * DnetCollectortWorkerApplication is the main class responsible to start the Dnet Collection into HDFS. This module - * will be executed on the hadoop cluster and taking in input some parameters that tells it which is the right collector - * plugin to use and where store the data into HDFS path - * - * @author Sandro La Bruzzo - */ -public class DnetCollectorWorkerApplication { - - private static final Logger log = LoggerFactory.getLogger(DnetCollectorWorkerApplication.class); - - private static final CollectorPluginFactory collectorPluginFactory = new CollectorPluginFactory(); - - private static ArgumentApplicationParser argumentParser; - - /** @param args */ - public static void main(final String[] args) throws Exception { - - argumentParser = new ArgumentApplicationParser( - IOUtils - .toString( - DnetCollectorWorker.class - .getResourceAsStream( - "/eu/dnetlib/collector/worker/collector_parameter.json"))); - argumentParser.parseArgument(args); - log.info("hdfsPath =" + argumentParser.get("hdfsPath")); - log.info("json = " + argumentParser.get("apidescriptor")); - final MessageManager manager = new MessageManager( - argumentParser.get("rabbitHost"), - argumentParser.get("rabbitUser"), - argumentParser.get("rabbitPassword"), - false, - false, - null); - final DnetCollectorWorker worker = new DnetCollectorWorker(collectorPluginFactory, argumentParser, manager); - worker.collect(); - } -} diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/worker/utils/CollectorPluginFactory.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/worker/utils/CollectorPluginFactory.java index 7a0028e79..6b070b191 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/worker/utils/CollectorPluginFactory.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/worker/utils/CollectorPluginFactory.java @@ -3,18 +3,18 @@ package eu.dnetlib.dhp.collection.worker.utils; import eu.dnetlib.dhp.collection.plugin.CollectorPlugin; import eu.dnetlib.dhp.collection.plugin.oai.OaiCollectorPlugin; -import eu.dnetlib.dhp.collection.worker.DnetCollectorException; +import eu.dnetlib.dhp.collection.worker.CollectorException; public class CollectorPluginFactory { - public CollectorPlugin getPluginByProtocol(final String protocol) throws DnetCollectorException { + public CollectorPlugin getPluginByProtocol(final String protocol) throws CollectorException { if (protocol == null) - throw new DnetCollectorException("protocol cannot be null"); + throw new CollectorException("protocol cannot be null"); switch (protocol.toLowerCase().trim()) { case "oai": return new OaiCollectorPlugin(); default: - throw new DnetCollectorException("UNknown protocol"); + throw new CollectorException("UNknown protocol"); } } } diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/worker/utils/HttpConnector.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/worker/utils/HttpConnector.java index 5d6108fad..ff3c18aba 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/worker/utils/HttpConnector.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/worker/utils/HttpConnector.java @@ -19,7 +19,7 @@ import org.apache.commons.lang.math.NumberUtils; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import eu.dnetlib.dhp.collection.worker.DnetCollectorException; +import eu.dnetlib.dhp.collection.worker.CollectorException; public class HttpConnector { @@ -42,9 +42,9 @@ public class HttpConnector { * * @param requestUrl the URL * @return the content of the downloaded resource - * @throws DnetCollectorException when retrying more than maxNumberOfRetry times + * @throws CollectorException when retrying more than maxNumberOfRetry times */ - public String getInputSource(final String requestUrl) throws DnetCollectorException { + public String getInputSource(final String requestUrl) throws CollectorException { return attemptDownlaodAsString(requestUrl, 1, new CollectorPluginErrorLogList()); } @@ -53,15 +53,15 @@ public class HttpConnector { * * @param requestUrl the URL * @return the content of the downloaded resource as InputStream - * @throws DnetCollectorException when retrying more than maxNumberOfRetry times + * @throws CollectorException when retrying more than maxNumberOfRetry times */ - public InputStream getInputSourceAsStream(final String requestUrl) throws DnetCollectorException { + public InputStream getInputSourceAsStream(final String requestUrl) throws CollectorException { return attemptDownload(requestUrl, 1, new CollectorPluginErrorLogList()); } private String attemptDownlaodAsString( final String requestUrl, final int retryNumber, final CollectorPluginErrorLogList errorList) - throws DnetCollectorException { + throws CollectorException { try { final InputStream s = attemptDownload(requestUrl, 1, new CollectorPluginErrorLogList()); try { @@ -75,16 +75,16 @@ public class HttpConnector { IOUtils.closeQuietly(s); } } catch (final InterruptedException e) { - throw new DnetCollectorException(e); + throw new CollectorException(e); } } private InputStream attemptDownload( final String requestUrl, final int retryNumber, final CollectorPluginErrorLogList errorList) - throws DnetCollectorException { + throws CollectorException { if (retryNumber > maxNumberOfRetry) { - throw new DnetCollectorException("Max number of retries exceeded. Cause: \n " + errorList); + throw new CollectorException("Max number of retries exceeded. Cause: \n " + errorList); } log.debug("Downloading " + requestUrl + " - try: " + retryNumber); @@ -144,7 +144,7 @@ public class HttpConnector { return attemptDownload(requestUrl, retryNumber + 1, errorList); } } catch (final InterruptedException e) { - throw new DnetCollectorException(e); + throw new CollectorException(e); } } @@ -173,13 +173,13 @@ public class HttpConnector { } private String obtainNewLocation(final Map> headerMap) - throws DnetCollectorException { + throws CollectorException { for (final String key : headerMap.keySet()) { if (key != null && key.toLowerCase().equals("location") && headerMap.get(key).size() > 0) { return headerMap.get(key).get(0); } } - throw new DnetCollectorException( + throw new CollectorException( "The requested url has been MOVED, but 'location' param is MISSING"); } diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/transformation/DnetTransformationException.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/transformation/DnetTransformationException.java new file mode 100644 index 000000000..45bd844e2 --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/transformation/DnetTransformationException.java @@ -0,0 +1,29 @@ + +package eu.dnetlib.dhp.transformation; + +public class DnetTransformationException extends Exception { + + public DnetTransformationException() { + super(); + } + + public DnetTransformationException( + final String message, + final Throwable cause, + final boolean enableSuppression, + final boolean writableStackTrace) { + super(message, cause, enableSuppression, writableStackTrace); + } + + public DnetTransformationException(final String message, final Throwable cause) { + super(message, cause); + } + + public DnetTransformationException(final String message) { + super(message); + } + + public DnetTransformationException(final Throwable cause) { + super(cause); + } +} diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/transformation/TransformSparkJobNode.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/transformation/TransformSparkJobNode.java index 8737d36ef..c6ed5a1e3 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/transformation/TransformSparkJobNode.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/transformation/TransformSparkJobNode.java @@ -9,9 +9,10 @@ import java.util.Map; import java.util.Objects; import java.util.Optional; -import org.apache.commons.cli.*; import org.apache.commons.io.IOUtils; +import org.apache.commons.lang3.StringUtils; import org.apache.spark.SparkConf; +import org.apache.spark.api.java.function.MapFunction; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Encoder; import org.apache.spark.sql.Encoders; @@ -24,12 +25,15 @@ import org.dom4j.io.SAXReader; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import eu.dnetlib.dhp.aggregation.common.AggregationCounter; import eu.dnetlib.dhp.application.ArgumentApplicationParser; -import eu.dnetlib.dhp.collection.GenerateNativeStoreSparkJob; +import eu.dnetlib.dhp.common.vocabulary.VocabularyGroup; import eu.dnetlib.dhp.model.mdstore.MetadataRecord; -import eu.dnetlib.dhp.transformation.vocabulary.Vocabulary; import eu.dnetlib.dhp.transformation.vocabulary.VocabularyHelper; +import eu.dnetlib.dhp.transformation.xslt.XSLTTransformationFunction; import eu.dnetlib.dhp.utils.DHPUtils; +import eu.dnetlib.dhp.utils.ISLookupClientFactory; +import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService; import eu.dnetlib.message.Message; import eu.dnetlib.message.MessageManager; import eu.dnetlib.message.MessageType; @@ -55,67 +59,38 @@ public class TransformSparkJobNode { .orElse(Boolean.TRUE); log.info("isSparkSessionManaged: {}", isSparkSessionManaged); - final String inputPath = parser.get("input"); - final String outputPath = parser.get("output"); - final String workflowId = parser.get("workflowId"); - final String trasformationRule = extractXSLTFromTR( - Objects.requireNonNull(DHPUtils.decompressString(parser.get("transformationRule")))); + final String inputPath = parser.get("mdstoreInputPath"); + final String outputPath = parser.get("mdstoreOutputPath"); + // TODO this variable will be used after implementing Messaging with DNet Aggregator - final String rabbitUser = parser.get("rabbitUser"); - final String rabbitPassword = parser.get("rabbitPassword"); - final String rabbitHost = parser.get("rabbitHost"); - final String rabbitReportQueue = parser.get("rabbitReportQueue"); - final long dateOfCollection = new Long(parser.get("dateOfCollection")); - final boolean test = parser.get("isTest") == null ? false : Boolean.valueOf(parser.get("isTest")); + final String isLookupUrl = parser.get("isLookupUrl"); + log.info(String.format("isLookupUrl: %s", isLookupUrl)); + + final ISLookUpService isLookupService = ISLookupClientFactory.getLookUpService(isLookupUrl); SparkConf conf = new SparkConf(); runWithSparkSession( conf, isSparkSessionManaged, - spark -> { - final Encoder encoder = Encoders.bean(MetadataRecord.class); - final Dataset mdstoreInput = spark.read().format("parquet").load(inputPath).as(encoder); - final LongAccumulator totalItems = spark.sparkContext().longAccumulator("TotalItems"); - final LongAccumulator errorItems = spark.sparkContext().longAccumulator("errorItems"); - final LongAccumulator transformedItems = spark.sparkContext().longAccumulator("transformedItems"); - final Map vocabularies = new HashMap<>(); - vocabularies.put("dnet:languages", VocabularyHelper.getVocabularyFromAPI("dnet:languages")); - final TransformFunction transformFunction = new TransformFunction( - totalItems, - errorItems, - transformedItems, - trasformationRule, - dateOfCollection, - vocabularies); - mdstoreInput.map(transformFunction, encoder).write().format("parquet").save(outputPath); - if (rabbitHost != null) { - System.out.println("SEND FINAL REPORT"); - final Map reportMap = new HashMap<>(); - reportMap.put("inputItem", "" + totalItems.value()); - reportMap.put("invalidRecords", "" + errorItems.value()); - reportMap.put("mdStoreSize", "" + transformedItems.value()); - System.out.println(new Message(workflowId, "Transform", MessageType.REPORT, reportMap)); - if (!test) { - final MessageManager manager = new MessageManager(rabbitHost, rabbitUser, rabbitPassword, false, - false, - null); - manager - .sendMessage( - new Message(workflowId, "Transform", MessageType.REPORT, reportMap), - rabbitReportQueue, - true, - false); - manager.close(); - } - } - }); - + spark -> transformRecords(parser.getObjectMap(), isLookupService, spark, inputPath, outputPath)); } - private static String extractXSLTFromTR(final String tr) throws DocumentException { - SAXReader reader = new SAXReader(); - Document document = reader.read(new ByteArrayInputStream(tr.getBytes())); - Node node = document.selectSingleNode("//CODE/*[local-name()='stylesheet']"); - return node.asXML(); + public static void transformRecords(final Map args, final ISLookUpService isLookUpService, + final SparkSession spark, final String inputPath, final String outputPath) throws DnetTransformationException { + + final LongAccumulator totalItems = spark.sparkContext().longAccumulator("TotalItems"); + final LongAccumulator errorItems = spark.sparkContext().longAccumulator("errorItems"); + final LongAccumulator transformedItems = spark.sparkContext().longAccumulator("transformedItems"); + final AggregationCounter ct = new AggregationCounter(totalItems, errorItems, transformedItems); + final Encoder encoder = Encoders.bean(MetadataRecord.class); + final Dataset mdstoreInput = spark.read().format("parquet").load(inputPath).as(encoder); + final MapFunction XSLTTransformationFunction = TransformationFactory + .getTransformationPlugin(args, ct, isLookUpService); + mdstoreInput.map(XSLTTransformationFunction, encoder).write().save(outputPath); + + log.info("Transformed item " + ct.getProcessedItems().count()); + log.info("Total item " + ct.getTotalItems().count()); + log.info("Transformation Error item " + ct.getErrorItems().count()); } + } diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/transformation/TransformationFactory.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/transformation/TransformationFactory.java new file mode 100644 index 000000000..58292139a --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/transformation/TransformationFactory.java @@ -0,0 +1,69 @@ + +package eu.dnetlib.dhp.transformation; + +import java.util.List; +import java.util.Map; + +import org.apache.commons.lang3.StringUtils; +import org.apache.spark.api.java.function.MapFunction; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import eu.dnetlib.dhp.aggregation.common.AggregationCounter; +import eu.dnetlib.dhp.common.vocabulary.VocabularyGroup; +import eu.dnetlib.dhp.model.mdstore.MetadataRecord; +import eu.dnetlib.dhp.transformation.xslt.XSLTTransformationFunction; +import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService; + +public class TransformationFactory { + + private static final Logger log = LoggerFactory.getLogger(TransformationFactory.class); + public static final String TRULE_XQUERY = "for $x in collection('/db/DRIVER/TransformationRuleDSResources/TransformationRuleDSResourceType') where $x//TITLE = \"%s\" return $x//CODE/text()"; + + public static MapFunction getTransformationPlugin( + final Map jobArgument, final AggregationCounter counters, final ISLookUpService isLookupService) + throws DnetTransformationException { + + try { + final String transformationPlugin = jobArgument.get("transformationPlugin"); + + log.info("Transformation plugin required " + transformationPlugin); + switch (transformationPlugin) { + case "XSLT_TRANSFORM": { + final String transformationRuleName = jobArgument.get("transformationRuleTitle"); + if (StringUtils.isBlank(transformationRuleName)) + throw new DnetTransformationException("Missing Parameter transformationRule"); + final VocabularyGroup vocabularies = VocabularyGroup.loadVocsFromIS(isLookupService); + + final String transformationRule = queryTransformationRuleFromIS( + transformationRuleName, isLookupService); + + final long dateOfTransformation = new Long(jobArgument.get("dateOfTransformation")); + return new XSLTTransformationFunction(counters, transformationRule, dateOfTransformation, + vocabularies); + + } + default: + throw new DnetTransformationException( + "transformation plugin does not exists for " + transformationPlugin); + + } + + } catch (Throwable e) { + throw new DnetTransformationException(e); + } + } + + private static String queryTransformationRuleFromIS(final String transformationRuleName, + final ISLookUpService isLookUpService) throws Exception { + final String query = String.format(TRULE_XQUERY, transformationRuleName); + log.info("asking query to IS: " + query); + List result = isLookUpService.quickSearchProfile(query); + + if (result == null || result.isEmpty()) + throw new DnetTransformationException( + "Unable to find transformation rule with name: " + transformationRuleName); + return result.get(0); + } + +} diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/transformation/functions/Cleaner.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/transformation/xslt/Cleaner.java similarity index 61% rename from dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/transformation/functions/Cleaner.java rename to dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/transformation/xslt/Cleaner.java index 7f9b6646c..7b0fdd484 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/transformation/functions/Cleaner.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/transformation/xslt/Cleaner.java @@ -1,19 +1,16 @@ -package eu.dnetlib.dhp.transformation.functions; +package eu.dnetlib.dhp.transformation.xslt; -import java.util.Map; -import java.util.Optional; - -import eu.dnetlib.dhp.transformation.vocabulary.Term; -import eu.dnetlib.dhp.transformation.vocabulary.Vocabulary; +import eu.dnetlib.dhp.common.vocabulary.VocabularyGroup; +import eu.dnetlib.dhp.schema.oaf.Qualifier; import net.sf.saxon.s9api.*; import scala.Serializable; public class Cleaner implements ExtensionFunction, Serializable { - private final Map vocabularies; + private final VocabularyGroup vocabularies; - public Cleaner(Map vocabularies) { + public Cleaner(final VocabularyGroup vocabularies) { this.vocabularies = vocabularies; } @@ -39,14 +36,9 @@ public class Cleaner implements ExtensionFunction, Serializable { public XdmValue call(XdmValue[] xdmValues) throws SaxonApiException { final String currentValue = xdmValues[0].itemAt(0).getStringValue(); final String vocabularyName = xdmValues[1].itemAt(0).getStringValue(); - Optional cleanedValue = vocabularies - .get(vocabularyName) - .getTerms() - .stream() - .filter(it -> it.getNativeName().equalsIgnoreCase(currentValue)) - .findAny(); + Qualifier cleanedValue = vocabularies.getSynonymAsQualifier(vocabularyName, currentValue); return new XdmAtomicValue( - cleanedValue.isPresent() ? cleanedValue.get().getCode() : currentValue); + cleanedValue != null ? cleanedValue.getClassid() : currentValue); } } diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/transformation/TransformFunction.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/transformation/xslt/XSLTTransformationFunction.java similarity index 67% rename from dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/transformation/TransformFunction.java rename to dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/transformation/xslt/XSLTTransformationFunction.java index f4bf78e18..d8707cd76 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/transformation/TransformFunction.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/transformation/xslt/XSLTTransformationFunction.java @@ -1,41 +1,35 @@ -package eu.dnetlib.dhp.transformation; +package eu.dnetlib.dhp.transformation.xslt; import java.io.ByteArrayInputStream; import java.io.StringWriter; -import java.util.Map; import javax.xml.transform.stream.StreamSource; import org.apache.spark.api.java.function.MapFunction; -import org.apache.spark.util.LongAccumulator; +import eu.dnetlib.dhp.aggregation.common.AggregationCounter; +import eu.dnetlib.dhp.common.vocabulary.VocabularyGroup; import eu.dnetlib.dhp.model.mdstore.MetadataRecord; -import eu.dnetlib.dhp.transformation.functions.Cleaner; -import eu.dnetlib.dhp.transformation.vocabulary.Vocabulary; import net.sf.saxon.s9api.*; -public class TransformFunction implements MapFunction { +public class XSLTTransformationFunction implements MapFunction { + + private final AggregationCounter aggregationCounter; - private final LongAccumulator totalItems; - private final LongAccumulator errorItems; - private final LongAccumulator transformedItems; private final String transformationRule; + private final Cleaner cleanFunction; private final long dateOfTransformation; - public TransformFunction( - LongAccumulator totalItems, - LongAccumulator errorItems, - LongAccumulator transformedItems, + public XSLTTransformationFunction( + final AggregationCounter aggregationCounter, final String transformationRule, long dateOfTransformation, - final Map vocabularies) + final VocabularyGroup vocabularies) throws Exception { - this.totalItems = totalItems; - this.errorItems = errorItems; - this.transformedItems = transformedItems; + this.aggregationCounter = aggregationCounter; this.transformationRule = transformationRule; this.dateOfTransformation = dateOfTransformation; cleanFunction = new Cleaner(vocabularies); @@ -43,7 +37,7 @@ public class TransformFunction implements MapFunction + + jobTracker + yarnRM + + + nameNode + hdfs://nameservice1 + + + oozie.use.system.libpath + true + + + oozie.action.sharelib.for.spark + spark2 + + \ No newline at end of file diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/collection/oozie_app/workflow.xml b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/collection/oozie_app/workflow.xml index 3e7f68401..38cd83da7 100644 --- a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/collection/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/collection/oozie_app/workflow.xml @@ -1,10 +1,5 @@ - - sequenceFilePath - the path to store the sequence file of the native metadata collected - - mdStorePath the path of the native mdstore @@ -39,72 +34,52 @@ The identifier of the workflow + + ${jobTracker} + ${nameNode} + - + Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}] - - - - - - - - - - + - ${jobTracker} - ${nameNode} - eu.dnetlib.dhp.collection.worker.DnetCollectorWorker - -p${sequenceFilePath} - -a${apiDescription} - -n${nameNode} - -rh${rmq_host} - -ru${rmq_user} - -rp${rmq_pwd} - -rr${rmq_report} - -ro${rmq_ongoing} - -usandro.labruzzo - -w${workflowId} + eu.dnetlib.dhp.collection.worker.CollectorWorkerApplication + --hdfsPath${workingDir}/sequenceFile_${mdstoreVersion} + --apidescriptor${apiDescription} + --namenode${nameNode} + - ${jobTracker} - ${nameNode} yarn cluster - GenerateNativeStoreSparkJob + Generate Native MetadataStore eu.dnetlib.dhp.collection.GenerateNativeStoreSparkJob - dhp-aggregations-1.0.0-SNAPSHOT.jar - --num-executors 50 --conf spark.yarn.jars="hdfs://hadoop-rm1.garr-pa1.d4science.org:8020/user/oozie/share/lib/lib_20180405103059/spark2" - --encoding ${metadataEncoding} - --dateOfCollection ${timestamp} - --provenance ${dataSourceInfo} + dhp-aggregation-${projectVersion}.jar + + --executor-memory=${sparkExecutorMemory} + --executor-cores=${sparkExecutorCores} + --driver-memory=${sparkDriverMemory} + --conf spark.extraListeners=${spark2ExtraListeners} + --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} + --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} + --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + + --encoding${metadataEncoding} + --dateOfCollection${timestamp} + --provenance${dataSourceInfo} --xpath${identifierPath} - --input${sequenceFilePath} + --input${workingDir}/sequenceFile --output${mdStorePath} - -rh${rmq_host} - -ru${rmq_user} - -rp${rmq_pwd} - -rr${rmq_report} - -ro${rmq_ongoing} -w${workflowId} - - - - - - - - diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/collector/worker/collector_parameter.json b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/collector/worker/collector_parameter.json deleted file mode 100644 index c247d15e4..000000000 --- a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/collector/worker/collector_parameter.json +++ /dev/null @@ -1,12 +0,0 @@ -[ - {"paramName":"p", "paramLongName":"hdfsPath", "paramDescription": "the path where storing the sequential file", "paramRequired": true}, - {"paramName":"a", "paramLongName":"apidescriptor", "paramDescription": "the JSON encoding of the API Descriptor", "paramRequired": true}, - {"paramName":"n", "paramLongName":"namenode", "paramDescription": "the Name Node URI", "paramRequired": true}, - {"paramName":"u", "paramLongName":"userHDFS", "paramDescription": "the user wich create the hdfs seq file", "paramRequired": true}, - {"paramName":"ru", "paramLongName":"rabbitUser", "paramDescription": "the user to connect with RabbitMq for messaging", "paramRequired": true}, - {"paramName":"rp", "paramLongName":"rabbitPassword", "paramDescription": "the password to connect with RabbitMq for messaging", "paramRequired": true}, - {"paramName":"rh", "paramLongName":"rabbitHost", "paramDescription": "the host of the RabbitMq server", "paramRequired": true}, - {"paramName":"ro", "paramLongName":"rabbitOngoingQueue", "paramDescription": "the name of the ongoing queue", "paramRequired": true}, - {"paramName":"rr", "paramLongName":"rabbitReportQueue", "paramDescription": "the name of the report queue", "paramRequired": true}, - {"paramName":"w", "paramLongName":"workflowId", "paramDescription": "the identifier of the dnet Workflow", "paramRequired": true} -] \ No newline at end of file diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/transformation/oozie_app/config-default.xml b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/transformation/oozie_app/config-default.xml new file mode 100644 index 000000000..2e0ed9aee --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/transformation/oozie_app/config-default.xml @@ -0,0 +1,18 @@ + + + jobTracker + yarnRM + + + nameNode + hdfs://nameservice1 + + + oozie.use.system.libpath + true + + + oozie.action.sharelib.for.spark + spark2 + + \ No newline at end of file diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/transformation/oozie_app/workflow.xml b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/transformation/oozie_app/workflow.xml index 4b1e3d84b..b36bc3766 100644 --- a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/transformation/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/transformation/oozie_app/workflow.xml @@ -2,7 +2,7 @@ mdstoreInputPath - the path of the input MDStore + the path of the native MDStore @@ -11,66 +11,57 @@ - transformationRule + transformationRuleTitle The transformation Rule to apply - timestamp - The timestamp of the collection date + transformationPlugin + The transformation Plugin - workflowId - The identifier of the workflow + dateOfTransformation + The timestamp of the transformation date + + - + Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}] - - - - - - - - + - ${jobTracker} - ${nameNode} yarn cluster - MDBuilder + Transform MetadataStore eu.dnetlib.dhp.transformation.TransformSparkJobNode - dhp-aggregations-1.0.0-SNAPSHOT.jar - --num-executors 50 --conf spark.yarn.jars="hdfs://hadoop-rm1.garr-pa1.d4science.org:8020/user/oozie/share/lib/lib_20180405103059/spark2" - --dateOfCollection ${timestamp} - -mt yarn - --input${mdstoreInputPath} - --output${mdstoreOutputPath} - -w${workflowId} - -tr${transformationRule} - -ru${rmq_user} - -rp${rmq_pwd} - -rh${rmq_host} - -ro${rmq_ongoing} - -rr${rmq_report} + dhp-aggregations-${projectVersion}.jar + + --executor-memory=${sparkExecutorMemory} + --executor-cores=${sparkExecutorCores} + --driver-memory=${sparkDriverMemory} + --conf spark.extraListeners=${spark2ExtraListeners} + --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} + --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} + --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + + --mdstoreInputPath${mdstoreInputPath} + --mdstoreOutputPath${mdstoreOutputPath} + --dateOfTransformation${dateOfTransformation} + --transformationPlugin${transformationPlugin} + --transformationRuleTitle${transformationRuleTitle} + + - - - - - - - - + + \ No newline at end of file diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/transformation/transformation_input_parameters.json b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/transformation/transformation_input_parameters.json index 4bb5fd56a..cbd2f25ab 100644 --- a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/transformation/transformation_input_parameters.json +++ b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/transformation/transformation_input_parameters.json @@ -7,68 +7,42 @@ }, { "paramName": "d", - "paramLongName": "dateOfCollection", + "paramLongName": "dateOfTransformation", "paramDescription": "the date when the record has been stored", "paramRequired": true }, { "paramName": "i", - "paramLongName": "input", + "paramLongName": "mdstoreInputPath", "paramDescription": "the path of the sequencial file to read", "paramRequired": true }, { "paramName": "o", - "paramLongName": "output", + "paramLongName": "mdstoreOutputPath", "paramDescription": "the path of the result DataFrame on HDFS", "paramRequired": true }, - { - "paramName": "w", - "paramLongName": "workflowId", - "paramDescription": "the identifier of the dnet Workflow", - "paramRequired": true - }, { "paramName": "tr", - "paramLongName": "transformationRule", + "paramLongName": "transformationRuleTitle", "paramDescription": "the transformation Rule to apply to the input MDStore", "paramRequired": true }, + { - "paramName": "ru", - "paramLongName": "rabbitUser", - "paramDescription": "the user to connect with RabbitMq for messaging", + "paramName": "i", + "paramLongName": "isLookupUrl", + "paramDescription": "the Information System Service LookUp URL", "paramRequired": true }, + + + { - "paramName": "rp", - "paramLongName": "rabbitPassword", - "paramDescription": "the password to connect with RabbitMq for messaging", + "paramName": "tp", + "paramLongName": "transformationPlugin", + "paramDescription": "the transformation plugin to apply", "paramRequired": true - }, - { - "paramName": "rh", - "paramLongName": "rabbitHost", - "paramDescription": "the host of the RabbitMq server", - "paramRequired": true - }, - { - "paramName": "ro", - "paramLongName": "rabbitOngoingQueue", - "paramDescription": "the name of the ongoing queue", - "paramRequired": true - }, - { - "paramName": "rr", - "paramLongName": "rabbitReportQueue", - "paramDescription": "the name of the report queue", - "paramRequired": true - }, - { - "paramName": "t", - "paramLongName": "isTest", - "paramDescription": "the name of the report queue", - "paramRequired": false } ] \ No newline at end of file diff --git a/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/actionmanager/project/EXCELParserTest.java b/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/actionmanager/project/EXCELParserTest.java index c1142ad9c..5c37e9ec3 100644 --- a/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/actionmanager/project/EXCELParserTest.java +++ b/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/actionmanager/project/EXCELParserTest.java @@ -6,16 +6,15 @@ import java.nio.file.Files; import java.nio.file.Path; import java.util.List; -import eu.dnetlib.dhp.collection.worker.DnetCollectorException; -import eu.dnetlib.dhp.collection.worker.utils.HttpConnector; import org.apache.poi.openxml4j.exceptions.InvalidFormatException; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Test; - import eu.dnetlib.dhp.actionmanager.project.utils.EXCELParser; +import eu.dnetlib.dhp.collection.worker.CollectorException; +import eu.dnetlib.dhp.collection.worker.utils.HttpConnector; @Disabled public class EXCELParserTest { @@ -31,7 +30,7 @@ public class EXCELParserTest { } @Test - public void test1() throws DnetCollectorException, IOException, InvalidFormatException, ClassNotFoundException, + public void test1() throws CollectorException, IOException, InvalidFormatException, ClassNotFoundException, IllegalAccessException, InstantiationException { EXCELParser excelParser = new EXCELParser(); diff --git a/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/actionmanager/project/httpconnector/HttpConnectorTest.java b/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/actionmanager/project/httpconnector/HttpConnectorTest.java index 3b9d1c3ab..f5ef280a0 100644 --- a/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/actionmanager/project/httpconnector/HttpConnectorTest.java +++ b/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/actionmanager/project/httpconnector/HttpConnectorTest.java @@ -1,8 +1,6 @@ package eu.dnetlib.dhp.actionmanager.project.httpconnector; -import eu.dnetlib.dhp.collection.worker.DnetCollectorException; -import eu.dnetlib.dhp.collection.worker.utils.HttpConnector; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.http.conn.ssl.SSLConnectionSocketFactory; @@ -11,6 +9,9 @@ import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Test; +import eu.dnetlib.dhp.collection.worker.CollectorException; +import eu.dnetlib.dhp.collection.worker.utils.HttpConnector; + @Disabled public class HttpConnectorTest { @@ -31,12 +32,12 @@ public class HttpConnectorTest { @Test - public void testGetInputSource() throws DnetCollectorException { + public void testGetInputSource() throws CollectorException { System.out.println(connector.getInputSource(URL)); } @Test - public void testGoodServers() throws DnetCollectorException { + public void testGoodServers() throws CollectorException { System.out.println(connector.getInputSource(URL_GOODSNI_SERVER)); } diff --git a/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/collector/worker/DnetCollectorWorkerApplicationTests.java b/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/collector/worker/DnetCollectorWorkerApplicationTests.java index c745219fe..fc19f2064 100644 --- a/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/collector/worker/DnetCollectorWorkerApplicationTests.java +++ b/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/collector/worker/DnetCollectorWorkerApplicationTests.java @@ -5,17 +5,19 @@ import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.mockito.Mockito.*; import java.io.File; +import java.nio.file.Path; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; import com.fasterxml.jackson.databind.ObjectMapper; import eu.dnetlib.collector.worker.model.ApiDescriptor; import eu.dnetlib.dhp.application.ArgumentApplicationParser; -import eu.dnetlib.dhp.collection.worker.DnetCollectorWorker; +import eu.dnetlib.dhp.collection.worker.CollectorWorker; import eu.dnetlib.dhp.collection.worker.utils.CollectorPluginFactory; import eu.dnetlib.message.Message; import eu.dnetlib.message.MessageManager; @@ -23,43 +25,6 @@ import eu.dnetlib.message.MessageManager; @Disabled public class DnetCollectorWorkerApplicationTests { - private final ArgumentApplicationParser argumentParser = mock(ArgumentApplicationParser.class); - private final MessageManager messageManager = mock(MessageManager.class); - - private DnetCollectorWorker worker; - - @BeforeEach - public void setup() throws Exception { - ObjectMapper mapper = new ObjectMapper(); - final String apiJson = mapper.writeValueAsString(getApi()); - when(argumentParser.get("apidescriptor")).thenReturn(apiJson); - when(argumentParser.get("namenode")).thenReturn("file://tmp/test.seq"); - when(argumentParser.get("hdfsPath")).thenReturn("/tmp/file.seq"); - when(argumentParser.get("userHDFS")).thenReturn("sandro"); - when(argumentParser.get("workflowId")).thenReturn("sandro"); - when(argumentParser.get("rabbitOngoingQueue")).thenReturn("sandro"); - - when(messageManager.sendMessage(any(Message.class), anyString(), anyBoolean(), anyBoolean())) - .thenAnswer( - a -> { - System.out.println("sent message: " + a.getArguments()[0]); - return true; - }); - when(messageManager.sendMessage(any(Message.class), anyString())) - .thenAnswer( - a -> { - System.out.println("Called"); - return true; - }); - worker = new DnetCollectorWorker(new CollectorPluginFactory(), argumentParser, messageManager); - } - - @AfterEach - public void dropDown() { - File f = new File("/tmp/file.seq"); - f.delete(); - } - @Test public void testFindPlugin() throws Exception { final CollectorPluginFactory collectorPluginEnumerator = new CollectorPluginFactory(); @@ -79,8 +44,14 @@ public class DnetCollectorWorkerApplicationTests { } @Test - public void testFeeding() throws Exception { + public void testFeeding(@TempDir Path testDir) throws Exception { + + System.out.println(testDir.toString()); + CollectorWorker worker = new CollectorWorker(new CollectorPluginFactory(), getApi(), + "file://" + testDir.toString() + "/file.seq", testDir.toString() + "/file.seq"); worker.collect(); + + // TODO create ASSERT HERE } private ApiDescriptor getApi() { diff --git a/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/transformation/TransformationJobTest.java b/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/transformation/TransformationJobTest.java index 98c8cf66c..6a80e01e2 100644 --- a/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/transformation/TransformationJobTest.java +++ b/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/transformation/TransformationJobTest.java @@ -1,45 +1,66 @@ package eu.dnetlib.dhp.transformation; +import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.mockito.Mockito.lenient; +import java.io.IOException; import java.io.StringWriter; import java.nio.file.Files; import java.nio.file.Path; -import java.util.HashMap; -import java.util.Map; +import java.util.*; +import java.util.stream.Collectors; +import java.util.stream.Stream; import javax.xml.transform.stream.StreamSource; import org.apache.commons.io.IOUtils; +import org.apache.commons.lang3.StringUtils; import org.apache.spark.SparkConf; +import org.apache.spark.api.java.function.FilterFunction; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Encoder; +import org.apache.spark.sql.Encoders; import org.apache.spark.sql.SparkSession; import org.apache.spark.util.LongAccumulator; import org.dom4j.Document; import org.dom4j.Node; import org.dom4j.io.SAXReader; -import org.junit.jupiter.api.AfterAll; -import org.junit.jupiter.api.BeforeAll; -import org.junit.jupiter.api.DisplayName; -import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.*; import org.junit.jupiter.api.extension.ExtendWith; import org.junit.jupiter.api.io.TempDir; import org.mockito.Mock; import org.mockito.junit.jupiter.MockitoExtension; +import eu.dnetlib.dhp.aggregation.common.AggregationCounter; import eu.dnetlib.dhp.collection.CollectionJobTest; +import eu.dnetlib.dhp.common.vocabulary.VocabularyGroup; import eu.dnetlib.dhp.model.mdstore.MetadataRecord; -import eu.dnetlib.dhp.transformation.functions.Cleaner; -import eu.dnetlib.dhp.transformation.vocabulary.Vocabulary; -import eu.dnetlib.dhp.transformation.vocabulary.VocabularyHelper; -import eu.dnetlib.dhp.utils.DHPUtils; -import net.sf.saxon.s9api.*; +import eu.dnetlib.dhp.transformation.xslt.XSLTTransformationFunction; +import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException; +import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService; @ExtendWith(MockitoExtension.class) public class TransformationJobTest { private static SparkSession spark; + @Mock + private ISLookUpService isLookUpService; + + private VocabularyGroup vocabularies; + + @BeforeEach + public void setUp() throws ISLookUpException, IOException { + lenient().when(isLookUpService.quickSearchProfile(VocabularyGroup.VOCABULARIES_XQUERY)).thenReturn(vocs()); + + lenient() + .when(isLookUpService.quickSearchProfile(VocabularyGroup.VOCABULARY_SYNONYMS_XQUERY)) + .thenReturn(synonyms()); + vocabularies = VocabularyGroup.loadVocsFromIS(isLookUpService); + } + @BeforeAll public static void beforeAll() { SparkConf conf = new SparkConf(); @@ -53,66 +74,68 @@ public class TransformationJobTest { spark.stop(); } - @Mock - private LongAccumulator accumulator; - @Test + @DisplayName("Test Transform Single XML using XSLTTransformator") public void testTransformSaxonHE() throws Exception { - Map vocabularies = new HashMap<>(); - vocabularies.put("dnet:languages", VocabularyHelper.getVocabularyFromAPI("dnet:languages")); - Cleaner cleanFunction = new Cleaner(vocabularies); - Processor proc = new Processor(false); - proc.registerExtensionFunction(cleanFunction); - final XsltCompiler comp = proc.newXsltCompiler(); - XsltExecutable exp = comp - .compile( - new StreamSource( - this.getClass().getResourceAsStream("/eu/dnetlib/dhp/transform/ext_simple.xsl"))); - XdmNode source = proc - .newDocumentBuilder() - .build( - new StreamSource( - this.getClass().getResourceAsStream("/eu/dnetlib/dhp/transform/input.xml"))); - XsltTransformer trans = exp.load(); - trans.setInitialContextNode(source); - final StringWriter output = new StringWriter(); - Serializer out = proc.newSerializer(output); - out.setOutputProperty(Serializer.Property.METHOD, "xml"); - out.setOutputProperty(Serializer.Property.INDENT, "yes"); - trans.setDestination(out); - trans.transform(); - System.out.println(output.toString()); + // We Set the input Record getting the XML from the classpath + final MetadataRecord mr = new MetadataRecord(); + mr.setBody(IOUtils.toString(getClass().getResourceAsStream("/eu/dnetlib/dhp/transform/input.xml"))); + + // We Load the XSLT transformation Rule from the classpath + XSLTTransformationFunction tr = loadTransformationRule("/eu/dnetlib/dhp/transform/ext_simple.xsl"); + + // Print the record + System.out.println(tr.call(mr).getBody()); + // TODO Create significant Assert + } @DisplayName("Test TransformSparkJobNode.main") @Test public void transformTest(@TempDir Path testDir) throws Exception { + final String mdstore_input = this.getClass().getResource("/eu/dnetlib/dhp/transform/mdstorenative").getFile(); final String mdstore_output = testDir.toString() + "/version"; - final String xslt = DHPUtils - .compressString( - IOUtils - .toString( - this.getClass().getResourceAsStream("/eu/dnetlib/dhp/transform/tr.xml"))); - TransformSparkJobNode - .main( - new String[] { - "-issm", "true", - "-i", mdstore_input, - "-o", mdstore_output, - "-d", "1", - "-w", "1", - "-tr", xslt, - "-t", "true", - "-ru", "", - "-rp", "", - "-rh", "", - "-ro", "", - "-rr", "" - }); + + mockupTrasformationRule("simpleTRule", "/eu/dnetlib/dhp/transform/ext_simple.xsl"); + +// final String arguments = "-issm true -i %s -o %s -d 1 -w 1 -tp XSLT_TRANSFORM -tr simpleTRule"; + + final Map parameters = Stream.of(new String[][] { + { + "dateOfTransformation", "1234" + }, + { + "transformationPlugin", "XSLT_TRANSFORM" + }, + { + "transformationRuleTitle", "simpleTRule" + }, + + }).collect(Collectors.toMap(data -> data[0], data -> data[1])); + + TransformSparkJobNode.transformRecords(parameters, isLookUpService, spark, mdstore_input, mdstore_output); // TODO introduce useful assertions + + final Encoder encoder = Encoders.bean(MetadataRecord.class); + final Dataset mOutput = spark.read().format("parquet").load(mdstore_output).as(encoder); + + final Long total = mOutput.count(); + + final long recordTs = mOutput + .filter((FilterFunction) p -> p.getDateOfTransformation() == 1234) + .count(); + + final long recordNotEmpty = mOutput + .filter((FilterFunction) p -> !StringUtils.isBlank(p.getBody())) + .count(); + + assertEquals(total, recordTs); + + assertEquals(total, recordNotEmpty); + } @Test @@ -127,39 +150,27 @@ public class TransformationJobTest { Files.deleteIfExists(tempDirWithPrefix); } - @Test - public void testTransformFunction() throws Exception { - SAXReader reader = new SAXReader(); - Document document = reader.read(this.getClass().getResourceAsStream("/eu/dnetlib/dhp/transform/tr.xml")); - Node node = document.selectSingleNode("//CODE/*[local-name()='stylesheet']"); - final String xslt = node.asXML(); - Map vocabularies = new HashMap<>(); - vocabularies.put("dnet:languages", VocabularyHelper.getVocabularyFromAPI("dnet:languages")); + private void mockupTrasformationRule(final String trule, final String path) throws Exception { + final String trValue = IOUtils.toString(this.getClass().getResourceAsStream(path)); - TransformFunction tf = new TransformFunction(accumulator, accumulator, accumulator, xslt, 1, vocabularies); - - MetadataRecord record = new MetadataRecord(); - record - .setBody( - IOUtils - .toString( - this.getClass().getResourceAsStream("/eu/dnetlib/dhp/transform/input.xml"))); - - final MetadataRecord result = tf.call(record); - assertNotNull(result.getBody()); - - System.out.println(result.getBody()); + lenient() + .when(isLookUpService.quickSearchProfile(String.format(TransformationFactory.TRULE_XQUERY, trule))) + .thenReturn(Collections.singletonList(trValue)); } - @Test - public void extractTr() throws Exception { + private XSLTTransformationFunction loadTransformationRule(final String path) throws Exception { + final String trValue = IOUtils.toString(this.getClass().getResourceAsStream(path)); + final LongAccumulator la = new LongAccumulator(); + return new XSLTTransformationFunction(new AggregationCounter(la, la, la), trValue, 0, vocabularies); + } - final String xmlTr = IOUtils.toString(this.getClass().getResourceAsStream("/eu/dnetlib/dhp/transform/tr.xml")); + private List vocs() throws IOException { + return IOUtils + .readLines(TransformationJobTest.class.getResourceAsStream("/eu/dnetlib/dhp/transform/terms.txt")); + } - SAXReader reader = new SAXReader(); - Document document = reader.read(this.getClass().getResourceAsStream("/eu/dnetlib/dhp/transform/tr.xml")); - Node node = document.selectSingleNode("//CODE/*[local-name()='stylesheet']"); - - System.out.println(node.asXML()); + private List synonyms() throws IOException { + return IOUtils + .readLines(TransformationJobTest.class.getResourceAsStream("/eu/dnetlib/dhp/transform/synonyms.txt")); } } diff --git a/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/transform/ext_simple.xsl b/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/transform/ext_simple.xsl index f22db961b..9e5f84c11 100644 --- a/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/transform/ext_simple.xsl +++ b/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/transform/ext_simple.xsl @@ -1,15 +1,16 @@ + exclude-result-prefixes="xsl vocabulary"> - - + + diff --git a/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/transform/input.xml b/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/transform/input.xml index 8760d3117..8efb3c487 100644 --- a/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/transform/input.xml +++ b/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/transform/input.xml @@ -1,37 +1,68 @@ - -
- oai:research.chalmers.se:243692 - 2018-01-25T18:04:43Z - openaire -
- - - Incipient Berezinskii-Kosterlitz-Thouless transition in two-dimensional coplanar Josephson junctions - https://research.chalmers.se/en/publication/243692 - 2016 - Massarotti, D. - Jouault, B. - Rouco, V. - Charpentier, Sophie - Bauch, Thilo - Michon, A. - De Candia, A. - Lucignano, P. - Lombardi, Floriana - Tafuri, F. - Tagliacozzo, A. - Acoli - Abkhazian - Condensed Matter Physics - Superconducting hybrid junctions are revealing a variety of effects. Some of them are due to the special layout of these devices, which often use a coplanar configuration with relatively large barrier channels and the possibility of hosting Pearl vortices. A Josephson junction with a quasi-ideal two-dimensional barrier has been realized by growing graphene on SiC with Al electrodes. Chemical vapor deposition offers centimeter size monolayer areas where it is possible to realize a comparative analysis of different devices with nominally the same barrier. In samples with a graphene gap below 400 nm, we have found evidence of Josephson coherence in the presence of an incipient Berezinskii-Kosterlitz-Thouless transition. When the magnetic field is cycled, a remarkable hysteretic collapse and revival of the Josephson supercurrent occurs. Similar hysteresis are found in granular systems and are usually justified within the Bean critical state model (CSM). We show that the CSM, with appropriate account for the low-dimensional geometry, can partly explain the odd features measured in these junctions. - info:eu-repo/grantAgreement/EC/FP7/604391//Graphene-Based Revolutions in ICT And Beyond (Graphene Flagship)/ - info:eu-repo/semantics/altIdentifier/doi/10.1103/PhysRevB.94.054525 - info:eu-repo/semantics/article - Physical Review B vol.94(2016) - info:eu-repo/semantics/openAccess + + + + od______2294::00029b7f0a2a7e090e55b625a9079d83 + oai:pub.uni-bielefeld.de:2578942 + 2018-11-23T15:15:33.974+01:00 + od______2294 + oai:pub.uni-bielefeld.de:2578942 + 2018-07-24T13:01:16Z + conference + ddc:000 + conferenceFtxt + driver + open_access + + + + Mobile recommendation agents making online use of visual attention information at the point of sale + Pfeiffer, Thies + Pfeiffer, Jella + Meißner, Martin + Davis, Fred + Riedl, René + Jan, vom Brocke + Léger, Pierre-Majorique + Randolph, Adriane + Mobile Cognitive Assistance Systems + Information Systems + ddc:000 + We aim to utilize online information about visual attention for developing mobile recommendation agents (RAs) for use at the point of sale. Up to now, most RAs are focussed exclusively at personalization in an e-commerce setting. Very little is known, however, about mobile RAs that offer information and assistance at the point of sale based on individual-level feature based preference models (Murray and Häubl 2009). Current attempts provide information about products at the point of sale by manually scanning barcodes or using RFID (Kowatsch et al. 2011, Heijden 2005), e.g. using specific apps for smartphones. We argue that an online access to the current visual attention of the user offers a much larger potential. Integrating mobile eye tracking into ordinary glasses would yield a direct benefit of applying neuroscience methods in the user’s everyday life. First, learning from consumers’ attentional processes over time and adapting recommendations based on this learning allows us to provide very accurate and relevant recommendations, potentially increasing the perceived usefulness. Second, our proposed system needs little explicit user input (no scanning or navigation on screen) making it easy to use. Thus, instead of learning from click behaviour and past customer ratings, as it is the case in the e-commerce setting, the mobile RA learns from eye movements by participating online in every day decision processes. We argue that mobile RAs should be built based on current research in human judgment and decision making (Murray et al. 2010). In our project, we therefore follow a two-step approach: In the empirical basic research stream, we aim to understand the user’s interaction with the product shelf: the actions and patterns of user’s behaviour (eye movements, gestures, approaching a product closer) and their correspondence to the user’s informational needs. In the empirical system development stream, we create prototypes of mobile RAs and test experimentally the factors that influence the user’s adoption. For example, we suggest that a user’s involvement in the process, such as a need for exact nutritional information or for assistance (e.g., reading support for elderly) will influence the user’s intention to use such as system. The experiments are conducted both in our immersive virtual reality supermarket presented in a CAVE, where we can also easily display information to the user and track the eye movement in great accuracy, as well as in real-world supermarkets (see Figure 1), so that the findings can be better generalized to natural decision situations (Gidlöf et al. 2013). In a first pilot study with five randomly chosen participants in a supermarket, we evaluated which sort of mobile RAs consumers favour in order to get a first impression of the user’s acceptance of the technology. Figure 1 shows an excerpt of one consumer’s eye movements during a decision process. First results show long eye cascades and short fixations on many products in situations where users are uncertain and in need for support. Furthermore, we find a surprising acceptance of the technology itself throughout all ages (23 – 61 years). At the same time, consumers express serious fear of being manipulated by such a technology. For that reason, they strongly prefer the information to be provided by trusted third party or shared with family members and friends (see also Murray and Häubl 2009). Our pilot will be followed by a larger field experiment in March in order to learn more about factors that influence the user’s acceptance as well as the eye movement patterns that reflect typical phases of decision processes and indicate the need for support by a RA. + 2013 + info:eu-repo/semantics/conferenceObject + doc-type:conferenceObject + text + https://pub.uni-bielefeld.de/record/2578942 + https://pub.uni-bielefeld.de/download/2578942/2602478 + Pfeiffer T, Pfeiffer J, Meißner M. Mobile recommendation agents making online use of visual attention information at the point of sale. In: Davis F, Riedl R, Jan vom B, Léger P-M, Randolph A, eds. Proceedings of the Gmunden Retreat on NeuroIS 2013. 2013: 3-3. eng - Researchers - application/pdf + info:eu-repo/semantics/openAccess -
+ + + + http://pub.uni-bielefeld.de/oai + oai:pub.uni-bielefeld.de:2578942 + 2018-07-24T13:01:16Z + http://www.openarchives.org/OAI/2.0/oai_dc/ + + + + false + false + 0.9 + + + + +
diff --git a/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/transform/synonyms.txt b/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/transform/synonyms.txt new file mode 100644 index 000000000..729296522 --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/transform/synonyms.txt @@ -0,0 +1,1234 @@ +dnet:access_modes @=@ CLOSED @=@ http://purl.org/coar/access_right/c_14cb +dnet:access_modes @=@ CLOSED @=@ info:eu-repo/semantics/closedAccess +dnet:access_modes @=@ EMBARGO @=@ http://purl.org/coar/access_right/c_f1cf +dnet:access_modes @=@ EMBARGO @=@ info:eu-repo/semantics/embargoedAccess +dnet:access_modes @=@ OPEN @=@ Creative Commons License [CC BY-NC-ND] http://creativecommons.org/licenses/by-nc-nd/3.0/de/ +dnet:access_modes @=@ OPEN @=@ Creative commons +dnet:access_modes @=@ OPEN @=@ http://creativecommons.org/licenses/by-nc-nd/3.0/ +dnet:access_modes @=@ OPEN @=@ http://creativecommons.org/licenses/by-nc/3.0/ +dnet:access_modes @=@ OPEN @=@ http://creativecommons.org/licenses/by-sa/3.0/ +dnet:access_modes @=@ OPEN @=@ http://creativecommons.org/licenses/by-sa/4.0/ +dnet:access_modes @=@ OPEN @=@ http://creativecommons.org/licenses/by/3.0/ +dnet:access_modes @=@ OPEN @=@ http://creativecommons.org/licenses/by/3.0/us/ +dnet:access_modes @=@ OPEN @=@ http://creativecommons.org/licenses/by/4.0/ +dnet:access_modes @=@ OPEN @=@ http://creativecommons.org/publicdomain/zero/1.0/ +dnet:access_modes @=@ OPEN @=@ http://creativecommons.org/publicdomain/zero/1.0/ & http://www.canadensys.net/norms +dnet:access_modes @=@ OPEN @=@ http://purl.org/coar/access_right/c_abf2 +dnet:access_modes @=@ OPEN @=@ https://creativecommons.org/licenses/by-nc/4.0/ +dnet:access_modes @=@ OPEN @=@ info:eu-repo/semantics/openAccess +dnet:access_modes @=@ OPEN @=@ open_access +dnet:access_modes @=@ RESTRICTED @=@ http://purl.org/coar/access_right/c_16ec +dnet:access_modes @=@ RESTRICTED @=@ info:eu-repo/semantics/restrictedAccess +dnet:compatibilityLevel @=@ openaire-pub_4.0 @=@ openaire4.0 +dnet:subject_classification_typologies @=@ jel @=@ jelElement +dnet:publication_resource @=@ 0018 @=@ Comment/debate +dnet:publication_resource @=@ 0018 @=@ http://purl.org/coar/resource_type/c_1162 +dnet:publication_resource @=@ 0018 @=@ info:eu-repo/semantics/annotation +dnet:publication_resource @=@ 0001 @=@ A1 Alkuperäisartikkeli tieteellisessä aikakauslehdessä +dnet:publication_resource @=@ 0001 @=@ Article +dnet:publication_resource @=@ 0001 @=@ Article (author) +dnet:publication_resource @=@ 0001 @=@ Article - letter to the editor +dnet:publication_resource @=@ 0001 @=@ Article / Letter to editor +dnet:publication_resource @=@ 0001 @=@ Article / Letter to the editor +dnet:publication_resource @=@ 0001 @=@ Article / Newspaper +dnet:publication_resource @=@ 0001 @=@ Article in journal +dnet:publication_resource @=@ 0001 @=@ Article in monograph or in proceedings +dnet:publication_resource @=@ 0001 @=@ Article in proceedings +dnet:publication_resource @=@ 0001 @=@ Article-letter to the editor +dnet:publication_resource @=@ 0001 @=@ Article/Letter to editor +dnet:publication_resource @=@ 0001 @=@ Articolo +dnet:publication_resource @=@ 0001 @=@ Artículo +dnet:publication_resource @=@ 0001 @=@ Aufsatz +dnet:publication_resource @=@ 0001 @=@ Clinical Study +dnet:publication_resource @=@ 0001 @=@ Institutional Series +dnet:publication_resource @=@ 0001 @=@ International Journal +dnet:publication_resource @=@ 0001 @=@ International Journal Abstract +dnet:publication_resource @=@ 0001 @=@ International Journal ISI/JCR +dnet:publication_resource @=@ 0001 @=@ Journal (full / special issue) +dnet:publication_resource @=@ 0001 @=@ Journal Article/Review +dnet:publication_resource @=@ 0001 @=@ Journal article +dnet:publication_resource @=@ 0001 @=@ Journal article (on-line or printed) +dnet:publication_resource @=@ 0001 @=@ Journal articles +dnet:publication_resource @=@ 0001 @=@ Journal paper +dnet:publication_resource @=@ 0001 @=@ National Journal +dnet:publication_resource @=@ 0001 @=@ Original article (non peer-reviewed) +dnet:publication_resource @=@ 0001 @=@ Original article (peer-reviewed) +dnet:publication_resource @=@ 0001 @=@ Peer-reviewed Article +dnet:publication_resource @=@ 0001 @=@ Published Journal Article +dnet:publication_resource @=@ 0001 @=@ Research Article +dnet:publication_resource @=@ 0001 @=@ Review article (non peer-reviewed) +dnet:publication_resource @=@ 0001 @=@ Review article (peer-reviewed) +dnet:publication_resource @=@ 0001 @=@ Volumes Edited / Special Issues +dnet:publication_resource @=@ 0001 @=@ article in non peer-reviewed journal +dnet:publication_resource @=@ 0001 @=@ article in peer-reviewed journal +dnet:publication_resource @=@ 0001 @=@ article-commentary +dnet:publication_resource @=@ 0001 @=@ article_site_web +dnet:publication_resource @=@ 0001 @=@ doc-type:Journal Article +dnet:publication_resource @=@ 0001 @=@ doc-type:article +dnet:publication_resource @=@ 0001 @=@ http://purl.org/coar/resource_type/c_2df8fbb1 +dnet:publication_resource @=@ 0001 @=@ http://purl.org/coar/resource_type/c_545b +dnet:publication_resource @=@ 0001 @=@ http://purl.org/coar/resource_type/c_6501 +dnet:publication_resource @=@ 0001 @=@ http://purl.org/coar/resource_type/c_7877 +dnet:publication_resource @=@ 0001 @=@ in-brief +dnet:publication_resource @=@ 0001 @=@ info:eu-repo/semantics/article +dnet:publication_resource @=@ 0001 @=@ journal-article +dnet:publication_resource @=@ 0001 @=@ journalArticle +dnet:publication_resource @=@ 0001 @=@ journal_article +dnet:publication_resource @=@ 0001 @=@ letter +dnet:publication_resource @=@ 0001 @=@ non peer-reviewed article +dnet:publication_resource @=@ 0001 @=@ partial-retraction +dnet:publication_resource @=@ 0001 @=@ proceeding with peer review +dnet:publication_resource @=@ 0001 @=@ publication-article +dnet:publication_resource @=@ 0001 @=@ rapid-communication +dnet:publication_resource @=@ 0001 @=@ reply +dnet:publication_resource @=@ 0001 @=@ research-article +dnet:publication_resource @=@ 0001 @=@ retraction +dnet:publication_resource @=@ 0001 @=@ review-article +dnet:publication_resource @=@ 0001 @=@ text (article) +dnet:publication_resource @=@ 0001 @=@ Статья +dnet:publication_resource @=@ 0001 @=@ ArticleArtikel +dnet:publication_resource @=@ 0033 @=@ AUDIOVISUAL_DOCUMENT +dnet:publication_resource @=@ 0033 @=@ Audiovisual/Audiovisual +dnet:publication_resource @=@ 0033 @=@ http://purl.org/coar/resource_type/c_c513 +dnet:publication_resource @=@ 0008 @=@ Bachelor's +dnet:publication_resource @=@ 0008 @=@ Bachelor's Degree +dnet:publication_resource @=@ 0008 @=@ Bachelors Thesis +dnet:publication_resource @=@ 0008 @=@ Proyecto fin de carrera +dnet:publication_resource @=@ 0008 @=@ Undergraduate Thesis +dnet:publication_resource @=@ 0008 @=@ http://purl.org/coar/resource_type/c_7a1f +dnet:publication_resource @=@ 0008 @=@ info:eu-repo/semantics/bachelorThesis +dnet:publication_resource @=@ 0008 @=@ выпускная бакалаврская работа +dnet:publication_resource @=@ 0002 @=@ Book (monograph) +dnet:publication_resource @=@ 0002 @=@ Book (non peer-reviewed) +dnet:publication_resource @=@ 0002 @=@ Book (peer-reviewed) +dnet:publication_resource @=@ 0002 @=@ Book - monograph - editorial book +dnet:publication_resource @=@ 0002 @=@ Book Section +dnet:publication_resource @=@ 0002 @=@ Book as author +dnet:publication_resource @=@ 0002 @=@ Buch +dnet:publication_resource @=@ 0002 @=@ International Book/Monograph +dnet:publication_resource @=@ 0002 @=@ Libro +dnet:publication_resource @=@ 0002 @=@ Monografia +dnet:publication_resource @=@ 0002 @=@ Monograph +dnet:publication_resource @=@ 0002 @=@ National Book/Monograph +dnet:publication_resource @=@ 0002 @=@ atlas +dnet:publication_resource @=@ 0002 @=@ book +dnet:publication_resource @=@ 0002 @=@ book-series +dnet:publication_resource @=@ 0002 @=@ book-set +dnet:publication_resource @=@ 0002 @=@ book-track +dnet:publication_resource @=@ 0002 @=@ book_series +dnet:publication_resource @=@ 0002 @=@ book_title +dnet:publication_resource @=@ 0002 @=@ doc-type:book +dnet:publication_resource @=@ 0002 @=@ edited-book +dnet:publication_resource @=@ 0002 @=@ http://purl.org/coar/resource_type/c_2f33 +dnet:publication_resource @=@ 0002 @=@ info:eu-repo/semantics/book +dnet:publication_resource @=@ 0002 @=@ ouvrage +dnet:publication_resource @=@ 0002 @=@ publication-book +dnet:publication_resource @=@ 0002 @=@ reference-book +dnet:publication_resource @=@ 0002 @=@ scientific book +dnet:publication_resource @=@ 0002 @=@ Монография +dnet:publication_resource @=@ 0002 @=@ Учебник +dnet:publication_resource @=@ 0037 @=@ clinicalTrial +dnet:publication_resource @=@ 0037 @=@ http://purl.org/coar/resource_type/c_cb28 +dnet:publication_resource @=@ 0022 @=@ collection +dnet:publication_resource @=@ 0004 @=@ A4 Artikkeli konferenssijulkaisussa +dnet:publication_resource @=@ 0004 @=@ Comunicación de congreso +dnet:publication_resource @=@ 0004 @=@ Conference Paper +dnet:publication_resource @=@ 0004 @=@ Conference Paper/Proceeding/Abstract +dnet:publication_resource @=@ 0004 @=@ Conference Proceedings +dnet:publication_resource @=@ 0004 @=@ Conference article +dnet:publication_resource @=@ 0004 @=@ Conference contribution +dnet:publication_resource @=@ 0004 @=@ Conference lecture +dnet:publication_resource @=@ 0004 @=@ Conference or Workshop Item +dnet:publication_resource @=@ 0004 @=@ Conference paper, poster, etc. +dnet:publication_resource @=@ 0004 @=@ Conference papers +dnet:publication_resource @=@ 0004 @=@ Conference report +dnet:publication_resource @=@ 0004 @=@ International Conference +dnet:publication_resource @=@ 0004 @=@ International Conference Abstract/Poster +dnet:publication_resource @=@ 0004 @=@ International Conference ISI/JCR +dnet:publication_resource @=@ 0004 @=@ International Conference communication/abstract/poster +dnet:publication_resource @=@ 0004 @=@ National Conference +dnet:publication_resource @=@ 0004 @=@ National Conference Abstract/Poster +dnet:publication_resource @=@ 0004 @=@ National Conference communication/abstract/poster +dnet:publication_resource @=@ 0004 @=@ PREFACE_PROCEEDINGS +dnet:publication_resource @=@ 0004 @=@ PROCEEDING_PAPER +dnet:publication_resource @=@ 0004 @=@ Papers in Conference Proceedings +dnet:publication_resource @=@ 0004 @=@ Presentación +dnet:publication_resource @=@ 0004 @=@ Proceedings (peer-reviewed) +dnet:publication_resource @=@ 0004 @=@ Proceedings of a Conference +dnet:publication_resource @=@ 0004 @=@ Proceedings paper +dnet:publication_resource @=@ 0004 @=@ Póster +dnet:publication_resource @=@ 0004 @=@ actes_congres +dnet:publication_resource @=@ 0004 @=@ communication_avec_actes +dnet:publication_resource @=@ 0004 @=@ communication_invitee +dnet:publication_resource @=@ 0004 @=@ communication_par_affiche +dnet:publication_resource @=@ 0004 @=@ communication_sans_actes +dnet:publication_resource @=@ 0004 @=@ conference +dnet:publication_resource @=@ 0004 @=@ conference item +dnet:publication_resource @=@ 0004 @=@ conference proceeding +dnet:publication_resource @=@ 0004 @=@ conferenceObject +dnet:publication_resource @=@ 0004 @=@ conference_paper +dnet:publication_resource @=@ 0004 @=@ doc-type:conferenceObject +dnet:publication_resource @=@ 0004 @=@ http://purl.org/coar/resource_type/c_18co +dnet:publication_resource @=@ 0004 @=@ http://purl.org/coar/resource_type/c_18cp +dnet:publication_resource @=@ 0004 @=@ http://purl.org/coar/resource_type/c_5794 +dnet:publication_resource @=@ 0004 @=@ http://purl.org/coar/resource_type/c_6670 +dnet:publication_resource @=@ 0004 @=@ http://purl.org/coar/resource_type/c_c94f +dnet:publication_resource @=@ 0004 @=@ http://purl.org/coar/resource_type/c_f744 +dnet:publication_resource @=@ 0004 @=@ info:eu-repo/semantics/conferenceItem +dnet:publication_resource @=@ 0004 @=@ info:eu-repo/semantics/conferenceObject +dnet:publication_resource @=@ 0004 @=@ invited conference talk +dnet:publication_resource @=@ 0004 @=@ poster +dnet:publication_resource @=@ 0004 @=@ presentation +dnet:publication_resource @=@ 0004 @=@ proceeding, seminar, workshop without peer review +dnet:publication_resource @=@ 0004 @=@ proceedings +dnet:publication_resource @=@ 0004 @=@ proceedings-article +dnet:publication_resource @=@ 0004 @=@ publication-conferencepaper +dnet:publication_resource @=@ 0004 @=@ научный доклад +dnet:publication_resource @=@ 0005 @=@ Newspaper or magazine article +dnet:publication_resource @=@ 0005 @=@ http://purl.org/coar/resource_type/c_998f +dnet:publication_resource @=@ 0005 @=@ info:eu-repo/semantics/contributionToPeriodical +dnet:publication_resource @=@ 0045 @=@ Data Management Plan +dnet:publication_resource @=@ 0045 @=@ Data Management Plan (NSF Generic) +dnet:publication_resource @=@ 0045 @=@ http://purl.org/coar/resource_type/c_ab20 +dnet:publication_resource @=@ 0045 @=@ http://purl.org/spar/fabio/DataManagementPolicy +dnet:publication_resource @=@ 0045 @=@ http://purl.org/spar/fabio/DataManagementPolicyDocument +dnet:publication_resource @=@ 0045 @=@ http://purl.org/spar/fabio/DataMangementPlan +dnet:publication_resource @=@ 0045 @=@ plan de gestión de datos +dnet:publication_resource @=@ 0045 @=@ publication-datamanagementplan +dnet:publication_resource @=@ 0031 @=@ Data Descriptor +dnet:publication_resource @=@ 0031 @=@ DataPaper +dnet:publication_resource @=@ 0031 @=@ data-article +dnet:publication_resource @=@ 0031 @=@ http://purl.org/coar/resource_type/c_beb9 +dnet:publication_resource @=@ 0021 @=@ Dataset/Dataset +dnet:publication_resource @=@ 0021 @=@ Research Data +dnet:publication_resource @=@ 0021 @=@ dataset +dnet:publication_resource @=@ 0021 @=@ http://purl.org/coar/resource_type/c_ddb1 +dnet:publication_resource @=@ 0021 @=@ info:eu-repo/semantics/DDIInstance +dnet:publication_resource @=@ 0021 @=@ info:eu-repo/semantics/datafile +dnet:publication_resource @=@ 0021 @=@ info:eu-repo/semantics/dataset +dnet:publication_resource @=@ 0021 @=@ info:eu-repo/semantics/enhancedObjectFile +dnet:publication_resource @=@ 0006 @=@ Diss +dnet:publication_resource @=@ 0006 @=@ Dissertation +dnet:publication_resource @=@ 0006 @=@ Doctoral +dnet:publication_resource @=@ 0006 @=@ DoctoralThesis +dnet:publication_resource @=@ 0006 @=@ PhD thesis +dnet:publication_resource @=@ 0006 @=@ Tesis +dnet:publication_resource @=@ 0006 @=@ Text.Thesis.Doctoral +dnet:publication_resource @=@ 0006 @=@ Theses +dnet:publication_resource @=@ 0006 @=@ Thesis +dnet:publication_resource @=@ 0006 @=@ Thesis or Dissertation +dnet:publication_resource @=@ 0006 @=@ Thesis.Doctoral +dnet:publication_resource @=@ 0006 @=@ doc-type:doctoralThesis +dnet:publication_resource @=@ 0006 @=@ http://purl.org/coar/resource_type/c_db06 +dnet:publication_resource @=@ 0006 @=@ info:eu-repo/semantics/doctoralThesis +dnet:publication_resource @=@ 0006 @=@ publication-thesis +dnet:publication_resource @=@ 0006 @=@ these +dnet:publication_resource @=@ 0006 @=@ these exercice +dnet:publication_resource @=@ 0023 @=@ Event/Event +dnet:publication_resource @=@ 0023 @=@ event +dnet:publication_resource @=@ 0009 @=@ Departmental Technical Report +dnet:publication_resource @=@ 0009 @=@ Informe Técnico +dnet:publication_resource @=@ 0009 @=@ RESEARCH_REPORT +dnet:publication_resource @=@ 0009 @=@ Tech-Report +dnet:publication_resource @=@ 0009 @=@ Technical Report +dnet:publication_resource @=@ 0009 @=@ http://purl.org/coar/resource_type/c_18gh +dnet:publication_resource @=@ 0009 @=@ publication-technicalnote +dnet:publication_resource @=@ 0009 @=@ research report +dnet:publication_resource @=@ 0024 @=@ Video +dnet:publication_resource @=@ 0024 @=@ film +dnet:publication_resource @=@ 0024 @=@ http://purl.org/coar/resource_type/c_12ce +dnet:publication_resource @=@ 0024 @=@ http://purl.org/coar/resource_type/c_8a7e +dnet:publication_resource @=@ 0025 @=@ Diagram +dnet:publication_resource @=@ 0025 @=@ Drawing +dnet:publication_resource @=@ 0025 @=@ Figure +dnet:publication_resource @=@ 0025 @=@ Image/Image +dnet:publication_resource @=@ 0025 @=@ Imagen +dnet:publication_resource @=@ 0025 @=@ Photo +dnet:publication_resource @=@ 0025 @=@ Plot +dnet:publication_resource @=@ 0025 @=@ fotó +dnet:publication_resource @=@ 0025 @=@ grafika +dnet:publication_resource @=@ 0025 @=@ http://purl.org/coar/resource_type/c_ecc8 +dnet:publication_resource @=@ 0025 @=@ image +dnet:publication_resource @=@ 0025 @=@ image-diagram +dnet:publication_resource @=@ 0025 @=@ image-drawing +dnet:publication_resource @=@ 0025 @=@ image-figure +dnet:publication_resource @=@ 0025 @=@ image-other +dnet:publication_resource @=@ 0025 @=@ image-photo +dnet:publication_resource @=@ 0025 @=@ image-plot +dnet:publication_resource @=@ 0026 @=@ http://purl.org/coar/resource_type/c_e9a0 +dnet:publication_resource @=@ 0026 @=@ interactiveResource +dnet:publication_resource @=@ 0011 @=@ Internal note +dnet:publication_resource @=@ 0011 @=@ http://purl.org/coar/resource_type/c_18ww +dnet:publication_resource @=@ 0043 @=@ http://purl.org/coar/resource_type/c_0640 +dnet:publication_resource @=@ 0010 @=@ Inaugural lecture +dnet:publication_resource @=@ 0010 @=@ Material didáctico +dnet:publication_resource @=@ 0010 @=@ Public-Lecture +dnet:publication_resource @=@ 0010 @=@ http://purl.org/coar/resource_type/c_8544 +dnet:publication_resource @=@ 0010 @=@ info:eu-repo/semantics/lecture +dnet:publication_resource @=@ 0010 @=@ lesson +dnet:publication_resource @=@ 0010 @=@ Учебный материал +dnet:publication_resource @=@ 0007 @=@ Diploma Project +dnet:publication_resource @=@ 0007 @=@ MSc Thesis +dnet:publication_resource @=@ 0007 @=@ Master Degree +dnet:publication_resource @=@ 0007 @=@ Master's +dnet:publication_resource @=@ 0007 @=@ Masterarbeit u.a. +dnet:publication_resource @=@ 0007 @=@ Masters (Taught) +dnet:publication_resource @=@ 0007 @=@ Masters thesis +dnet:publication_resource @=@ 0007 @=@ Masters-Thesis.Magister +dnet:publication_resource @=@ 0007 @=@ Tesina +dnet:publication_resource @=@ 0007 @=@ Thesis.Master +dnet:publication_resource @=@ 0007 @=@ Trabajo fin de Máster +dnet:publication_resource @=@ 0007 @=@ doc-type:masterThesis +dnet:publication_resource @=@ 0007 @=@ hdr +dnet:publication_resource @=@ 0007 @=@ http://purl.org/coar/resource_type/c_bdcc +dnet:publication_resource @=@ 0007 @=@ info:eu-repo/semantics/masterThesis +dnet:publication_resource @=@ 0007 @=@ masterThesis +dnet:publication_resource @=@ 0007 @=@ memoire +dnet:publication_resource @=@ 0027 @=@ Model/Model +dnet:publication_resource @=@ 0027 @=@ model +dnet:publication_resource @=@ 0020 @=@ Exhibition +dnet:publication_resource @=@ 0020 @=@ Learning Object +dnet:publication_resource @=@ 0020 @=@ Mapa +dnet:publication_resource @=@ 0020 @=@ Modelo de utilidad +dnet:publication_resource @=@ 0020 @=@ PEDAGOGICAL_DOCUMENT +dnet:publication_resource @=@ 0020 @=@ Partitura +dnet:publication_resource @=@ 0020 @=@ Sitio web +dnet:publication_resource @=@ 0020 @=@ Trabajo de divulgación +dnet:publication_resource @=@ 0020 @=@ Web publication/site +dnet:publication_resource @=@ 0020 @=@ application +dnet:publication_resource @=@ 0020 @=@ artefact +dnet:publication_resource @=@ 0020 @=@ carte +dnet:publication_resource @=@ 0020 @=@ composition +dnet:publication_resource @=@ 0020 @=@ document_audiovisuel +dnet:publication_resource @=@ 0020 @=@ http://purl.org/coar/resource_type/c_12cc +dnet:publication_resource @=@ 0020 @=@ http://purl.org/coar/resource_type/c_12cd +dnet:publication_resource @=@ 0020 @=@ http://purl.org/coar/resource_type/c_1843 +dnet:publication_resource @=@ 0020 @=@ http://purl.org/coar/resource_type/c_18cd +dnet:publication_resource @=@ 0020 @=@ http://purl.org/coar/resource_type/c_18cw +dnet:publication_resource @=@ 0020 @=@ http://purl.org/coar/resource_type/c_26e4 +dnet:publication_resource @=@ 0020 @=@ http://purl.org/coar/resource_type/c_7ad9 +dnet:publication_resource @=@ 0020 @=@ http://purl.org/coar/resource_type/c_e059 +dnet:publication_resource @=@ 0020 @=@ info:eu-repo/semantics/other +dnet:publication_resource @=@ 0020 @=@ learningObject +dnet:publication_resource @=@ 0020 @=@ map +dnet:publication_resource @=@ 0020 @=@ misc +dnet:publication_resource @=@ 0020 @=@ other +dnet:publication_resource @=@ 0020 @=@ revue +dnet:publication_resource @=@ 0038 @=@ Abstract +dnet:publication_resource @=@ 0038 @=@ Blog +dnet:publication_resource @=@ 0038 @=@ Book Prospectus +dnet:publication_resource @=@ 0038 @=@ Dictionary Entry +dnet:publication_resource @=@ 0038 @=@ Disclosure +dnet:publication_resource @=@ 0038 @=@ Editorial +dnet:publication_resource @=@ 0038 @=@ Editorial ISI/JCR +dnet:publication_resource @=@ 0038 @=@ Editors +dnet:publication_resource @=@ 0038 @=@ Editors (non peer-reviewed) +dnet:publication_resource @=@ 0038 @=@ Editors (peer-reviewed) +dnet:publication_resource @=@ 0038 @=@ Encyclopedia Entry +dnet:publication_resource @=@ 0038 @=@ Entrada de blog +dnet:publication_resource @=@ 0038 @=@ Funding Submission +dnet:publication_resource @=@ 0038 @=@ HabilitationThesis +dnet:publication_resource @=@ 0038 @=@ License +dnet:publication_resource @=@ 0038 @=@ Manual +dnet:publication_resource @=@ 0038 @=@ Manuscript +dnet:publication_resource @=@ 0038 @=@ Manuscrito +dnet:publication_resource @=@ 0038 @=@ Other publication (non peer-review) +dnet:publication_resource @=@ 0038 @=@ Other publication (peer-review) +dnet:publication_resource @=@ 0038 @=@ Revista +dnet:publication_resource @=@ 0038 @=@ Supervised Student Publication +dnet:publication_resource @=@ 0038 @=@ Tesis/trabajos de grado – Thesis +dnet:publication_resource @=@ 0038 @=@ Text +dnet:publication_resource @=@ 0038 @=@ Text/Text +dnet:publication_resource @=@ 0038 @=@ Trademark +dnet:publication_resource @=@ 0038 @=@ Translation +dnet:publication_resource @=@ 0038 @=@ afterword +dnet:publication_resource @=@ 0038 @=@ avantpropos +dnet:publication_resource @=@ 0038 @=@ bibliography +dnet:publication_resource @=@ 0038 @=@ chronique +dnet:publication_resource @=@ 0038 @=@ compte rendu +dnet:publication_resource @=@ 0038 @=@ correction +dnet:publication_resource @=@ 0038 @=@ foreword +dnet:publication_resource @=@ 0038 @=@ habilitation à diriger des recherches +dnet:publication_resource @=@ 0038 @=@ historicalDocument +dnet:publication_resource @=@ 0038 @=@ http://purl.org/coar/resource_type/c_0040 +dnet:publication_resource @=@ 0038 @=@ http://purl.org/coar/resource_type/c_0857 +dnet:publication_resource @=@ 0038 @=@ http://purl.org/coar/resource_type/c_18cf +dnet:publication_resource @=@ 0038 @=@ http://purl.org/coar/resource_type/c_18wz +dnet:publication_resource @=@ 0038 @=@ http://purl.org/coar/resource_type/c_3e5a +dnet:publication_resource @=@ 0038 @=@ http://purl.org/coar/resource_type/c_46ec +dnet:publication_resource @=@ 0038 @=@ http://purl.org/coar/resource_type/c_6947 +dnet:publication_resource @=@ 0038 @=@ http://purl.org/coar/resource_type/c_7acd +dnet:publication_resource @=@ 0038 @=@ http://purl.org/coar/resource_type/c_86bc +dnet:publication_resource @=@ 0038 @=@ http://purl.org/coar/resource_type/c_b239 +dnet:publication_resource @=@ 0038 @=@ note de lecture +dnet:publication_resource @=@ 0038 @=@ notedelecture +dnet:publication_resource @=@ 0038 @=@ other publication +dnet:publication_resource @=@ 0038 @=@ postface +dnet:publication_resource @=@ 0038 @=@ publication-other +dnet:publication_resource @=@ 0038 @=@ revuedepresse +dnet:publication_resource @=@ 0038 @=@ sa_component +dnet:publication_resource @=@ 0038 @=@ standard +dnet:publication_resource @=@ 0038 @=@ standard-series +dnet:publication_resource @=@ 0013 @=@ A3 Kirjan tai muun kokoomateoksen osa +dnet:publication_resource @=@ 0013 @=@ Book Part (author) +dnet:publication_resource @=@ 0013 @=@ Book Section / Chapter +dnet:publication_resource @=@ 0013 @=@ Book chapter or Essay in book +dnet:publication_resource @=@ 0013 @=@ Book editorial +dnet:publication_resource @=@ 0013 @=@ Book section +dnet:publication_resource @=@ 0013 @=@ Book_Chapter +dnet:publication_resource @=@ 0013 @=@ Buchbeitrag +dnet:publication_resource @=@ 0013 @=@ Capítulo de libro +dnet:publication_resource @=@ 0013 @=@ Contribution to International Book/Monograph +dnet:publication_resource @=@ 0013 @=@ Contribution to International Book/Monograph ISI/JCR +dnet:publication_resource @=@ 0013 @=@ Contribution to National Book/Monograph +dnet:publication_resource @=@ 0013 @=@ Contribution to book (non peer-reviewed) +dnet:publication_resource @=@ 0013 @=@ Contribution to book (peer-reviewed) +dnet:publication_resource @=@ 0013 @=@ Part of book - chapter +dnet:publication_resource @=@ 0013 @=@ book chapter +dnet:publication_resource @=@ 0013 @=@ book-part +dnet:publication_resource @=@ 0013 @=@ bookPart +dnet:publication_resource @=@ 0013 @=@ book_content +dnet:publication_resource @=@ 0013 @=@ chapitre_ouvrage +dnet:publication_resource @=@ 0013 @=@ chapter +dnet:publication_resource @=@ 0013 @=@ doc-type:bookPart +dnet:publication_resource @=@ 0013 @=@ http://purl.org/coar/resource_type/c_3248 +dnet:publication_resource @=@ 0013 @=@ info:eu-repo/semantics/bookPart +dnet:publication_resource @=@ 0013 @=@ publication-section +dnet:publication_resource @=@ 0013 @=@ reference-entry +dnet:publication_resource @=@ 0013 @=@ reference_entry +dnet:publication_resource @=@ 0013 @=@ scientific book chapter +dnet:publication_resource @=@ 0013 @=@ Глава монографии +dnet:publication_resource @=@ 0019 @=@ H1 Myönnetty patentti +dnet:publication_resource @=@ 0019 @=@ Patent +dnet:publication_resource @=@ 0019 @=@ Patente +dnet:publication_resource @=@ 0019 @=@ Solicitud de patente +dnet:publication_resource @=@ 0019 @=@ Traducción de patente +dnet:publication_resource @=@ 0019 @=@ brevet +dnet:publication_resource @=@ 0019 @=@ http://purl.org/coar/resource_type/c_15cd +dnet:publication_resource @=@ 0019 @=@ info:eu-repo/semantics/patent +dnet:publication_resource @=@ 0019 @=@ publication-patent +dnet:publication_resource @=@ 0028 @=@ Service +dnet:publication_resource @=@ 0028 @=@ physicalObject +dnet:publication_resource @=@ 0016 @=@ Pre Print +dnet:publication_resource @=@ 0016 @=@ Pre-print +dnet:publication_resource @=@ 0016 @=@ http://purl.org/coar/resource_type/c_816b +dnet:publication_resource @=@ 0016 @=@ info:eu-repo/semantics/preprint +dnet:publication_resource @=@ 0016 @=@ publication-preprint +dnet:publication_resource @=@ 0016 @=@ Препринт +dnet:publication_resource @=@ 0034 @=@ Project deliverable +dnet:publication_resource @=@ 0034 @=@ http://purl.org/coar/resource_type/c_18op +dnet:publication_resource @=@ 0034 @=@ publication-deliverable +dnet:publication_resource @=@ 0035 @=@ Project milestone +dnet:publication_resource @=@ 0035 @=@ publication-milestone +dnet:publication_resource @=@ 0036 @=@ Proposal +dnet:publication_resource @=@ 0036 @=@ http://purl.org/coar/resource_type/c_baaf +dnet:publication_resource @=@ 0036 @=@ research-proposal +dnet:publication_resource @=@ 0017 @=@ ACTIVITY_REPORT +dnet:publication_resource @=@ 0017 @=@ Commissioned report +dnet:publication_resource @=@ 0017 @=@ D4 Julkaistu kehittämis- tai tutkimusraportti tai -selvitys +dnet:publication_resource @=@ 0017 @=@ Deliverable +dnet:publication_resource @=@ 0017 @=@ Documento tecnico +dnet:publication_resource @=@ 0017 @=@ Project Report +dnet:publication_resource @=@ 0017 @=@ Software documentation +dnet:publication_resource @=@ 0017 @=@ brief-report +dnet:publication_resource @=@ 0017 @=@ case-report +dnet:publication_resource @=@ 0017 @=@ chapitre_rapport +dnet:publication_resource @=@ 0017 @=@ doc-type:report +dnet:publication_resource @=@ 0017 @=@ document_institutionnel +dnet:publication_resource @=@ 0017 @=@ document_technique +dnet:publication_resource @=@ 0017 @=@ http://purl.org/coar/resource_type/c_186u +dnet:publication_resource @=@ 0017 @=@ http://purl.org/coar/resource_type/c_18hj +dnet:publication_resource @=@ 0017 @=@ http://purl.org/coar/resource_type/c_18wq +dnet:publication_resource @=@ 0017 @=@ http://purl.org/coar/resource_type/c_18ws +dnet:publication_resource @=@ 0017 @=@ http://purl.org/coar/resource_type/c_71bd +dnet:publication_resource @=@ 0017 @=@ http://purl.org/coar/resource_type/c_93fc +dnet:publication_resource @=@ 0017 @=@ http://purl.org/coar/resource_type/c_ba1f +dnet:publication_resource @=@ 0017 @=@ info:eu-repo/semantics/report +dnet:publication_resource @=@ 0017 @=@ publication-report +dnet:publication_resource @=@ 0017 @=@ publication-softwaredocumentation +dnet:publication_resource @=@ 0017 @=@ rapport_expertise +dnet:publication_resource @=@ 0017 @=@ rapport_mission +dnet:publication_resource @=@ 0017 @=@ report +dnet:publication_resource @=@ 0017 @=@ report-paper +dnet:publication_resource @=@ 0017 @=@ report-paper_title +dnet:publication_resource @=@ 0017 @=@ report-series +dnet:publication_resource @=@ 0017 @=@ support_cours +dnet:publication_resource @=@ 0014 @=@ Arbeitspapier +dnet:publication_resource @=@ 0014 @=@ Departmental Bulletin Paper +dnet:publication_resource @=@ 0014 @=@ Documento de trabajo +dnet:publication_resource @=@ 0014 @=@ Paper +dnet:publication_resource @=@ 0014 @=@ Project description +dnet:publication_resource @=@ 0014 @=@ Research-Paper +dnet:publication_resource @=@ 0014 @=@ ResearchPaper +dnet:publication_resource @=@ 0014 @=@ Working / discussion paper +dnet:publication_resource @=@ 0014 @=@ Working Paper +dnet:publication_resource @=@ 0014 @=@ Working Paper / Technical Report +dnet:publication_resource @=@ 0014 @=@ doc-type:workingPaper +dnet:publication_resource @=@ 0014 @=@ http://purl.org/coar/resource_type/c_8042 +dnet:publication_resource @=@ 0014 @=@ info:eu-repo/semantics/paper +dnet:publication_resource @=@ 0014 @=@ info:eu-repo/semantics/workingPaper +dnet:publication_resource @=@ 0014 @=@ publication-workingpaper +dnet:publication_resource @=@ 0014 @=@ workingPaper +dnet:publication_resource @=@ 0015 @=@ A2 Katsausartikkeli tieteellisessä aikakauslehdessä +dnet:publication_resource @=@ 0015 @=@ Book Review +dnet:publication_resource @=@ 0015 @=@ Book/Film/Article review +dnet:publication_resource @=@ 0015 @=@ Literature review +dnet:publication_resource @=@ 0015 @=@ Peer review +dnet:publication_resource @=@ 0015 @=@ Reseña bibliográfica +dnet:publication_resource @=@ 0015 @=@ Review Article +dnet:publication_resource @=@ 0015 @=@ RezensionReview +dnet:publication_resource @=@ 0015 @=@ book-review +dnet:publication_resource @=@ 0015 @=@ http://purl.org/coar/resource_type/c_ba08 +dnet:publication_resource @=@ 0015 @=@ http://purl.org/coar/resource_type/c_dcae04bc +dnet:publication_resource @=@ 0015 @=@ http://purl.org/coar/resource_type/c_efa0 +dnet:publication_resource @=@ 0015 @=@ info:eu-repo/semantics/review +dnet:publication_resource @=@ 0015 @=@ peer-review +dnet:publication_resource @=@ 0029 @=@ Software +dnet:publication_resource @=@ 0029 @=@ Software/Software +dnet:publication_resource @=@ 0029 @=@ Workflow +dnet:publication_resource @=@ 0029 @=@ Workflow/Workflow +dnet:publication_resource @=@ 0029 @=@ http://purl.org/coar/resource_type/c_393c +dnet:publication_resource @=@ 0029 @=@ http://purl.org/coar/resource_type/c_5ce6 +dnet:publication_resource @=@ 0029 @=@ http://purl.org/coar/resource_type/c_c950 +dnet:publication_resource @=@ 0032 @=@ http://purl.org/coar/resource_type/c_7bab +dnet:publication_resource @=@ 0030 @=@ http://purl.org/coar/resource_type/c_18cc +dnet:publication_resource @=@ 0030 @=@ sound +dnet:publication_resource @=@ 0044 @=@ Graduate diploma +dnet:publication_resource @=@ 0044 @=@ Undergraduate diploma +dnet:publication_resource @=@ 0000 @=@ UNKNOWN +dnet:publication_resource @=@ 0042 @=@ EGI Virtual Appliance +dnet:languages @=@ abk @=@ ab +dnet:languages @=@ aar @=@ aa +dnet:languages @=@ afr @=@ af +dnet:languages @=@ alb/sqi @=@ sq +dnet:languages @=@ amh @=@ am +dnet:languages @=@ ara @=@ ar +dnet:languages @=@ arm/hye @=@ hy +dnet:languages @=@ asm @=@ as +dnet:languages @=@ ina @=@ ia +dnet:languages @=@ aym @=@ ay +dnet:languages @=@ aze @=@ az +dnet:languages @=@ bak @=@ ba +dnet:languages @=@ baq/eus @=@ eu +dnet:languages @=@ bel @=@ be +dnet:languages @=@ ben @=@ bn +dnet:languages @=@ bih @=@ bh +dnet:languages @=@ bis @=@ bi +dnet:languages @=@ bre @=@ br +dnet:languages @=@ bul @=@ bg +dnet:languages @=@ bur/mya @=@ my +dnet:languages @=@ cat @=@ ca +dnet:languages @=@ chi/zho @=@ zh +dnet:languages @=@ cos @=@ co +dnet:languages @=@ hrv @=@ hr +dnet:languages @=@ hrv @=@ hr +dnet:languages @=@ hrv @=@ scr/hrv +dnet:languages @=@ ces/cze @=@ cs +dnet:languages @=@ dan @=@ da +dnet:languages @=@ dut/nld @=@ dut/nla +dnet:languages @=@ dut/nld @=@ dutdut +dnet:languages @=@ dut/nld @=@ nl +dnet:languages @=@ dut/nld @=@ nl_be +dnet:languages @=@ dut/nld @=@ nl_nl +dnet:languages @=@ dut/nld @=@ nld +dnet:languages @=@ dzo @=@ dz +dnet:languages @=@ eng @=@ en +dnet:languages @=@ eng @=@ en_au +dnet:languages @=@ eng @=@ en_en +dnet:languages @=@ eng @=@ en_gb +dnet:languages @=@ eng @=@ en_nz +dnet:languages @=@ eng @=@ en_us +dnet:languages @=@ eng @=@ english +dnet:languages @=@ eng @=@ en-us +dnet:languages @=@ eng @=@ en-US +dnet:languages @=@ eng @=@ English +dnet:languages @=@ eng @=@ EN +dnet:languages @=@ eng @=@ en angielski +dnet:languages @=@ eng @=@ en-GB +dnet:languages @=@ eng @=@ Englisch +dnet:languages @=@ epo @=@ eo +dnet:languages @=@ est @=@ et +dnet:languages @=@ fao @=@ fo +dnet:languages @=@ fij @=@ fj +dnet:languages @=@ fin @=@ fi +dnet:languages @=@ fin @=@ Finnish +dnet:languages @=@ fra/fre @=@ fr +dnet:languages @=@ fra/fre @=@ FR +dnet:languages @=@ fra/fre @=@ fr_be +dnet:languages @=@ fra/fre @=@ fr_fr +dnet:languages @=@ fra/fre @=@ fre/fra +dnet:languages @=@ fra/fre @=@ fra +dnet:languages @=@ fry @=@ fy +dnet:languages @=@ glg @=@ gl +dnet:languages @=@ geo/kat @=@ ka +dnet:languages @=@ deu/ger @=@ de +dnet:languages @=@ deu/ger @=@ ger/deu +dnet:languages @=@ deu/ger @=@ german +dnet:languages @=@ deu/ger @=@ ger +dnet:languages @=@ deu/ger @=@ deu +dnet:languages @=@ deu/ger @=@ DE-de +dnet:languages @=@ ell/gre @=@ el +dnet:languages @=@ ell/gre @=@ gr +dnet:languages @=@ ell/gre @=@ el-GR +dnet:languages @=@ kal @=@ kl +dnet:languages @=@ grn @=@ gn +dnet:languages @=@ guj @=@ gu +dnet:languages @=@ hau @=@ ha +dnet:languages @=@ heb @=@ he +dnet:languages @=@ hin @=@ hi +dnet:languages @=@ hun @=@ hu +dnet:languages @=@ ice/isl @=@ is +dnet:languages @=@ ine @=@ - +dnet:languages @=@ ind @=@ id +dnet:languages @=@ iku @=@ iu +dnet:languages @=@ ipk @=@ ik +dnet:languages @=@ gai/iri @=@ ga +dnet:languages @=@ gai/iri @=@ gle +dnet:languages @=@ ita @=@ it +dnet:languages @=@ jpn @=@ ja +dnet:languages @=@ jav @=@ jv +dnet:languages @=@ jav @=@ jv/jw +dnet:languages @=@ jav @=@ jw +dnet:languages @=@ kan @=@ kn +dnet:languages @=@ kas @=@ ks +dnet:languages @=@ kaz @=@ kk +dnet:languages @=@ khm @=@ km +dnet:languages @=@ kin @=@ rw +dnet:languages @=@ kir @=@ ky +dnet:languages @=@ kor @=@ ko +dnet:languages @=@ kur @=@ ku +dnet:languages @=@ lao @=@ lo +dnet:languages @=@ lat @=@ la +dnet:languages @=@ lav @=@ lv +dnet:languages @=@ lin @=@ ln +dnet:languages @=@ lit @=@ lt +dnet:languages @=@ mac/mak @=@ mk +dnet:languages @=@ mlg @=@ mg +dnet:languages @=@ may/msa @=@ ms +dnet:languages @=@ mlt @=@ ml +dnet:languages @=@ mao/mri @=@ mi +dnet:languages @=@ mar @=@ mr +dnet:languages @=@ mol @=@ mo +dnet:languages @=@ mon @=@ mn +dnet:languages @=@ nau @=@ na +dnet:languages @=@ nep @=@ ne +dnet:languages @=@ nor @=@ no +dnet:languages @=@ oci @=@ oc +dnet:languages @=@ ori @=@ or +dnet:languages @=@ orm @=@ om +dnet:languages @=@ pan @=@ pa +dnet:languages @=@ fas/per @=@ fa +dnet:languages @=@ pol @=@ pl +dnet:languages @=@ por @=@ pt +dnet:languages @=@ por @=@ pt_pt +dnet:languages @=@ pus @=@ ps +dnet:languages @=@ que @=@ qu +dnet:languages @=@ roh @=@ rm +dnet:languages @=@ ron/rum @=@ ro +dnet:languages @=@ run @=@ rn +dnet:languages @=@ rus @=@ ru +dnet:languages @=@ smo @=@ sm +dnet:languages @=@ sag @=@ sg +dnet:languages @=@ san @=@ sa +dnet:languages @=@ srp @=@ scc/srp +dnet:languages @=@ srp @=@ sr +dnet:languages @=@ scr @=@ sh +dnet:languages @=@ sna @=@ sn +dnet:languages @=@ snd @=@ sd +dnet:languages @=@ sin @=@ si +dnet:languages @=@ sit @=@ - +dnet:languages @=@ slk/slo @=@ sk +dnet:languages @=@ slv @=@ sl +dnet:languages @=@ som @=@ so +dnet:languages @=@ sot @=@ st +dnet:languages @=@ esl/spa @=@ es +dnet:languages @=@ sun @=@ su +dnet:languages @=@ swa @=@ sw +dnet:languages @=@ ssw @=@ ss +dnet:languages @=@ swe @=@ sv +dnet:languages @=@ swe @=@ sve/swe +dnet:languages @=@ tgl @=@ tl +dnet:languages @=@ tgk @=@ tg +dnet:languages @=@ tam @=@ ta +dnet:languages @=@ tat @=@ tt +dnet:languages @=@ tel @=@ te +dnet:languages @=@ tha @=@ th +dnet:languages @=@ tha @=@ thai +dnet:languages @=@ bod/tib @=@ bo +dnet:languages @=@ tir @=@ ti +dnet:languages @=@ tog @=@ to +dnet:languages @=@ tso @=@ ts +dnet:languages @=@ tsn @=@ tn +dnet:languages @=@ tur @=@ tr +dnet:languages @=@ tuk @=@ tk +dnet:languages @=@ twi @=@ tw +dnet:languages @=@ uig @=@ ug +dnet:languages @=@ ukr @=@ uk +dnet:languages @=@ und @=@ UNKNOWN +dnet:languages @=@ und @=@ none +dnet:languages @=@ urd @=@ ur +dnet:languages @=@ uzb @=@ uz +dnet:languages @=@ vie @=@ vi +dnet:languages @=@ vol @=@ vo +dnet:languages @=@ wln @=@ wa +dnet:languages @=@ cym/wel @=@ cy +dnet:languages @=@ wol @=@ wo +dnet:languages @=@ xho @=@ xh +dnet:languages @=@ yid @=@ yi +dnet:languages @=@ yor @=@ yo +dnet:languages @=@ zha @=@ za +dnet:languages @=@ zul @=@ zu +dnet:result_typologies @=@ dataset @=@ 0021 +dnet:result_typologies @=@ dataset @=@ 0024 +dnet:result_typologies @=@ dataset @=@ 0025 +dnet:result_typologies @=@ dataset @=@ 0030 +dnet:result_typologies @=@ dataset @=@ 0033 +dnet:result_typologies @=@ dataset @=@ 0037 +dnet:result_typologies @=@ dataset @=@ 0039 +dnet:result_typologies @=@ dataset @=@ 0046 +dnet:result_typologies @=@ other @=@ 0000 +dnet:result_typologies @=@ other @=@ 0010 +dnet:result_typologies @=@ other @=@ 0018 +dnet:result_typologies @=@ other @=@ 0020 +dnet:result_typologies @=@ other @=@ 0022 +dnet:result_typologies @=@ other @=@ 0023 +dnet:result_typologies @=@ other @=@ 0026 +dnet:result_typologies @=@ other @=@ 0027 +dnet:result_typologies @=@ other @=@ 0028 +dnet:result_typologies @=@ other @=@ 0042 +dnet:result_typologies @=@ publication @=@ 0001 +dnet:result_typologies @=@ publication @=@ 0002 +dnet:result_typologies @=@ publication @=@ 0004 +dnet:result_typologies @=@ publication @=@ 0005 +dnet:result_typologies @=@ publication @=@ 0006 +dnet:result_typologies @=@ publication @=@ 0007 +dnet:result_typologies @=@ publication @=@ 0008 +dnet:result_typologies @=@ publication @=@ 0009 +dnet:result_typologies @=@ publication @=@ 0011 +dnet:result_typologies @=@ publication @=@ 0012 +dnet:result_typologies @=@ publication @=@ 0013 +dnet:result_typologies @=@ publication @=@ 0014 +dnet:result_typologies @=@ publication @=@ 0015 +dnet:result_typologies @=@ publication @=@ 0016 +dnet:result_typologies @=@ publication @=@ 0017 +dnet:result_typologies @=@ publication @=@ 0019 +dnet:result_typologies @=@ publication @=@ 0031 +dnet:result_typologies @=@ publication @=@ 0032 +dnet:result_typologies @=@ publication @=@ 0034 +dnet:result_typologies @=@ publication @=@ 0035 +dnet:result_typologies @=@ publication @=@ 0036 +dnet:result_typologies @=@ publication @=@ 0038 +dnet:result_typologies @=@ publication @=@ 0044 +dnet:result_typologies @=@ publication @=@ 0045 +dnet:result_typologies @=@ software @=@ 0029 +dnet:result_typologies @=@ software @=@ 0040 +dnet:countries @=@ AF @=@ AFG +dnet:countries @=@ AF @=@ Afghanistan +dnet:countries @=@ AD @=@ Andorra +dnet:countries @=@ AO @=@ Angola +dnet:countries @=@ AR @=@ ARG +dnet:countries @=@ AR @=@ Argentina +dnet:countries @=@ AU @=@ AUS +dnet:countries @=@ AU @=@ Australia +dnet:countries @=@ AT @=@ AUT +dnet:countries @=@ AT @=@ Austria +dnet:countries @=@ AZ @=@ AZE +dnet:countries @=@ BD @=@ Bangladesh +dnet:countries @=@ BY @=@ Belarus +dnet:countries @=@ BE @=@ BEL +dnet:countries @=@ BE @=@ Belgium +dnet:countries @=@ BJ @=@ BEN +dnet:countries @=@ BO @=@ Bolivia, Plurinational State of +dnet:countries @=@ BA @=@ BIH +dnet:countries @=@ BA @=@ Bosnia-Hercegovina +dnet:countries @=@ BR @=@ BRA +dnet:countries @=@ BR @=@ Brazil +dnet:countries @=@ BG @=@ Bulgaria +dnet:countries @=@ BF @=@ BFA +dnet:countries @=@ KH @=@ Cambodia +dnet:countries @=@ KH @=@ Cambogia +dnet:countries @=@ KH @=@ Campuchea +dnet:countries @=@ CM @=@ CMR +dnet:countries @=@ CA @=@ CAN +dnet:countries @=@ CA @=@ Canada +dnet:countries @=@ CV @=@ Cape Verde +dnet:countries @=@ CL @=@ CHL +dnet:countries @=@ CL @=@ Chile +dnet:countries @=@ CN @=@ CHN +dnet:countries @=@ CN @=@ China +dnet:countries @=@ CO @=@ COL +dnet:countries @=@ CO @=@ Colombia +dnet:countries @=@ CD @=@ Congo +dnet:countries @=@ CD @=@ Congo Democratic Republic (formerly Zaire) +dnet:countries @=@ CD @=@ Congo, Republic +dnet:countries @=@ CD @=@ Congo, the Democratic Republic of the +dnet:countries @=@ CD @=@ Zaire +dnet:countries @=@ CR @=@ CRI +dnet:countries @=@ CI @=@ CIV +dnet:countries @=@ CI @=@ Ivory Coast +dnet:countries @=@ HR @=@ Croatia +dnet:countries @=@ HR @=@ HRV +dnet:countries @=@ CY @=@ CYP +dnet:countries @=@ CY @=@ Cyprus +dnet:countries @=@ CZ @=@ CZE +dnet:countries @=@ CZ @=@ Czech Republic +dnet:countries @=@ CZ @=@ Czechia +dnet:countries @=@ CZ @=@ Czechoslovakia +dnet:countries @=@ DK @=@ DNK +dnet:countries @=@ DK @=@ Denmark +dnet:countries @=@ EC @=@ Ecuador +dnet:countries @=@ EG @=@ EGY +dnet:countries @=@ EG @=@ Egypt +dnet:countries @=@ SV @=@ SLV +dnet:countries @=@ EE @=@ EST +dnet:countries @=@ EE @=@ Estonia +dnet:countries @=@ ET @=@ ETH +dnet:countries @=@ EU @=@ EEC +dnet:countries @=@ FJ @=@ FJI +dnet:countries @=@ FI @=@ FIN +dnet:countries @=@ FI @=@ Finland +dnet:countries @=@ MK @=@ Macedonia +dnet:countries @=@ MK @=@ Macedonia, the Former Yugoslav Republic Of +dnet:countries @=@ MK @=@ North Macedonia +dnet:countries @=@ FR @=@ FRA +dnet:countries @=@ FR @=@ France +dnet:countries @=@ PF @=@ French Polynesia +dnet:countries @=@ PF @=@ PYF +dnet:countries @=@ TF @=@ French Southern Territories +dnet:countries @=@ GE @=@ Georgia +dnet:countries @=@ DE @=@ DEU +dnet:countries @=@ DE @=@ Germany +dnet:countries @=@ DE @=@ Germany, Berlin +dnet:countries @=@ GH @=@ GHA +dnet:countries @=@ GR @=@ EL +dnet:countries @=@ GR @=@ GRC +dnet:countries @=@ GL @=@ GRL +dnet:countries @=@ GN @=@ Guinea +dnet:countries @=@ GW @=@ Guinea-Bissau +dnet:countries @=@ VA @=@ Vatican State +dnet:countries @=@ HK @=@ HKG +dnet:countries @=@ HK @=@ Hong Kong +dnet:countries @=@ HK @=@ Hongkong +dnet:countries @=@ HU @=@ HUN +dnet:countries @=@ HU @=@ Hungary +dnet:countries @=@ IS @=@ ISL +dnet:countries @=@ IN @=@ IND +dnet:countries @=@ IN @=@ India +dnet:countries @=@ ID @=@ IDN +dnet:countries @=@ ID @=@ Indonesia +dnet:countries @=@ IR @=@ Iran +dnet:countries @=@ IR @=@ Iran, Islamic Republic of +dnet:countries @=@ IE @=@ IRL +dnet:countries @=@ IE @=@ Ireland +dnet:countries @=@ IL @=@ ISR +dnet:countries @=@ IL @=@ Israel +dnet:countries @=@ IT @=@ ITA +dnet:countries @=@ IT @=@ Italy +dnet:countries @=@ JM @=@ Jamaica +dnet:countries @=@ JP @=@ JPN +dnet:countries @=@ JP @=@ Japan +dnet:countries @=@ KZ @=@ KAZ +dnet:countries @=@ KZ @=@ Kazakistan +dnet:countries @=@ KZ @=@ Kazakstan +dnet:countries @=@ KE @=@ KEN +dnet:countries @=@ KE @=@ Kenya +dnet:countries @=@ KR @=@ KOR +dnet:countries @=@ KR @=@ Korea, Republic of +dnet:countries @=@ KR @=@ Korean Republic (South Korea) +dnet:countries @=@ KP @=@ PRK +dnet:countries @=@ LV @=@ LVA +dnet:countries @=@ LY @=@ Libya +dnet:countries @=@ LT @=@ LTU +dnet:countries @=@ LU @=@ LUX +dnet:countries @=@ LU @=@ Luxembourg +dnet:countries @=@ MO @=@ Macao +dnet:countries @=@ MG @=@ Madagascar +dnet:countries @=@ MY @=@ Malaysia +dnet:countries @=@ ML @=@ Mali +dnet:countries @=@ MT @=@ Malta +dnet:countries @=@ MU @=@ Mauritius +dnet:countries @=@ MX @=@ MEX +dnet:countries @=@ MX @=@ Mexico +dnet:countries @=@ FM @=@ Micronesia +dnet:countries @=@ MD @=@ Moldova +dnet:countries @=@ MD @=@ Moldova, Republic of +dnet:countries @=@ MN @=@ Mongolia +dnet:countries @=@ MA @=@ Morocco +dnet:countries @=@ MZ @=@ Mozambique +dnet:countries @=@ NA @=@ NAM +dnet:countries @=@ NL @=@ NLD +dnet:countries @=@ NL @=@ Netherlands +dnet:countries @=@ AN @=@ Netherlands Antilles +dnet:countries @=@ NC @=@ NCL +dnet:countries @=@ NZ @=@ NZL +dnet:countries @=@ NZ @=@ New Zealand +dnet:countries @=@ NO @=@ NOR +dnet:countries @=@ NO @=@ Norway +dnet:countries @=@ OC @=@ Australasia +dnet:countries @=@ OM @=@ Oman +dnet:countries @=@ PK @=@ PAK +dnet:countries @=@ PK @=@ Pakistan +dnet:countries @=@ PS @=@ Palestin, State of +dnet:countries @=@ PS @=@ Palestine, State of +dnet:countries @=@ PS @=@ Palestinian Territory, Occupied +dnet:countries @=@ PA @=@ PAN +dnet:countries @=@ PA @=@ Panama +dnet:countries @=@ PG @=@ PapuaNew Guinea +dnet:countries @=@ PE @=@ PER +dnet:countries @=@ PH @=@ PHL +dnet:countries @=@ PH @=@ Philippines +dnet:countries @=@ PL @=@ POL +dnet:countries @=@ PL @=@ Poland +dnet:countries @=@ PT @=@ PRT +dnet:countries @=@ PT @=@ Portugal +dnet:countries @=@ PR @=@ Puerto Rico +dnet:countries @=@ RO @=@ ROU +dnet:countries @=@ RO @=@ Romania +dnet:countries @=@ RU @=@ RUS +dnet:countries @=@ RU @=@ Russia +dnet:countries @=@ RU @=@ Russian Federation +dnet:countries @=@ RE @=@ Réunion +dnet:countries @=@ KN @=@ Saint Kitts And Nevis +dnet:countries @=@ SA @=@ Saudi Arabia +dnet:countries @=@ SN @=@ SEN +dnet:countries @=@ RS @=@ SRB +dnet:countries @=@ CS @=@ Serbia and Montenegro +dnet:countries @=@ SG @=@ SGP +dnet:countries @=@ SG @=@ Singapore +dnet:countries @=@ SK @=@ SVK +dnet:countries @=@ SI @=@ SVN +dnet:countries @=@ SI @=@ Slovenia +dnet:countries @=@ ZA @=@ South Africa +dnet:countries @=@ ZA @=@ ZAF +dnet:countries @=@ ES @=@ ESP +dnet:countries @=@ ES @=@ Spain +dnet:countries @=@ LK @=@ LKA +dnet:countries @=@ LK @=@ Sri Lanka +dnet:countries @=@ SD @=@ SDN +dnet:countries @=@ SR @=@ Suriname +dnet:countries @=@ SE @=@ SWE +dnet:countries @=@ SE @=@ Sweden +dnet:countries @=@ CH @=@ CHE +dnet:countries @=@ CH @=@ Switzerland +dnet:countries @=@ SY @=@ Syria +dnet:countries @=@ ST @=@ Sao Tome and Principe +dnet:countries @=@ TW @=@ TWN +dnet:countries @=@ TW @=@ Taiwan +dnet:countries @=@ TW @=@ Taiwan, Province of China +dnet:countries @=@ TZ @=@ Tanzania +dnet:countries @=@ TZ @=@ Tanzania, United Republic of +dnet:countries @=@ TH @=@ THA +dnet:countries @=@ TH @=@ Thailand +dnet:countries @=@ TL @=@ East Timor +dnet:countries @=@ TN @=@ TUN +dnet:countries @=@ TN @=@ Tunisia +dnet:countries @=@ TR @=@ TUR +dnet:countries @=@ TR @=@ Turkey +dnet:countries @=@ UNKNOWN @=@ AAA +dnet:countries @=@ UNKNOWN @=@ [Unknown] +dnet:countries @=@ UNKNOWN @=@ _? +dnet:countries @=@ UA @=@ UKR +dnet:countries @=@ UA @=@ Ukraine +dnet:countries @=@ AE @=@ United Arab Emirates +dnet:countries @=@ GB @=@ England +dnet:countries @=@ GB @=@ GBR +dnet:countries @=@ GB @=@ Great Britain +dnet:countries @=@ GB @=@ Great Britain and Northern Ireland +dnet:countries @=@ GB @=@ Scotland +dnet:countries @=@ GB @=@ UK +dnet:countries @=@ GB @=@ United Kingdom +dnet:countries @=@ US @=@ USA +dnet:countries @=@ US @=@ United States +dnet:countries @=@ US @=@ United States of America +dnet:countries @=@ UY @=@ Uruguay +dnet:countries @=@ UZ @=@ Uzbekistan +dnet:countries @=@ VE @=@ Venezuela, Bolivarian Republic of +dnet:countries @=@ VN @=@ Vietnam +dnet:countries @=@ VG @=@ British Virgin Islands +dnet:countries @=@ YU @=@ Jugoslavia +dnet:countries @=@ YU @=@ Yugoslavia +dnet:countries @=@ ZW @=@ ABW +dnet:protocols @=@ oai @=@ OAI-PMH +dnet:protocols @=@ oai @=@ OAI_PMH +dnet:pid_types @=@ orcid @=@ ORCID12 +dnet:pid_types @=@ handle @=@ hdl +dnet:review_levels @=@ 0000 @=@ UNKNOWN +dnet:review_levels @=@ 0002 @=@ 80 大阪経大学会「Working Paper」 +dnet:review_levels @=@ 0002 @=@ AO +dnet:review_levels @=@ 0002 @=@ ARTICLE SANS COMITE DE LECTURE (ASCL) +dnet:review_levels @=@ 0002 @=@ Arbeitspapier +dnet:review_levels @=@ 0002 @=@ Arbeitspapier [workingPaper] +dnet:review_levels @=@ 0002 @=@ Article (author) +dnet:review_levels @=@ 0002 @=@ Article type: preprint +dnet:review_levels @=@ 0002 @=@ Article(author version) +dnet:review_levels @=@ 0002 @=@ Article, not peer-reviewed +dnet:review_levels @=@ 0002 @=@ Articulo no evaluado +dnet:review_levels @=@ 0002 @=@ Artigo Solicitado e Não Avaliado por Pares +dnet:review_levels @=@ 0002 @=@ Artigo não avaliado pelos pares +dnet:review_levels @=@ 0002 @=@ Artigo não avaliado por pares +dnet:review_levels @=@ 0002 @=@ Artigo não avaliado por pres +dnet:review_levels @=@ 0002 @=@ Artikkeli|Artikkeli ammattilehdessä. Ei vertaisarvioitu +dnet:review_levels @=@ 0002 @=@ Artículo no evaluado +dnet:review_levels @=@ 0002 @=@ Book (non peer-reviewed) +dnet:review_levels @=@ 0002 @=@ Book Part (author) +dnet:review_levels @=@ 0002 @=@ Book item; Non-peer-reviewed +dnet:review_levels @=@ 0002 @=@ Conference preprint +dnet:review_levels @=@ 0002 @=@ Contribution to book (non peer-reviewed) +dnet:review_levels @=@ 0002 @=@ Discussion Paper +dnet:review_levels @=@ 0002 @=@ Document de travail (Working Paper) +dnet:review_levels @=@ 0002 @=@ Documento de trabajo +dnet:review_levels @=@ 0002 @=@ Documento de trabajo de investigaci??n +dnet:review_levels @=@ 0002 @=@ Draft +dnet:review_levels @=@ 0002 @=@ E-pub ahead of print +dnet:review_levels @=@ 0002 @=@ Editorial de revista, no evaluado por pares +dnet:review_levels @=@ 0002 @=@ Editorial de revista, não avaliado por pares +dnet:review_levels @=@ 0002 @=@ Editorial não avaliado pelos pares +dnet:review_levels @=@ 0002 @=@ Editors (non peer-reviewed) +dnet:review_levels @=@ 0002 @=@ Epub ahead of print +dnet:review_levels @=@ 0002 @=@ Hakemlik Sürecinden Geçmiş Makale +dnet:review_levels @=@ 0002 @=@ Hakemlik sürecindeki makale +dnet:review_levels @=@ 0002 @=@ Hakemlik sürecinden geçmemiş kitap değerlendirmesi +dnet:review_levels @=@ 0002 @=@ Journal Article (author version) +dnet:review_levels @=@ 0002 @=@ Journal Article Preprint +dnet:review_levels @=@ 0002 @=@ Journal Editorial, not peer-reviewed +dnet:review_levels @=@ 0002 @=@ Journal article; Non-peer-reviewed +dnet:review_levels @=@ 0002 @=@ Journal:WorkingPaper +dnet:review_levels @=@ 0002 @=@ Manuscript (preprint) +dnet:review_levels @=@ 0002 @=@ Monográfico (Informes, Documentos de trabajo, etc.) +dnet:review_levels @=@ 0002 @=@ NOTE INTERNE OU DE TRAVAIL +dnet:review_levels @=@ 0002 @=@ Nicht begutachteter Beitrag +dnet:review_levels @=@ 0002 @=@ No evaluado por pares +dnet:review_levels @=@ 0002 @=@ Non-Refereed +dnet:review_levels @=@ 0002 @=@ Non-refeered article +dnet:review_levels @=@ 0002 @=@ Non-refereed Article +dnet:review_levels @=@ 0002 @=@ Non-refereed Book Review +dnet:review_levels @=@ 0002 @=@ Non-refereed Review +dnet:review_levels @=@ 0002 @=@ Non-refereed Text +dnet:review_levels @=@ 0002 @=@ NonPeerReviewed +dnet:review_levels @=@ 0002 @=@ Not Peer reviewed +dnet:review_levels @=@ 0002 @=@ Not Reviewed +dnet:review_levels @=@ 0002 @=@ Not peer-reviewed +dnet:review_levels @=@ 0002 @=@ Não Avaliado por Pares +dnet:review_levels @=@ 0002 @=@ Não avaliada pelos pares +dnet:review_levels @=@ 0002 @=@ Não avaliado pelos pares +dnet:review_levels @=@ 0002 @=@ Original article (non peer-reviewed) +dnet:review_levels @=@ 0002 @=@ Other publication (non peer-review) +dnet:review_levels @=@ 0002 @=@ Pre Print +dnet:review_levels @=@ 0002 @=@ Pre-print +dnet:review_levels @=@ 0002 @=@ Preprint Article +dnet:review_levels @=@ 0002 @=@ Preprints +dnet:review_levels @=@ 0002 @=@ Preprints, Working Papers, ... +dnet:review_levels @=@ 0002 @=@ Rapporto tecnico / Working Paper / Rapporto di progetto +dnet:review_levels @=@ 0002 @=@ Resumo Não Avaliado por Pares +dnet:review_levels @=@ 0002 @=@ Review article (non peer-reviewed) +dnet:review_levels @=@ 0002 @=@ SMUR +dnet:review_levels @=@ 0002 @=@ Submissão dos artigos +dnet:review_levels @=@ 0002 @=@ Submitted version +dnet:review_levels @=@ 0002 @=@ Vertaisarvioimaton kirjan tai muun kokoomateoksen osa +dnet:review_levels @=@ 0002 @=@ Vorabdruck +dnet:review_levels @=@ 0002 @=@ Wetensch. publ. non-refereed +dnet:review_levels @=@ 0002 @=@ Working / discussion paper +dnet:review_levels @=@ 0002 @=@ Working Document +dnet:review_levels @=@ 0002 @=@ Working Notes +dnet:review_levels @=@ 0002 @=@ Working Paper +dnet:review_levels @=@ 0002 @=@ Working Paper / Technical Report +dnet:review_levels @=@ 0002 @=@ Working Papers +dnet:review_levels @=@ 0002 @=@ WorkingPaper +dnet:review_levels @=@ 0002 @=@ article in non peer-reviewed journal +dnet:review_levels @=@ 0002 @=@ articolo preliminare +dnet:review_levels @=@ 0002 @=@ articulo preliminar +dnet:review_levels @=@ 0002 @=@ articulo sin revision por pares +dnet:review_levels @=@ 0002 @=@ artigo preliminar +dnet:review_levels @=@ 0002 @=@ artigo sem revisão +dnet:review_levels @=@ 0002 @=@ artículo preliminar +dnet:review_levels @=@ 0002 @=@ artículo sin revisión por pares +dnet:review_levels @=@ 0002 @=@ bookchapter (author version) +dnet:review_levels @=@ 0002 @=@ borrador +dnet:review_levels @=@ 0002 @=@ column (author version) +dnet:review_levels @=@ 0002 @=@ communication_invitee +dnet:review_levels @=@ 0002 @=@ doc-type:preprint +dnet:review_levels @=@ 0002 @=@ doc-type:workingPaper +dnet:review_levels @=@ 0002 @=@ draf +dnet:review_levels @=@ 0002 @=@ eu-repo/semantics/submittedVersion +dnet:review_levels @=@ 0002 @=@ http://purl.org/coar/resource_type/c_8042 +dnet:review_levels @=@ 0002 @=@ http://purl.org/coar/resource_type/c_816b +dnet:review_levels @=@ 0002 @=@ http://purl.org/coar/version/c_71e4c1898caa6e32 +dnet:review_levels @=@ 0002 @=@ http://purl.org/coar/version/c_b1a7d7d4d402bcce +dnet:review_levels @=@ 0002 @=@ http://purl.org/eprint/type/SubmittedBookItem +dnet:review_levels @=@ 0002 @=@ http://purl.org/eprint/type/SubmittedJournalArticle +dnet:review_levels @=@ 0002 @=@ http://purl.org/info:eu-repo/semantics/authorVersion +dnet:review_levels @=@ 0002 @=@ http://purl.org/info:eu-repo/semantics/submittedVersion +dnet:review_levels @=@ 0002 @=@ http://purl.org/spar/fabio/Preprint +dnet:review_levels @=@ 0002 @=@ http://purl.org/spar/fabio/WorkingPaper +dnet:review_levels @=@ 0002 @=@ https://dictionary.casrai.org/Preprint +dnet:review_levels @=@ 0002 @=@ info:ar-repo/semantics/documento de trabajo +dnet:review_levels @=@ 0002 @=@ info:ar-repo/semantics/documentoDeTrabajo +dnet:review_levels @=@ 0002 @=@ info:eu repo/semantics/draft +dnet:review_levels @=@ 0002 @=@ info:eu-repo/semantics/authorVersion +dnet:review_levels @=@ 0002 @=@ info:eu-repo/semantics/draft +dnet:review_levels @=@ 0002 @=@ info:eu-repo/semantics/preprint +dnet:review_levels @=@ 0002 @=@ info:eu-repo/semantics/submitedVersion +dnet:review_levels @=@ 0002 @=@ info:eu-repo/semantics/submittedVersion +dnet:review_levels @=@ 0002 @=@ info:eu-repo/semantics/unReviewed +dnet:review_levels @=@ 0002 @=@ info:eu-repo/semantics/updatedVersion +dnet:review_levels @=@ 0002 @=@ info:eu-repo/semantics/workingPaper +dnet:review_levels @=@ 0002 @=@ info:eu-repo/submittedVersion +dnet:review_levels @=@ 0002 @=@ info:ulb-repo/semantics/articleNonPeerReview +dnet:review_levels @=@ 0002 @=@ info:ulb-repo/semantics/openurl/vlink-workingpaper +dnet:review_levels @=@ 0002 @=@ info:ulb-repo/semantics/workingPaper +dnet:review_levels @=@ 0002 @=@ non peer-reviewed article +dnet:review_levels @=@ 0002 @=@ non-refereed review article +dnet:review_levels @=@ 0002 @=@ não avaliado +dnet:review_levels @=@ 0002 @=@ preprint +dnet:review_levels @=@ 0002 @=@ prepublicación +dnet:review_levels @=@ 0002 @=@ proceeding, seminar, workshop without peer review +dnet:review_levels @=@ 0002 @=@ proceedings (author version) +dnet:review_levels @=@ 0002 @=@ pré-print +dnet:review_levels @=@ 0002 @=@ pré-publication +dnet:review_levels @=@ 0002 @=@ préprint +dnet:review_levels @=@ 0002 @=@ prépublication +dnet:review_levels @=@ 0002 @=@ publicació preliminar +dnet:review_levels @=@ 0002 @=@ publication-preprint +dnet:review_levels @=@ 0002 @=@ publication-workingpaper +dnet:review_levels @=@ 0002 @=@ submitedVersion +dnet:review_levels @=@ 0002 @=@ submittedVersion +dnet:review_levels @=@ 0002 @=@ voordruk +dnet:review_levels @=@ 0002 @=@ workingPaper +dnet:review_levels @=@ 0002 @=@ ön baskı +dnet:review_levels @=@ 0002 @=@ Препринт +dnet:review_levels @=@ 0002 @=@ предпечатная версия публикации +dnet:review_levels @=@ 0002 @=@ препринт статьи +dnet:review_levels @=@ 0002 @=@ ディスカッション/ワーキング・ペーパー DP/WP +dnet:review_levels @=@ 0002 @=@ プレプリント +dnet:review_levels @=@ 0002 @=@ プレプリント Preprint +dnet:review_levels @=@ 0002 @=@ プレプリント(Preprint) +dnet:review_levels @=@ 0002 @=@ 印刷物/電子媒体-その他(査読無し) +dnet:review_levels @=@ 0002 @=@ 印刷物/電子媒体-テクニカルレポート類(査読無し) +dnet:review_levels @=@ 0002 @=@ 印刷物/電子媒体-会議発表論文(査読無し) +dnet:review_levels @=@ 0002 @=@ 印刷物/電子媒体-図書(査読無し) +dnet:review_levels @=@ 0002 @=@ 印刷物/電子媒体-学術雑誌論文(査読無し) +dnet:review_levels @=@ 0002 @=@ 印刷物/電子媒体-紀要論文(査読無し) +dnet:review_levels @=@ 0002 @=@ 印刷物/電子媒体-雑誌記事(査読無し) +dnet:review_levels @=@ 0002 @=@ 预印本 +dnet:review_levels @=@ 0001 @=@ ##rt.metadata.pkp.peerReviewed## +dnet:review_levels @=@ 0001 @=@ A1 Alkuperäisartikkeli tieteellisessä aikakauslehdessä +dnet:review_levels @=@ 0001 @=@ Art?culo revisado por pares +dnet:review_levels @=@ 0001 @=@ Article revisat per persones expertes +dnet:review_levels @=@ 0001 @=@ Article type: peer review +dnet:review_levels @=@ 0001 @=@ Article évalué par les pairs +dnet:review_levels @=@ 0001 @=@ Article évalué par des pairs +dnet:review_levels @=@ 0001 @=@ Article évalué par les pairs +dnet:review_levels @=@ 0001 @=@ Articolo valutato secondo i criteri della peer review +dnet:review_levels @=@ 0001 @=@ Articulo evaluado por dos pares +dnet:review_levels @=@ 0001 @=@ Articulo revisado por pares +dnet:review_levels @=@ 0001 @=@ Artigo Avaliado pelos Pares +dnet:review_levels @=@ 0001 @=@ Artigo Revisto por Pares +dnet:review_levels @=@ 0001 @=@ Artigo avaliado por blind peer review +dnet:review_levels @=@ 0001 @=@ Artigo avaliado por pares +dnet:review_levels @=@ 0001 @=@ Artigo de convidado. Avaliado pelos pares +dnet:review_levels @=@ 0001 @=@ Artigos; Avaliado pelos pares +dnet:review_levels @=@ 0001 @=@ Artículo de investigación, Investigaciones originales, Artículo evaluado por pares, Investigaciones empíricas +dnet:review_levels @=@ 0001 @=@ Artículo evaluado por pares +dnet:review_levels @=@ 0001 @=@ Artículo evaluado por pares, Ensayos de investigación +dnet:review_levels @=@ 0001 @=@ Artículo evaluado por pares, Investigaciones empíricas, Artículos de investigación +dnet:review_levels @=@ 0001 @=@ Artículo revisado +dnet:review_levels @=@ 0001 @=@ Artículo revisado por pares +dnet:review_levels @=@ 0001 @=@ Artículos de estudiantes, Artículo evaluado por pares, Artículos de investigación +dnet:review_levels @=@ 0001 @=@ Artículos de investigación evaluados por doble ciego +dnet:review_levels @=@ 0001 @=@ Artículos evaluadores por doble ciego +dnet:review_levels @=@ 0001 @=@ Artículos evaluados por pares +dnet:review_levels @=@ 0001 @=@ Artículos evaluados por pares académicos +dnet:review_levels @=@ 0001 @=@ Artículos revisados por pares +dnet:review_levels @=@ 0001 @=@ Avaliadas pelos pares +dnet:review_levels @=@ 0001 @=@ Avaliado anonimamente por pares +dnet:review_levels @=@ 0001 @=@ Avaliado em duplo cego por pares +dnet:review_levels @=@ 0001 @=@ Avaliado pela Editoria +dnet:review_levels @=@ 0001 @=@ Avaliado pela Editoria. Avaliado pelos pares. +dnet:review_levels @=@ 0001 @=@ Avaliado pelo Editoria +dnet:review_levels @=@ 0001 @=@ Avaliado pelo pares +dnet:review_levels @=@ 0001 @=@ Avaliado pelos Editores +dnet:review_levels @=@ 0001 @=@ Avaliado pelos pares +dnet:review_levels @=@ 0001 @=@ Avaliado pelos pares, Artigo de convidado +dnet:review_levels @=@ 0001 @=@ Avaliado pelos pares, Artigos Originais +dnet:review_levels @=@ 0001 @=@ Avaliado pelos pares, Artigos Originais, Artigos de Revisão +dnet:review_levels @=@ 0001 @=@ Avaliado pelos pares. Avaliado pelo Editoria +dnet:review_levels @=@ 0001 @=@ Avaliado po Pares +dnet:review_levels @=@ 0001 @=@ Avaliado por Editor +dnet:review_levels @=@ 0001 @=@ Avaliado por pares +dnet:review_levels @=@ 0001 @=@ Avaliados pelos pares +dnet:review_levels @=@ 0001 @=@ Avaliados por Pares +dnet:review_levels @=@ 0001 @=@ Blind Peer-reviewed Article +dnet:review_levels @=@ 0001 @=@ Book (peer-reviewed) +dnet:review_levels @=@ 0001 @=@ Comentario de libros, Comentario de revistas, Comentario de conferencias, Artículo evaluado por pares, Artículo de investigación +dnet:review_levels @=@ 0001 @=@ Conference paper; Peer-reviewed +dnet:review_levels @=@ 0001 @=@ Contribution to book (peer-reviewed) +dnet:review_levels @=@ 0001 @=@ Documento Avaliado por Pares +dnet:review_levels @=@ 0001 @=@ Double blind evaluation articles +dnet:review_levels @=@ 0001 @=@ Double blind peer review +dnet:review_levels @=@ 0001 @=@ Editors (peer-reviewed) +dnet:review_levels @=@ 0001 @=@ Evaluación por pares +dnet:review_levels @=@ 0001 @=@ Evaluado por pares +dnet:review_levels @=@ 0001 @=@ Evaluados por los pares +dnet:review_levels @=@ 0001 @=@ Hakem sürecinden geçmiş makale +dnet:review_levels @=@ 0001 @=@ Hakemli makale +dnet:review_levels @=@ 0001 @=@ Hakemlik Sürecinden Geçmiş +dnet:review_levels @=@ 0001 @=@ Invited Peer-Reviewed Article +dnet:review_levels @=@ 0001 @=@ Journal article; Peer-reviewed +dnet:review_levels @=@ 0001 @=@ Original article (peer-reviewed) +dnet:review_levels @=@ 0001 @=@ Other publication (peer-review) +dnet:review_levels @=@ 0001 @=@ Paper peer-reviewed +dnet:review_levels @=@ 0001 @=@ Papers evaluated by academic peers +dnet:review_levels @=@ 0001 @=@ Peer reviewed +dnet:review_levels @=@ 0001 @=@ Peer reviewed article +dnet:review_levels @=@ 0001 @=@ Peer reviewed invited commentry +dnet:review_levels @=@ 0001 @=@ Peer-Reviewed Protocol +dnet:review_levels @=@ 0001 @=@ Peer-reviewd Article +dnet:review_levels @=@ 0001 @=@ Peer-reviewed +dnet:review_levels @=@ 0001 @=@ Peer-reviewed Article +dnet:review_levels @=@ 0001 @=@ Peer-reviewed Paper +dnet:review_levels @=@ 0001 @=@ Peer-reviewed Review +dnet:review_levels @=@ 0001 @=@ Peer-reviewed Review Article +dnet:review_levels @=@ 0001 @=@ Peer-reviewed Text +dnet:review_levels @=@ 0001 @=@ Peer-reviewed communication +dnet:review_levels @=@ 0001 @=@ Peer-reviewed conference proceedings +dnet:review_levels @=@ 0001 @=@ Peer-reviewed research article +dnet:review_levels @=@ 0001 @=@ Peer-reviewed short communication +dnet:review_levels @=@ 0001 @=@ PeerReviewed +dnet:review_levels @=@ 0001 @=@ Proceedings (peer-reviewed) +dnet:review_levels @=@ 0001 @=@ Refereed +dnet:review_levels @=@ 0001 @=@ Refereed Article +dnet:review_levels @=@ 0001 @=@ Research articles evaluated by double blind +dnet:review_levels @=@ 0001 @=@ Resenha avaliada pelos pares +dnet:review_levels @=@ 0001 @=@ Review article (peer-reviewed) +dnet:review_levels @=@ 0001 @=@ Reviewed by peers +dnet:review_levels @=@ 0001 @=@ Revisión por Expertos +dnet:review_levels @=@ 0001 @=@ Revisto por Pares +dnet:review_levels @=@ 0001 @=@ SBBq abstracts / peer-reviewed +dnet:review_levels @=@ 0001 @=@ SBBq resúmenes - revisada por pares +dnet:review_levels @=@ 0001 @=@ Scholarly publ. Refereed +dnet:review_levels @=@ 0001 @=@ Scientific Publ (refereed) +dnet:review_levels @=@ 0001 @=@ Vertaisarvioimaton kirjoitus tieteellisessä aikakauslehdessä +dnet:review_levels @=@ 0001 @=@ Vertaisarvioitu alkuperäisartikkeli tieteellisessä aikakauslehdessä +dnet:review_levels @=@ 0001 @=@ Vertaisarvioitu artikkeli konferenssijulkaisussa +dnet:review_levels @=@ 0001 @=@ Vertaisarvioitu artikkeli tieteellisessä aikakauslehdessä +dnet:review_levels @=@ 0001 @=@ Vertaisarvioitu kirjan tai muun kokoomateoksen osa +dnet:review_levels @=@ 0001 @=@ Wetensch. publ. Refereed +dnet:review_levels @=@ 0001 @=@ article in peer-reviewed journal +dnet:review_levels @=@ 0001 @=@ articles validés +dnet:review_levels @=@ 0001 @=@ avaliado por pares, temas livres +dnet:review_levels @=@ 0001 @=@ info:eu-repo/semantics/peerReviewed +dnet:review_levels @=@ 0001 @=@ info:ulb-repo/semantics/articlePeerReview +dnet:review_levels @=@ 0001 @=@ proceeding with peer review +dnet:review_levels @=@ 0001 @=@ refereed_publications +dnet:review_levels @=@ 0001 @=@ ul_published_reviewed +dnet:review_levels @=@ 0001 @=@ Άρθρο που έχει αξιολογηθεί από ομότιμους ειδικούς +dnet:review_levels @=@ 0001 @=@ Άρθρο το οποίο έχει περάσει από ομότιμη αξιολόγηση +dnet:review_levels @=@ 0001 @=@ レフェリー付き論文 +dnet:review_levels @=@ 0001 @=@ 印刷物/電子媒体-テクニカルレポート類(査読有り) +dnet:review_levels @=@ 0001 @=@ 印刷物/電子媒体-会議発表論文(査読有り) +dnet:review_levels @=@ 0001 @=@ 印刷物/電子媒体-図書(査読有り) +dnet:review_levels @=@ 0001 @=@ 印刷物/電子媒体-学術雑誌論文(査読有り) +dnet:review_levels @=@ 0001 @=@ 印刷物/電子媒体-紀要論文(査読有り) +dnet:review_levels @=@ 0001 @=@ 印刷物/電子媒体-雑誌記事(査読有り) +dnet:review_levels @=@ 0001 @=@ 原著論文(査読有り) +dnet:review_levels @=@ 0001 @=@ 査読論文 \ No newline at end of file diff --git a/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/transform/terms.txt b/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/transform/terms.txt new file mode 100644 index 000000000..93cc00eca --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/transform/terms.txt @@ -0,0 +1,1080 @@ +ModularUiLabels @=@ ModularUiLabels @=@ PendingRepositoryResources @=@ Pending datasource +ModularUiLabels @=@ ModularUiLabels @=@ RepositoryServiceResources @=@ Valid datasource +dnet:content_description_typologies @=@ D-Net Content Description Typologies @=@ file::EuropePMC @=@ file::EuropePMC +dnet:content_description_typologies @=@ D-Net Content Description Typologies @=@ file::PDF @=@ file::PDF +dnet:content_description_typologies @=@ D-Net Content Description Typologies @=@ file::WoS @=@ file::WoS +dnet:content_description_typologies @=@ D-Net Content Description Typologies @=@ metadata @=@ metadata +dnet:content_description_typologies @=@ D-Net Content Description Typologies @=@ file::hybrid @=@ file::hybrid +dnet:provenanceActions @=@ dnet:provenanceActions @=@ sysimport:crosswalk:cris @=@ Harvested +dnet:provenanceActions @=@ dnet:provenanceActions @=@ sysimport:actionset:orcidworks-no-doi @=@ Harvested +dnet:provenanceActions @=@ dnet:provenanceActions @=@ sysimport:crosswalk:infospace @=@ Harvested +dnet:provenanceActions @=@ dnet:provenanceActions @=@ sysimport:crosswalk @=@ Harvested +dnet:provenanceActions @=@ dnet:provenanceActions @=@ sysimport:crosswalk:aggregator @=@ Harvested +dnet:provenanceActions @=@ dnet:provenanceActions @=@ sysimport:crosswalk:datasetarchive @=@ Harvested +dnet:provenanceActions @=@ dnet:provenanceActions @=@ sysimport:actionset @=@ Harvested +dnet:provenanceActions @=@ dnet:provenanceActions @=@ sysimport:crosswalk:entityregistry @=@ Harvested +dnet:provenanceActions @=@ dnet:provenanceActions @=@ sysimport:crosswalk:repository @=@ Harvested +dnet:provenanceActions @=@ dnet:provenanceActions @=@ sysimport:mining:aggregator @=@ Inferred by OpenAIRE +dnet:provenanceActions @=@ dnet:provenanceActions @=@ community:subject @=@ Inferred by OpenAIRE +dnet:provenanceActions @=@ dnet:provenanceActions @=@ community:zenodocommunity @=@ Inferred by OpenAIRE +dnet:provenanceActions @=@ dnet:provenanceActions @=@ iis @=@ Inferred by OpenAIRE +dnet:provenanceActions @=@ dnet:provenanceActions @=@ sysimport:mining:entityregistry @=@ Inferred by OpenAIRE +dnet:provenanceActions @=@ dnet:provenanceActions @=@ community:organization @=@ Inferred by OpenAIRE +dnet:provenanceActions @=@ dnet:provenanceActions @=@ sysimport:mining:infospace @=@ Inferred by OpenAIRE +dnet:provenanceActions @=@ dnet:provenanceActions @=@ sysimport:dedup @=@ Inferred by OpenAIRE +dnet:provenanceActions @=@ dnet:provenanceActions @=@ community:datasource @=@ Inferred by OpenAIRE +dnet:provenanceActions @=@ dnet:provenanceActions @=@ propagation:project:semrel @=@ Inferred by OpenAIRE +dnet:provenanceActions @=@ dnet:provenanceActions @=@ sysimport:mining:cris @=@ Inferred by OpenAIRE +dnet:provenanceActions @=@ dnet:provenanceActions @=@ sysimport:mining:repository @=@ Inferred by OpenAIRE +dnet:provenanceActions @=@ dnet:provenanceActions @=@ sysimport:mining:datasetarchive @=@ Inferred by OpenAIRE +dnet:provenanceActions @=@ dnet:provenanceActions @=@ community:semrel @=@ Inferred by OpenAIRE +dnet:provenanceActions @=@ dnet:provenanceActions @=@ user:claim @=@ Linked by user +dnet:provenanceActions @=@ dnet:provenanceActions @=@ user:claim:pid @=@ Linked by user +dnet:provenanceActions @=@ dnet:provenanceActions @=@ user:insert @=@ Linked by user +dnet:provenanceActions @=@ dnet:provenanceActions @=@ user:claim:search @=@ Linked by user +dnet:provenanceActions @=@ dnet:provenanceActions @=@ UNKNOWN @=@ UNKNOWN +dnet:provenanceActions @=@ dnet:provenanceActions @=@ country:instrepos @=@ Inferred by OpenAIRE +dnet:access_modes @=@ dnet:access_modes @=@ 12MONTHS @=@ 12 Months Embargo +dnet:access_modes @=@ dnet:access_modes @=@ 6MONTHS @=@ 6 Months Embargo +dnet:access_modes @=@ dnet:access_modes @=@ CLOSED @=@ Closed Access +dnet:access_modes @=@ dnet:access_modes @=@ EMBARGO @=@ Embargo +dnet:access_modes @=@ dnet:access_modes @=@ OPEN @=@ Open Access +dnet:access_modes @=@ dnet:access_modes @=@ OPEN SOURCE @=@ Open Source +dnet:access_modes @=@ dnet:access_modes @=@ OTHER @=@ Other +dnet:access_modes @=@ dnet:access_modes @=@ RESTRICTED @=@ Restricted +dnet:access_modes @=@ dnet:access_modes @=@ UNKNOWN @=@ not available +fct:funding_typologies @=@ fct:funding_typologies @=@ fct:program @=@ fct:program +dnet:compatibilityLevel @=@ dnet:compatibilityLevel @=@ openaire2.0 @=@ OpenAIRE 2.0 (EC funding) +dnet:compatibilityLevel @=@ dnet:compatibilityLevel @=@ openaire3.0 @=@ OpenAIRE 3.0 (OA, funding) +dnet:compatibilityLevel @=@ dnet:compatibilityLevel @=@ driver @=@ OpenAIRE Basic (DRIVER OA) +dnet:compatibilityLevel @=@ dnet:compatibilityLevel @=@ openaire-cris_1.1 @=@ OpenAIRE CRIS v1.1 +dnet:compatibilityLevel @=@ dnet:compatibilityLevel @=@ openaire2.0_data @=@ OpenAIRE Data (funded, referenced datasets) +dnet:compatibilityLevel @=@ dnet:compatibilityLevel @=@ openaire-pub_4.0 @=@ OpenAIRE PubRepos v4.0 +dnet:compatibilityLevel @=@ dnet:compatibilityLevel @=@ hostedBy @=@ collected from a compatible aggregator +dnet:compatibilityLevel @=@ dnet:compatibilityLevel @=@ files @=@ files +dnet:compatibilityLevel @=@ dnet:compatibilityLevel @=@ native @=@ native +dnet:compatibilityLevel @=@ dnet:compatibilityLevel @=@ UNKNOWN @=@ not available +dnet:compatibilityLevel @=@ dnet:compatibilityLevel @=@ notCompatible @=@ under validation +dnet:dataCite_date @=@ dnet:dataCite_date @=@ UNKNOWN @=@ UNKNOWN +dnet:dataCite_date @=@ dnet:dataCite_date @=@ available @=@ available +dnet:dataCite_date @=@ dnet:dataCite_date @=@ copyrighted @=@ copyrighted +dnet:dataCite_date @=@ dnet:dataCite_date @=@ created @=@ created +dnet:dataCite_date @=@ dnet:dataCite_date @=@ endDate @=@ endDate +dnet:dataCite_date @=@ dnet:dataCite_date @=@ issued @=@ issued +dnet:dataCite_date @=@ dnet:dataCite_date @=@ startDate @=@ startDate +dnet:dataCite_date @=@ dnet:dataCite_date @=@ submitted @=@ submitted +dnet:dataCite_date @=@ dnet:dataCite_date @=@ updated @=@ updated +dnet:dataCite_date @=@ dnet:dataCite_date @=@ valid @=@ valid +dnet:dataCite_date @=@ dnet:dataCite_date @=@ published-print @=@ published-print +dnet:dataCite_date @=@ dnet:dataCite_date @=@ published-online @=@ published-online +dnet:dataCite_date @=@ dnet:dataCite_date @=@ accepted @=@ accepted +dnet:datasource_typologies @=@ dnet:datasource_typologies @=@ crissystem @=@ CRIS System +dnet:datasource_typologies @=@ dnet:datasource_typologies @=@ datarepository::unknown @=@ Data Repository +dnet:datasource_typologies @=@ dnet:datasource_typologies @=@ aggregator::datarepository @=@ Data Repository Aggregator +dnet:datasource_typologies @=@ dnet:datasource_typologies @=@ entityregistry::projects @=@ Funder database +dnet:datasource_typologies @=@ dnet:datasource_typologies @=@ infospace @=@ Information Space +dnet:datasource_typologies @=@ dnet:datasource_typologies @=@ pubsrepository::institutional @=@ Institutional Repository +dnet:datasource_typologies @=@ dnet:datasource_typologies @=@ aggregator::pubsrepository::institutional @=@ Institutional Repository Aggregator +dnet:datasource_typologies @=@ dnet:datasource_typologies @=@ pubsrepository::journal @=@ Journal +dnet:datasource_typologies @=@ dnet:datasource_typologies @=@ aggregator::pubsrepository::journals @=@ Journal Aggregator/Publisher +dnet:datasource_typologies @=@ dnet:datasource_typologies @=@ pubsrepository::mock @=@ Other +dnet:datasource_typologies @=@ dnet:datasource_typologies @=@ pubscatalogue::unknown @=@ Publication Catalogue +dnet:datasource_typologies @=@ dnet:datasource_typologies @=@ pubsrepository::unknown @=@ Publication Repository +dnet:datasource_typologies @=@ dnet:datasource_typologies @=@ aggregator::pubsrepository::unknown @=@ Publication Repository Aggregator +dnet:datasource_typologies @=@ dnet:datasource_typologies @=@ entityregistry @=@ Registry +dnet:datasource_typologies @=@ dnet:datasource_typologies @=@ entityregistry::repositories @=@ Registry of repositories +dnet:datasource_typologies @=@ dnet:datasource_typologies @=@ entityregistry::products @=@ Registry of research products +dnet:datasource_typologies @=@ dnet:datasource_typologies @=@ entityregistry::researchers @=@ Registry of researchers +dnet:datasource_typologies @=@ dnet:datasource_typologies @=@ entityregistry::organizations @=@ Registry of organizations +dnet:datasource_typologies @=@ dnet:datasource_typologies @=@ scholarcomminfra @=@ Scholarly Comm. Infrastructure +dnet:datasource_typologies @=@ dnet:datasource_typologies @=@ softwarerepository @=@ Software Repository +dnet:datasource_typologies @=@ dnet:datasource_typologies @=@ pubsrepository::thematic @=@ Thematic Repository +dnet:datasource_typologies @=@ dnet:datasource_typologies @=@ websource @=@ Web Source +dnet:datasource_typologies @=@ dnet:datasource_typologies @=@ aggregator::softwarerepository @=@ Software Repository Aggregator +dnet:datasource_typologies @=@ dnet:datasource_typologies @=@ orprepository @=@ Repository +dnet:datasource_typologies @=@ dnet:datasource_typologies @=@ researchgraph @=@ Research Graph +dnet:subject_classification_typologies @=@ dnet:subject_classification_typologies @=@ ACM @=@ ACM Computing Classification System +dnet:subject_classification_typologies @=@ dnet:subject_classification_typologies @=@ agrovoc @=@ AGROVOC +dnet:subject_classification_typologies @=@ dnet:subject_classification_typologies @=@ bicssc @=@ BIC standard subject categories +dnet:subject_classification_typologies @=@ dnet:subject_classification_typologies @=@ DFG @=@ DFG Classification +dnet:subject_classification_typologies @=@ dnet:subject_classification_typologies @=@ ddc @=@ Dewey Decimal Classification +dnet:subject_classification_typologies @=@ dnet:subject_classification_typologies @=@ nsf:fieldOfApplication @=@ Field of Application (NSF) +dnet:subject_classification_typologies @=@ dnet:subject_classification_typologies @=@ gok @=@ Göttingen Online Classification +dnet:subject_classification_typologies @=@ dnet:subject_classification_typologies @=@ ec:h2020topics @=@ Horizon 2020 Topics +dnet:subject_classification_typologies @=@ dnet:subject_classification_typologies @=@ IPC @=@ International Patent Classification +dnet:subject_classification_typologies @=@ dnet:subject_classification_typologies @=@ jel @=@ JEL Classification +dnet:subject_classification_typologies @=@ dnet:subject_classification_typologies @=@ lcsh @=@ Library of Congress Subject Headings +dnet:subject_classification_typologies @=@ dnet:subject_classification_typologies @=@ msc @=@ Mathematics Subject Classification +dnet:subject_classification_typologies @=@ dnet:subject_classification_typologies @=@ mesheuropmc @=@ Medical Subject Headings +dnet:subject_classification_typologies @=@ dnet:subject_classification_typologies @=@ mesh @=@ Medical Subject Headings +dnet:subject_classification_typologies @=@ dnet:subject_classification_typologies @=@ bk @=@ Nederlandse basisclassificatie +dnet:subject_classification_typologies @=@ dnet:subject_classification_typologies @=@ dnet:od_subjects @=@ OpenDOAR subjects +dnet:subject_classification_typologies @=@ dnet:subject_classification_typologies @=@ ocis @=@ Optics Classification and Indexing Scheme +dnet:subject_classification_typologies @=@ dnet:subject_classification_typologies @=@ pacs @=@ Physics and Astronomy Classification Scheme +dnet:subject_classification_typologies @=@ dnet:subject_classification_typologies @=@ rvk @=@ Regensburger Verbundklassifikation +dnet:subject_classification_typologies @=@ dnet:subject_classification_typologies @=@ UNKNOWN @=@ UNKNOWN +dnet:subject_classification_typologies @=@ dnet:subject_classification_typologies @=@ udc @=@ Universal Decimal Classification +dnet:subject_classification_typologies @=@ dnet:subject_classification_typologies @=@ wos @=@ Web of Science Subject Areas +dnet:subject_classification_typologies @=@ dnet:subject_classification_typologies @=@ arxiv @=@ arXiv +dnet:subject_classification_typologies @=@ dnet:subject_classification_typologies @=@ keyword @=@ keyword +dnet:subject_classification_typologies @=@ dnet:subject_classification_typologies @=@ MAG @=@ Microsoft Academic Graph classification +fct:contractTypes @=@ fct:contractTypes @=@ UNKNOWN @=@ UNKNOWN +dnet:publication_resource @=@ dnet:publication_resource @=@ 0018 @=@ Annotation +dnet:publication_resource @=@ dnet:publication_resource @=@ 0001 @=@ Article +dnet:publication_resource @=@ dnet:publication_resource @=@ 0033 @=@ Audiovisual +dnet:publication_resource @=@ dnet:publication_resource @=@ 0008 @=@ Bachelor thesis +dnet:publication_resource @=@ dnet:publication_resource @=@ 0046 @=@ Bioentity +dnet:publication_resource @=@ dnet:publication_resource @=@ 0002 @=@ Book +dnet:publication_resource @=@ dnet:publication_resource @=@ 0037 @=@ Clinical Trial +dnet:publication_resource @=@ dnet:publication_resource @=@ 0022 @=@ Collection +dnet:publication_resource @=@ dnet:publication_resource @=@ 0004 @=@ Conference object +dnet:publication_resource @=@ dnet:publication_resource @=@ 0005 @=@ Contribution for newspaper or weekly magazine +dnet:publication_resource @=@ dnet:publication_resource @=@ 0045 @=@ Data Management Plan +dnet:publication_resource @=@ dnet:publication_resource @=@ 0031 @=@ Data Paper +dnet:publication_resource @=@ dnet:publication_resource @=@ 0021 @=@ Dataset +dnet:publication_resource @=@ dnet:publication_resource @=@ 0006 @=@ Doctoral thesis +dnet:publication_resource @=@ dnet:publication_resource @=@ 0023 @=@ Event +dnet:publication_resource @=@ dnet:publication_resource @=@ 0009 @=@ External research report +dnet:publication_resource @=@ dnet:publication_resource @=@ 0024 @=@ Film +dnet:publication_resource @=@ dnet:publication_resource @=@ 0025 @=@ Image +dnet:publication_resource @=@ dnet:publication_resource @=@ 0026 @=@ InteractiveResource +dnet:publication_resource @=@ dnet:publication_resource @=@ 0011 @=@ Internal report +dnet:publication_resource @=@ dnet:publication_resource @=@ 0043 @=@ Journal +dnet:publication_resource @=@ dnet:publication_resource @=@ 0010 @=@ Lecture +dnet:publication_resource @=@ dnet:publication_resource @=@ 0007 @=@ Master thesis +dnet:publication_resource @=@ dnet:publication_resource @=@ 0027 @=@ Model +dnet:publication_resource @=@ dnet:publication_resource @=@ 0012 @=@ Newsletter +dnet:publication_resource @=@ dnet:publication_resource @=@ 0020 @=@ Other ORP type +dnet:publication_resource @=@ dnet:publication_resource @=@ 0039 @=@ Other dataset type +dnet:publication_resource @=@ dnet:publication_resource @=@ 0038 @=@ Other literature type +dnet:publication_resource @=@ dnet:publication_resource @=@ 0040 @=@ Other software type +dnet:publication_resource @=@ dnet:publication_resource @=@ 0013 @=@ Part of book or chapter of book +dnet:publication_resource @=@ dnet:publication_resource @=@ 0019 @=@ Patent +dnet:publication_resource @=@ dnet:publication_resource @=@ 0028 @=@ PhysicalObject +dnet:publication_resource @=@ dnet:publication_resource @=@ 0016 @=@ Preprint +dnet:publication_resource @=@ dnet:publication_resource @=@ 0034 @=@ Project deliverable +dnet:publication_resource @=@ dnet:publication_resource @=@ 0035 @=@ Project milestone +dnet:publication_resource @=@ dnet:publication_resource @=@ 0036 @=@ Project proposal +dnet:publication_resource @=@ dnet:publication_resource @=@ 0017 @=@ Report +dnet:publication_resource @=@ dnet:publication_resource @=@ 0014 @=@ Research +dnet:publication_resource @=@ dnet:publication_resource @=@ 0015 @=@ Review +dnet:publication_resource @=@ dnet:publication_resource @=@ 0029 @=@ Software +dnet:publication_resource @=@ dnet:publication_resource @=@ 0032 @=@ Software Paper +dnet:publication_resource @=@ dnet:publication_resource @=@ 0030 @=@ Sound +dnet:publication_resource @=@ dnet:publication_resource @=@ 0044 @=@ Thesis +dnet:publication_resource @=@ dnet:publication_resource @=@ 0000 @=@ Unknown +dnet:publication_resource @=@ dnet:publication_resource @=@ 0042 @=@ Virtual Appliance +ec:funding_typologies @=@ ec:funding_typologies @=@ ec:frameworkprogram @=@ frameworkprogram +ec:funding_typologies @=@ ec:funding_typologies @=@ ec:program @=@ program +ec:funding_typologies @=@ ec:funding_typologies @=@ ec:specificprogram @=@ specificprogram +ec:FP7contractTypes @=@ ec:FP7contractTypes @=@ 171 @=@ Article 171 of the Treaty +ec:FP7contractTypes @=@ ec:FP7contractTypes @=@ BSG @=@ Research for the benefit of specific groups +ec:FP7contractTypes @=@ ec:FP7contractTypes @=@ CIP-EIP-TN @=@ CIP-Eco-Innovation - CIP-Thematic Network +ec:FP7contractTypes @=@ ec:FP7contractTypes @=@ CP @=@ Collaborative project +ec:FP7contractTypes @=@ ec:FP7contractTypes @=@ CP-CSA @=@ Combination of CP & CSA +ec:FP7contractTypes @=@ ec:FP7contractTypes @=@ CSA @=@ Coordination and support action +ec:FP7contractTypes @=@ ec:FP7contractTypes @=@ ERC @=@ Support for frontier research (ERC) +ec:FP7contractTypes @=@ ec:FP7contractTypes @=@ MC @=@ Support for training and career development of researchers (Marie Curie) +ec:FP7contractTypes @=@ ec:FP7contractTypes @=@ NoE @=@ Network of Excellence +wt:funding_relations @=@ wt:funding_relations @=@ wt:hasParentFunding @=@ wt:hasParentFunding +dnet:languages @=@ dnet:languages @=@ abk @=@ Abkhazian +dnet:languages @=@ dnet:languages @=@ ace @=@ Achinese +dnet:languages @=@ dnet:languages @=@ ach @=@ Acoli +dnet:languages @=@ dnet:languages @=@ ada @=@ Adangme +dnet:languages @=@ dnet:languages @=@ aar @=@ Afar +dnet:languages @=@ dnet:languages @=@ afh @=@ Afrihili +dnet:languages @=@ dnet:languages @=@ afr @=@ Afrikaans +dnet:languages @=@ dnet:languages @=@ afa @=@ Afro-Asiatic +dnet:languages @=@ dnet:languages @=@ aka @=@ Akan +dnet:languages @=@ dnet:languages @=@ akk @=@ Akkadian +dnet:languages @=@ dnet:languages @=@ alb/sqi @=@ Albanian +dnet:languages @=@ dnet:languages @=@ ale @=@ Aleut +dnet:languages @=@ dnet:languages @=@ alg @=@ Algonquian languages +dnet:languages @=@ dnet:languages @=@ tut @=@ Altaic +dnet:languages @=@ dnet:languages @=@ amh @=@ Amharic +dnet:languages @=@ dnet:languages @=@ egy @=@ Ancient Egyptian +dnet:languages @=@ dnet:languages @=@ grc @=@ Ancient Greek +dnet:languages @=@ dnet:languages @=@ apa @=@ Apache +dnet:languages @=@ dnet:languages @=@ ara @=@ Arabic +dnet:languages @=@ dnet:languages @=@ arg @=@ Aragonese +dnet:languages @=@ dnet:languages @=@ arc @=@ Aramaic +dnet:languages @=@ dnet:languages @=@ arp @=@ Arapaho +dnet:languages @=@ dnet:languages @=@ arn @=@ Araucanian +dnet:languages @=@ dnet:languages @=@ arw @=@ Arawak +dnet:languages @=@ dnet:languages @=@ arm/hye @=@ Armenian +dnet:languages @=@ dnet:languages @=@ art @=@ Artificial +dnet:languages @=@ dnet:languages @=@ asm @=@ Assamese +dnet:languages @=@ dnet:languages @=@ ath @=@ Athapascan +dnet:languages @=@ dnet:languages @=@ map @=@ Austronesian +dnet:languages @=@ dnet:languages @=@ ina @=@ Auxiliary Language Association) +dnet:languages @=@ dnet:languages @=@ ava @=@ Avaric +dnet:languages @=@ dnet:languages @=@ ave @=@ Avestan +dnet:languages @=@ dnet:languages @=@ awa @=@ Awadhi +dnet:languages @=@ dnet:languages @=@ aym @=@ Aymara +dnet:languages @=@ dnet:languages @=@ aze @=@ Azerbaijani +dnet:languages @=@ dnet:languages @=@ nah @=@ Aztec +dnet:languages @=@ dnet:languages @=@ ban @=@ Balinese +dnet:languages @=@ dnet:languages @=@ bat @=@ Baltic +dnet:languages @=@ dnet:languages @=@ bal @=@ Baluchi +dnet:languages @=@ dnet:languages @=@ bam @=@ Bambara +dnet:languages @=@ dnet:languages @=@ bai @=@ Bamileke +dnet:languages @=@ dnet:languages @=@ bad @=@ Banda +dnet:languages @=@ dnet:languages @=@ bnt @=@ Bantu +dnet:languages @=@ dnet:languages @=@ bas @=@ Basa +dnet:languages @=@ dnet:languages @=@ bak @=@ Bashkir +dnet:languages @=@ dnet:languages @=@ baq/eus @=@ Basque +dnet:languages @=@ dnet:languages @=@ bej @=@ Beja +dnet:languages @=@ dnet:languages @=@ bel @=@ Belarusian +dnet:languages @=@ dnet:languages @=@ bem @=@ Bemba +dnet:languages @=@ dnet:languages @=@ ben @=@ Bengali +dnet:languages @=@ dnet:languages @=@ ber @=@ Berber +dnet:languages @=@ dnet:languages @=@ bho @=@ Bhojpuri +dnet:languages @=@ dnet:languages @=@ bih @=@ Bihari +dnet:languages @=@ dnet:languages @=@ bik @=@ Bikol +dnet:languages @=@ dnet:languages @=@ bin @=@ Bini +dnet:languages @=@ dnet:languages @=@ bis @=@ Bislama +dnet:languages @=@ dnet:languages @=@ nob @=@ Bokmål, Norwegian; Norwegian Bokmål +dnet:languages @=@ dnet:languages @=@ bos @=@ Bosnian +dnet:languages @=@ dnet:languages @=@ bra @=@ Braj +dnet:languages @=@ dnet:languages @=@ bre @=@ Breton +dnet:languages @=@ dnet:languages @=@ bug @=@ Buginese +dnet:languages @=@ dnet:languages @=@ bul @=@ Bulgarian +dnet:languages @=@ dnet:languages @=@ bua @=@ Buriat +dnet:languages @=@ dnet:languages @=@ bur/mya @=@ Burmese +dnet:languages @=@ dnet:languages @=@ cad @=@ Caddo +dnet:languages @=@ dnet:languages @=@ car @=@ Carib +dnet:languages @=@ dnet:languages @=@ cat @=@ Catalan; Valencian +dnet:languages @=@ dnet:languages @=@ cau @=@ Caucasian +dnet:languages @=@ dnet:languages @=@ ceb @=@ Cebuano +dnet:languages @=@ dnet:languages @=@ cel @=@ Celtic +dnet:languages @=@ dnet:languages @=@ cai @=@ Central American Indian +dnet:languages @=@ dnet:languages @=@ chg @=@ Chagatai +dnet:languages @=@ dnet:languages @=@ cha @=@ Chamorro +dnet:languages @=@ dnet:languages @=@ che @=@ Chechen +dnet:languages @=@ dnet:languages @=@ chr @=@ Cherokee +dnet:languages @=@ dnet:languages @=@ nya @=@ Chewa; Chichewa; Nyanja +dnet:languages @=@ dnet:languages @=@ chy @=@ Cheyenne +dnet:languages @=@ dnet:languages @=@ chb @=@ Chibcha +dnet:languages @=@ dnet:languages @=@ chi/zho @=@ Chinese +dnet:languages @=@ dnet:languages @=@ chn @=@ Chinook jargon +dnet:languages @=@ dnet:languages @=@ cho @=@ Choctaw +dnet:languages @=@ dnet:languages @=@ chu @=@ Church Slavic; Slavonic; Church Slavonic; Old Bulgarian; Old Church Slavonic +dnet:languages @=@ dnet:languages @=@ chv @=@ Chuvash +dnet:languages @=@ dnet:languages @=@ cop @=@ Coptic +dnet:languages @=@ dnet:languages @=@ cor @=@ Cornish +dnet:languages @=@ dnet:languages @=@ cos @=@ Corsican +dnet:languages @=@ dnet:languages @=@ cre @=@ Cree +dnet:languages @=@ dnet:languages @=@ mus @=@ Creek +dnet:languages @=@ dnet:languages @=@ crp @=@ Creoles and Pidgins +dnet:languages @=@ dnet:languages @=@ hrv @=@ Croatian +dnet:languages @=@ dnet:languages @=@ cus @=@ Cushitic +dnet:languages @=@ dnet:languages @=@ ces/cze @=@ Czech +dnet:languages @=@ dnet:languages @=@ dak @=@ Dakota +dnet:languages @=@ dnet:languages @=@ dan @=@ Danish +dnet:languages @=@ dnet:languages @=@ del @=@ Delaware +dnet:languages @=@ dnet:languages @=@ din @=@ Dinka +dnet:languages @=@ dnet:languages @=@ div @=@ Divehi +dnet:languages @=@ dnet:languages @=@ doi @=@ Dogri +dnet:languages @=@ dnet:languages @=@ dra @=@ Dravidian +dnet:languages @=@ dnet:languages @=@ dua @=@ Duala +dnet:languages @=@ dnet:languages @=@ dut/nld @=@ Dutch; Flemish +dnet:languages @=@ dnet:languages @=@ dyu @=@ Dyula +dnet:languages @=@ dnet:languages @=@ dzo @=@ Dzongkha +dnet:languages @=@ dnet:languages @=@ efi @=@ Efik +dnet:languages @=@ dnet:languages @=@ eka @=@ Ekajuk +dnet:languages @=@ dnet:languages @=@ elx @=@ Elamite +dnet:languages @=@ dnet:languages @=@ eng @=@ English +dnet:languages @=@ dnet:languages @=@ cpe @=@ English-based Creoles and Pidgins +dnet:languages @=@ dnet:languages @=@ esk @=@ Eskimo +dnet:languages @=@ dnet:languages @=@ epo @=@ Esperanto +dnet:languages @=@ dnet:languages @=@ est @=@ Estonian +dnet:languages @=@ dnet:languages @=@ ewe @=@ Ewe +dnet:languages @=@ dnet:languages @=@ ewo @=@ Ewondo +dnet:languages @=@ dnet:languages @=@ fan @=@ Fang +dnet:languages @=@ dnet:languages @=@ fat @=@ Fanti +dnet:languages @=@ dnet:languages @=@ fao @=@ Faroese +dnet:languages @=@ dnet:languages @=@ fij @=@ Fijian +dnet:languages @=@ dnet:languages @=@ fin @=@ Finnish +dnet:languages @=@ dnet:languages @=@ fiu @=@ Finno-Ugrian +dnet:languages @=@ dnet:languages @=@ fon @=@ Fon +dnet:languages @=@ dnet:languages @=@ fra/fre @=@ French +dnet:languages @=@ dnet:languages @=@ cpf @=@ French-based Creoles and Pidgins +dnet:languages @=@ dnet:languages @=@ fry @=@ Frisian +dnet:languages @=@ dnet:languages @=@ ful @=@ Fulah +dnet:languages @=@ dnet:languages @=@ gaa @=@ Ga +dnet:languages @=@ dnet:languages @=@ gae/gdh @=@ Gaelic +dnet:languages @=@ dnet:languages @=@ gla @=@ Gaelic; Scottish Gaelic +dnet:languages @=@ dnet:languages @=@ glg @=@ Galician +dnet:languages @=@ dnet:languages @=@ lug @=@ Ganda +dnet:languages @=@ dnet:languages @=@ gay @=@ Gayo +dnet:languages @=@ dnet:languages @=@ gez @=@ Geez +dnet:languages @=@ dnet:languages @=@ geo/kat @=@ Georgian +dnet:languages @=@ dnet:languages @=@ deu/ger @=@ German +dnet:languages @=@ dnet:languages @=@ gem @=@ Germanic +dnet:languages @=@ dnet:languages @=@ kik @=@ Gikuyu; Kikuyu +dnet:languages @=@ dnet:languages @=@ gil @=@ Gilbertese +dnet:languages @=@ dnet:languages @=@ gon @=@ Gondi +dnet:languages @=@ dnet:languages @=@ got @=@ Gothic +dnet:languages @=@ dnet:languages @=@ grb @=@ Grebo +dnet:languages @=@ dnet:languages @=@ ell/gre @=@ Greek +dnet:languages @=@ dnet:languages @=@ gre/ell @=@ Greek, Modern (1453-) +dnet:languages @=@ dnet:languages @=@ kal @=@ Greenlandic; Kalaallisut +dnet:languages @=@ dnet:languages @=@ grn @=@ Guarani +dnet:languages @=@ dnet:languages @=@ guj @=@ Gujarati +dnet:languages @=@ dnet:languages @=@ hai @=@ Haida +dnet:languages @=@ dnet:languages @=@ hat @=@ Haitian; Haitian Creole +dnet:languages @=@ dnet:languages @=@ hau @=@ Hausa +dnet:languages @=@ dnet:languages @=@ haw @=@ Hawaiian +dnet:languages @=@ dnet:languages @=@ heb @=@ Hebrew +dnet:languages @=@ dnet:languages @=@ her @=@ Herero +dnet:languages @=@ dnet:languages @=@ hil @=@ Hiligaynon +dnet:languages @=@ dnet:languages @=@ him @=@ Himachali +dnet:languages @=@ dnet:languages @=@ hin @=@ Hindi +dnet:languages @=@ dnet:languages @=@ hmo @=@ Hiri Motu +dnet:languages @=@ dnet:languages @=@ hun @=@ Hungarian +dnet:languages @=@ dnet:languages @=@ hup @=@ Hupa +dnet:languages @=@ dnet:languages @=@ iba @=@ Iban +dnet:languages @=@ dnet:languages @=@ ice/isl @=@ Icelandic +dnet:languages @=@ dnet:languages @=@ ido @=@ Ido +dnet:languages @=@ dnet:languages @=@ ibo @=@ Igbo +dnet:languages @=@ dnet:languages @=@ ijo @=@ Ijo +dnet:languages @=@ dnet:languages @=@ ilo @=@ Iloko +dnet:languages @=@ dnet:languages @=@ inc @=@ Indic +dnet:languages @=@ dnet:languages @=@ ine @=@ Indo-European +dnet:languages @=@ dnet:languages @=@ ind @=@ Indonesian +dnet:languages @=@ dnet:languages @=@ ile @=@ Interlingue +dnet:languages @=@ dnet:languages @=@ iku @=@ Inuktitut +dnet:languages @=@ dnet:languages @=@ ipk @=@ Inupiaq +dnet:languages @=@ dnet:languages @=@ ira @=@ Iranian +dnet:languages @=@ dnet:languages @=@ gai/iri @=@ Irish +dnet:languages @=@ dnet:languages @=@ iro @=@ Iroquoian +dnet:languages @=@ dnet:languages @=@ ita @=@ Italian +dnet:languages @=@ dnet:languages @=@ jpn @=@ Japanese +dnet:languages @=@ dnet:languages @=@ jav @=@ Javanese +dnet:languages @=@ dnet:languages @=@ jrb @=@ Judeo-Arabic +dnet:languages @=@ dnet:languages @=@ jpr @=@ Judeo-Persian +dnet:languages @=@ dnet:languages @=@ kab @=@ Kabyle +dnet:languages @=@ dnet:languages @=@ kac @=@ Kachin +dnet:languages @=@ dnet:languages @=@ kam @=@ Kamba +dnet:languages @=@ dnet:languages @=@ kan @=@ Kannada +dnet:languages @=@ dnet:languages @=@ kau @=@ Kanuri +dnet:languages @=@ dnet:languages @=@ kaa @=@ Kara-Kalpak +dnet:languages @=@ dnet:languages @=@ kar @=@ Karen +dnet:languages @=@ dnet:languages @=@ kas @=@ Kashmiri +dnet:languages @=@ dnet:languages @=@ kaw @=@ Kawi +dnet:languages @=@ dnet:languages @=@ kaz @=@ Kazakh +dnet:languages @=@ dnet:languages @=@ kha @=@ Khasi +dnet:languages @=@ dnet:languages @=@ khm @=@ Khmer +dnet:languages @=@ dnet:languages @=@ khi @=@ Khoisan +dnet:languages @=@ dnet:languages @=@ kho @=@ Khotanese +dnet:languages @=@ dnet:languages @=@ kin @=@ Kinyarwanda +dnet:languages @=@ dnet:languages @=@ kir @=@ Kirghiz +dnet:languages @=@ dnet:languages @=@ kom @=@ Komi +dnet:languages @=@ dnet:languages @=@ kon @=@ Kongo +dnet:languages @=@ dnet:languages @=@ kok @=@ Konkani +dnet:languages @=@ dnet:languages @=@ kor @=@ Korean +dnet:languages @=@ dnet:languages @=@ kpe @=@ Kpelle +dnet:languages @=@ dnet:languages @=@ kro @=@ Kru +dnet:languages @=@ dnet:languages @=@ kua @=@ Kuanyama; Kwanyama +dnet:languages @=@ dnet:languages @=@ kum @=@ Kumyk +dnet:languages @=@ dnet:languages @=@ kur @=@ Kurdish +dnet:languages @=@ dnet:languages @=@ kru @=@ Kurukh +dnet:languages @=@ dnet:languages @=@ kus @=@ Kusaie +dnet:languages @=@ dnet:languages @=@ kut @=@ Kutenai +dnet:languages @=@ dnet:languages @=@ lad @=@ Ladino +dnet:languages @=@ dnet:languages @=@ lah @=@ Lahnda +dnet:languages @=@ dnet:languages @=@ lam @=@ Lamba +dnet:languages @=@ dnet:languages @=@ lao @=@ Lao +dnet:languages @=@ dnet:languages @=@ lat @=@ Latin +dnet:languages @=@ dnet:languages @=@ lav @=@ Latvian +dnet:languages @=@ dnet:languages @=@ ltz @=@ Letzeburgesch; Luxembourgish +dnet:languages @=@ dnet:languages @=@ lez @=@ Lezghian +dnet:languages @=@ dnet:languages @=@ lim @=@ Limburgan; Limburger; Limburgish +dnet:languages @=@ dnet:languages @=@ lin @=@ Lingala +dnet:languages @=@ dnet:languages @=@ lit @=@ Lithuanian +dnet:languages @=@ dnet:languages @=@ loz @=@ Lozi +dnet:languages @=@ dnet:languages @=@ lub @=@ Luba-Katanga +dnet:languages @=@ dnet:languages @=@ lui @=@ Luiseno +dnet:languages @=@ dnet:languages @=@ lun @=@ Lunda +dnet:languages @=@ dnet:languages @=@ luo @=@ Luo +dnet:languages @=@ dnet:languages @=@ mac/mak @=@ Macedonian +dnet:languages @=@ dnet:languages @=@ mad @=@ Madurese +dnet:languages @=@ dnet:languages @=@ mag @=@ Magahi +dnet:languages @=@ dnet:languages @=@ mai @=@ Maithili +dnet:languages @=@ dnet:languages @=@ mak @=@ Makasar +dnet:languages @=@ dnet:languages @=@ mlg @=@ Malagasy +dnet:languages @=@ dnet:languages @=@ may/msa @=@ Malay +dnet:languages @=@ dnet:languages @=@ mal @=@ Malayalam +dnet:languages @=@ dnet:languages @=@ mlt @=@ Maltese +dnet:languages @=@ dnet:languages @=@ man @=@ Mandingo +dnet:languages @=@ dnet:languages @=@ mni @=@ Manipuri +dnet:languages @=@ dnet:languages @=@ mno @=@ Manobo +dnet:languages @=@ dnet:languages @=@ glv @=@ Manx +dnet:languages @=@ dnet:languages @=@ mao/mri @=@ Maori +dnet:languages @=@ dnet:languages @=@ mar @=@ Marathi +dnet:languages @=@ dnet:languages @=@ chm @=@ Mari +dnet:languages @=@ dnet:languages @=@ mah @=@ Marshallese +dnet:languages @=@ dnet:languages @=@ mwr @=@ Marwari +dnet:languages @=@ dnet:languages @=@ mas @=@ Masai +dnet:languages @=@ dnet:languages @=@ myn @=@ Mayan +dnet:languages @=@ dnet:languages @=@ men @=@ Mende +dnet:languages @=@ dnet:languages @=@ mic @=@ Micmac +dnet:languages @=@ dnet:languages @=@ dum @=@ Middle Dutch +dnet:languages @=@ dnet:languages @=@ enm @=@ Middle English +dnet:languages @=@ dnet:languages @=@ frm @=@ Middle French +dnet:languages @=@ dnet:languages @=@ gmh @=@ Middle High German +dnet:languages @=@ dnet:languages @=@ mga @=@ Middle Irish +dnet:languages @=@ dnet:languages @=@ min @=@ Minangkabau +dnet:languages @=@ dnet:languages @=@ mis @=@ Miscellaneous +dnet:languages @=@ dnet:languages @=@ moh @=@ Mohawk +dnet:languages @=@ dnet:languages @=@ mol @=@ Moldavian +dnet:languages @=@ dnet:languages @=@ mkh @=@ Mon-Kmer +dnet:languages @=@ dnet:languages @=@ lol @=@ Mongo +dnet:languages @=@ dnet:languages @=@ mon @=@ Mongolian +dnet:languages @=@ dnet:languages @=@ mos @=@ Mossi +dnet:languages @=@ dnet:languages @=@ mul @=@ Multiple languages +dnet:languages @=@ dnet:languages @=@ mun @=@ Munda +dnet:languages @=@ dnet:languages @=@ nau @=@ Nauru +dnet:languages @=@ dnet:languages @=@ nav @=@ Navajo; Navaho +dnet:languages @=@ dnet:languages @=@ nde @=@ Ndebele, North +dnet:languages @=@ dnet:languages @=@ nbl @=@ Ndebele, South +dnet:languages @=@ dnet:languages @=@ ndo @=@ Ndonga +dnet:languages @=@ dnet:languages @=@ nep @=@ Nepali +dnet:languages @=@ dnet:languages @=@ new @=@ Newari +dnet:languages @=@ dnet:languages @=@ nic @=@ Niger-Kordofanian +dnet:languages @=@ dnet:languages @=@ ssa @=@ Nilo-Saharan +dnet:languages @=@ dnet:languages @=@ niu @=@ Niuean +dnet:languages @=@ dnet:languages @=@ non @=@ Norse +dnet:languages @=@ dnet:languages @=@ nai @=@ North American Indian +dnet:languages @=@ dnet:languages @=@ sme @=@ Northern Sami +dnet:languages @=@ dnet:languages @=@ nor @=@ Norwegian +dnet:languages @=@ dnet:languages @=@ nno @=@ Norwegian Nynorsk; Nynorsk, Norwegian +dnet:languages @=@ dnet:languages @=@ nub @=@ Nubian +dnet:languages @=@ dnet:languages @=@ nym @=@ Nyamwezi +dnet:languages @=@ dnet:languages @=@ nyn @=@ Nyankole +dnet:languages @=@ dnet:languages @=@ nyo @=@ Nyoro +dnet:languages @=@ dnet:languages @=@ nzi @=@ Nzima +dnet:languages @=@ dnet:languages @=@ oci @=@ Occitan (post 1500); Provençal +dnet:languages @=@ dnet:languages @=@ oji @=@ Ojibwa +dnet:languages @=@ dnet:languages @=@ ang @=@ Old English +dnet:languages @=@ dnet:languages @=@ fro @=@ Old French +dnet:languages @=@ dnet:languages @=@ goh @=@ Old High German +dnet:languages @=@ dnet:languages @=@ ori @=@ Oriya +dnet:languages @=@ dnet:languages @=@ orm @=@ Oromo +dnet:languages @=@ dnet:languages @=@ osa @=@ Osage +dnet:languages @=@ dnet:languages @=@ oss @=@ Ossetian; Ossetic +dnet:languages @=@ dnet:languages @=@ oto @=@ Otomian +dnet:languages @=@ dnet:languages @=@ ota @=@ Ottoman +dnet:languages @=@ dnet:languages @=@ pal @=@ Pahlavi +dnet:languages @=@ dnet:languages @=@ pau @=@ Palauan +dnet:languages @=@ dnet:languages @=@ pli @=@ Pali +dnet:languages @=@ dnet:languages @=@ pam @=@ Pampanga +dnet:languages @=@ dnet:languages @=@ pag @=@ Pangasinan +dnet:languages @=@ dnet:languages @=@ pan @=@ Panjabi; Punjabi +dnet:languages @=@ dnet:languages @=@ pap @=@ Papiamento +dnet:languages @=@ dnet:languages @=@ paa @=@ Papuan-Australian +dnet:languages @=@ dnet:languages @=@ fas/per @=@ Persian +dnet:languages @=@ dnet:languages @=@ peo @=@ Persian, Old (ca 600 - 400 B.C.) +dnet:languages @=@ dnet:languages @=@ phn @=@ Phoenician +dnet:languages @=@ dnet:languages @=@ pol @=@ Polish +dnet:languages @=@ dnet:languages @=@ pon @=@ Ponape +dnet:languages @=@ dnet:languages @=@ por @=@ Portuguese +dnet:languages @=@ dnet:languages @=@ cpp @=@ Portuguese-based Creoles and Pidgins +dnet:languages @=@ dnet:languages @=@ pra @=@ Prakrit +dnet:languages @=@ dnet:languages @=@ pro @=@ Provencal +dnet:languages @=@ dnet:languages @=@ pus @=@ Pushto +dnet:languages @=@ dnet:languages @=@ que @=@ Quechua +dnet:languages @=@ dnet:languages @=@ roh @=@ Raeto-Romance +dnet:languages @=@ dnet:languages @=@ raj @=@ Rajasthani +dnet:languages @=@ dnet:languages @=@ rar @=@ Rarotongan +dnet:languages @=@ dnet:languages @=@ roa @=@ Romance +dnet:languages @=@ dnet:languages @=@ ron/rum @=@ Romanian +dnet:languages @=@ dnet:languages @=@ rom @=@ Romany +dnet:languages @=@ dnet:languages @=@ run @=@ Rundi +dnet:languages @=@ dnet:languages @=@ rus @=@ Russian +dnet:languages @=@ dnet:languages @=@ sal @=@ Salishan +dnet:languages @=@ dnet:languages @=@ sam @=@ Samaritan +dnet:languages @=@ dnet:languages @=@ smi @=@ Sami +dnet:languages @=@ dnet:languages @=@ smo @=@ Samoan +dnet:languages @=@ dnet:languages @=@ sad @=@ Sandawe +dnet:languages @=@ dnet:languages @=@ sag @=@ Sango +dnet:languages @=@ dnet:languages @=@ san @=@ Sanskrit +dnet:languages @=@ dnet:languages @=@ srd @=@ Sardinian +dnet:languages @=@ dnet:languages @=@ sco @=@ Scots +dnet:languages @=@ dnet:languages @=@ sel @=@ Selkup +dnet:languages @=@ dnet:languages @=@ sem @=@ Semitic +dnet:languages @=@ dnet:languages @=@ srp @=@ Serbian +dnet:languages @=@ dnet:languages @=@ scr @=@ Serbo-Croatian +dnet:languages @=@ dnet:languages @=@ srr @=@ Serer +dnet:languages @=@ dnet:languages @=@ shn @=@ Shan +dnet:languages @=@ dnet:languages @=@ sna @=@ Shona +dnet:languages @=@ dnet:languages @=@ iii @=@ Sichuan Yi +dnet:languages @=@ dnet:languages @=@ sid @=@ Sidamo +dnet:languages @=@ dnet:languages @=@ bla @=@ Siksika +dnet:languages @=@ dnet:languages @=@ snd @=@ Sindhi +dnet:languages @=@ dnet:languages @=@ sin @=@ Sinhala; Sinhalese +dnet:languages @=@ dnet:languages @=@ sit @=@ Sino-Tibetan +dnet:languages @=@ dnet:languages @=@ sio @=@ Siouan +dnet:languages @=@ dnet:languages @=@ sla @=@ Slavic +dnet:languages @=@ dnet:languages @=@ slk/slo @=@ Slovak +dnet:languages @=@ dnet:languages @=@ slv @=@ Slovenian +dnet:languages @=@ dnet:languages @=@ sog @=@ Sogdian +dnet:languages @=@ dnet:languages @=@ som @=@ Somali +dnet:languages @=@ dnet:languages @=@ son @=@ Songhai +dnet:languages @=@ dnet:languages @=@ wen @=@ Sorbian +dnet:languages @=@ dnet:languages @=@ nso @=@ Sotho +dnet:languages @=@ dnet:languages @=@ sot @=@ Sotho, Southern +dnet:languages @=@ dnet:languages @=@ sai @=@ South American Indian +dnet:languages @=@ dnet:languages @=@ esl/spa @=@ Spanish +dnet:languages @=@ dnet:languages @=@ spa @=@ Spanish; Castilian +dnet:languages @=@ dnet:languages @=@ suk @=@ Sukuma +dnet:languages @=@ dnet:languages @=@ sux @=@ Sumerian +dnet:languages @=@ dnet:languages @=@ sun @=@ Sundanese +dnet:languages @=@ dnet:languages @=@ sus @=@ Susu +dnet:languages @=@ dnet:languages @=@ swa @=@ Swahili +dnet:languages @=@ dnet:languages @=@ ssw @=@ Swati +dnet:languages @=@ dnet:languages @=@ swe @=@ Swedish +dnet:languages @=@ dnet:languages @=@ syr @=@ Syriac +dnet:languages @=@ dnet:languages @=@ tgl @=@ Tagalog +dnet:languages @=@ dnet:languages @=@ tah @=@ Tahitian +dnet:languages @=@ dnet:languages @=@ tgk @=@ Tajik +dnet:languages @=@ dnet:languages @=@ tmh @=@ Tamashek +dnet:languages @=@ dnet:languages @=@ tam @=@ Tamil +dnet:languages @=@ dnet:languages @=@ tat @=@ Tatar +dnet:languages @=@ dnet:languages @=@ tel @=@ Telugu +dnet:languages @=@ dnet:languages @=@ ter @=@ Tereno +dnet:languages @=@ dnet:languages @=@ tha @=@ Thai +dnet:languages @=@ dnet:languages @=@ bod/tib @=@ Tibetan +dnet:languages @=@ dnet:languages @=@ tig @=@ Tigre +dnet:languages @=@ dnet:languages @=@ tir @=@ Tigrinya +dnet:languages @=@ dnet:languages @=@ tem @=@ Timne +dnet:languages @=@ dnet:languages @=@ tiv @=@ Tivi +dnet:languages @=@ dnet:languages @=@ tli @=@ Tlingit +dnet:languages @=@ dnet:languages @=@ ton @=@ Tonga (Tonga Islands) +dnet:languages @=@ dnet:languages @=@ tog @=@ Tonga(Nyasa) +dnet:languages @=@ dnet:languages @=@ tru @=@ Truk +dnet:languages @=@ dnet:languages @=@ tsi @=@ Tsimshian +dnet:languages @=@ dnet:languages @=@ tso @=@ Tsonga +dnet:languages @=@ dnet:languages @=@ tsn @=@ Tswana +dnet:languages @=@ dnet:languages @=@ tum @=@ Tumbuka +dnet:languages @=@ dnet:languages @=@ tur @=@ Turkish +dnet:languages @=@ dnet:languages @=@ tuk @=@ Turkmen +dnet:languages @=@ dnet:languages @=@ tyv @=@ Tuvinian +dnet:languages @=@ dnet:languages @=@ twi @=@ Twi +dnet:languages @=@ dnet:languages @=@ uga @=@ Ugaritic +dnet:languages @=@ dnet:languages @=@ uig @=@ Uighur; Uyghur +dnet:languages @=@ dnet:languages @=@ ukr @=@ Ukrainian +dnet:languages @=@ dnet:languages @=@ umb @=@ Umbundu +dnet:languages @=@ dnet:languages @=@ und @=@ Undetermined +dnet:languages @=@ dnet:languages @=@ urd @=@ Urdu +dnet:languages @=@ dnet:languages @=@ uzb @=@ Uzbek +dnet:languages @=@ dnet:languages @=@ vai @=@ Vai +dnet:languages @=@ dnet:languages @=@ ven @=@ Venda +dnet:languages @=@ dnet:languages @=@ vie @=@ Vietnamese +dnet:languages @=@ dnet:languages @=@ vol @=@ Volapük +dnet:languages @=@ dnet:languages @=@ vot @=@ Votic +dnet:languages @=@ dnet:languages @=@ wak @=@ Wakashan +dnet:languages @=@ dnet:languages @=@ wal @=@ Walamo +dnet:languages @=@ dnet:languages @=@ wln @=@ Walloon +dnet:languages @=@ dnet:languages @=@ war @=@ Waray +dnet:languages @=@ dnet:languages @=@ was @=@ Washo +dnet:languages @=@ dnet:languages @=@ cym/wel @=@ Welsh +dnet:languages @=@ dnet:languages @=@ wol @=@ Wolof +dnet:languages @=@ dnet:languages @=@ xho @=@ Xhosa +dnet:languages @=@ dnet:languages @=@ sah @=@ Yakut +dnet:languages @=@ dnet:languages @=@ yao @=@ Yao +dnet:languages @=@ dnet:languages @=@ yap @=@ Yap +dnet:languages @=@ dnet:languages @=@ yid @=@ Yiddish +dnet:languages @=@ dnet:languages @=@ yor @=@ Yoruba +dnet:languages @=@ dnet:languages @=@ zap @=@ Zapotec +dnet:languages @=@ dnet:languages @=@ zen @=@ Zenaga +dnet:languages @=@ dnet:languages @=@ zha @=@ Zhuang; Chuang +dnet:languages @=@ dnet:languages @=@ zul @=@ Zulu +dnet:languages @=@ dnet:languages @=@ zun @=@ Zuni +dnet:languages @=@ dnet:languages @=@ sga @=@ old Irish +nsf:contractTypes @=@ NSF Contract Types @=@ BOA/Task Order @=@ BOA/Task Order +nsf:contractTypes @=@ NSF Contract Types @=@ Continuing grant @=@ Continuing grant +nsf:contractTypes @=@ NSF Contract Types @=@ Contract @=@ Contract +nsf:contractTypes @=@ NSF Contract Types @=@ Contract Interagency Agreement @=@ Contract Interagency Agreement +nsf:contractTypes @=@ NSF Contract Types @=@ Cooperative Agreement @=@ Cooperative Agreement +nsf:contractTypes @=@ NSF Contract Types @=@ Fellowship @=@ Fellowship +nsf:contractTypes @=@ NSF Contract Types @=@ Fixed Price Award @=@ Fixed Price Award +nsf:contractTypes @=@ NSF Contract Types @=@ GAA @=@ GAA +nsf:contractTypes @=@ NSF Contract Types @=@ Interagency Agreement @=@ Interagency Agreement +nsf:contractTypes @=@ NSF Contract Types @=@ Intergovernmental Personnel Award @=@ Intergovernmental Personnel Award +nsf:contractTypes @=@ NSF Contract Types @=@ Personnel Agreement @=@ Personnel Agreement +nsf:contractTypes @=@ NSF Contract Types @=@ Standard Grant @=@ Standard Grant +ec:funding_relations @=@ ec:funding_relations @=@ ec:hasframeworkprogram @=@ hasframeworkprogram +ec:funding_relations @=@ ec:funding_relations @=@ ec:hasprogram @=@ hasprogram +ec:funding_relations @=@ ec:funding_relations @=@ ec:hasspecificprogram @=@ hasspecificprogram +dnet:dataCite_resource @=@ dnet:dataCite_resource @=@ UNKNOWN @=@ UNKNOWN +dnet:dataCite_resource @=@ dnet:dataCite_resource @=@ collection @=@ collection +dnet:dataCite_resource @=@ dnet:dataCite_resource @=@ dataset @=@ dataset +dnet:dataCite_resource @=@ dnet:dataCite_resource @=@ event @=@ event +dnet:dataCite_resource @=@ dnet:dataCite_resource @=@ film @=@ film +dnet:dataCite_resource @=@ dnet:dataCite_resource @=@ image @=@ image +dnet:dataCite_resource @=@ dnet:dataCite_resource @=@ interactiveResource @=@ interactiveResource +dnet:dataCite_resource @=@ dnet:dataCite_resource @=@ model @=@ model +dnet:dataCite_resource @=@ dnet:dataCite_resource @=@ physicalObject @=@ physicalObject +dnet:dataCite_resource @=@ dnet:dataCite_resource @=@ service @=@ service +dnet:dataCite_resource @=@ dnet:dataCite_resource @=@ software @=@ software +dnet:dataCite_resource @=@ dnet:dataCite_resource @=@ sound @=@ sound +dnet:dataCite_resource @=@ dnet:dataCite_resource @=@ text @=@ text +dnet:dataCite_resource @=@ dnet:dataCite_resource @=@ clinicalTrial @=@ Clinical trial +dnet:dataCite_title @=@ dnet:dataCite_title @=@ alternative title @=@ alternative title +dnet:dataCite_title @=@ dnet:dataCite_title @=@ main title @=@ main title +dnet:dataCite_title @=@ dnet:dataCite_title @=@ subtitle @=@ subtitle +dnet:dataCite_title @=@ dnet:dataCite_title @=@ translated title @=@ translated title +datacite:relation_typologies @=@ datacite:relation_typologies @=@ IsCitedBy @=@ IsCitedBy +datacite:relation_typologies @=@ datacite:relation_typologies @=@ IsNewVersionOf @=@ IsNewVersionOf +datacite:relation_typologies @=@ datacite:relation_typologies @=@ IsPartOf @=@ IsPartOf +datacite:relation_typologies @=@ datacite:relation_typologies @=@ IsPreviousVersionOf @=@ IsPreviousVersionOf +datacite:relation_typologies @=@ datacite:relation_typologies @=@ IsReferencedBy @=@ IsReferencedBy +datacite:relation_typologies @=@ datacite:relation_typologies @=@ References @=@ References +datacite:relation_typologies @=@ datacite:relation_typologies @=@ UNKNOWN @=@ UNKNOWN +dnet:result_typologies @=@ dnet:result_typologies @=@ dataset @=@ dataset +dnet:result_typologies @=@ dnet:result_typologies @=@ other @=@ other +dnet:result_typologies @=@ dnet:result_typologies @=@ publication @=@ publication +dnet:result_typologies @=@ dnet:result_typologies @=@ software @=@ software +ec:h2020toas @=@ Horizon 2020 - Type of Actions @=@ ERC-ADG @=@ Advanced Grant +ec:h2020toas @=@ Horizon 2020 - Type of Actions @=@ BBI-CSA @=@ Bio-based Industries Coordination and Support action +ec:h2020toas @=@ Horizon 2020 - Type of Actions @=@ BBI-IA-DEMO @=@ Bio-based Industries Innovation action - Demonstration +ec:h2020toas @=@ Horizon 2020 - Type of Actions @=@ BBI-IA-FLAG @=@ Bio-based Industries Innovation action - Flagship +ec:h2020toas @=@ Horizon 2020 - Type of Actions @=@ BBI-RIA @=@ Bio-based Industries Research and Innovation action +ec:h2020toas @=@ Horizon 2020 - Type of Actions @=@ MSCA-IF-EF-CAR @=@ CAR – Career Restart panel +ec:h2020toas @=@ Horizon 2020 - Type of Actions @=@ COFUND-EJP @=@ COFUND (European Joint Programme) +ec:h2020toas @=@ Horizon 2020 - Type of Actions @=@ COFUND-PCP @=@ COFUND (PCP) +ec:h2020toas @=@ Horizon 2020 - Type of Actions @=@ COFUND-PPI @=@ COFUND (PPI) +ec:h2020toas @=@ Horizon 2020 - Type of Actions @=@ CS2-CSA @=@ CS2 Coordination and Support action +ec:h2020toas @=@ Horizon 2020 - Type of Actions @=@ CS2-IA @=@ CS2 Innovation Action +ec:h2020toas @=@ Horizon 2020 - Type of Actions @=@ CS2-RIA @=@ CS2 Research and Innovation action +ec:h2020toas @=@ Horizon 2020 - Type of Actions @=@ CSA-LS @=@ CSA Lump sum +ec:h2020toas @=@ Horizon 2020 - Type of Actions @=@ ERC-COG @=@ Consolidator Grant +ec:h2020toas @=@ Horizon 2020 - Type of Actions @=@ FCH2-CSA @=@ Coordination & support action +ec:h2020toas @=@ Horizon 2020 - Type of Actions @=@ CSA @=@ Coordination and support action +ec:h2020toas @=@ Horizon 2020 - Type of Actions @=@ MSCA-COFUND-DP @=@ Doctoral programmes +ec:h2020toas @=@ Horizon 2020 - Type of Actions @=@ ECSEL-CSA @=@ ECSEL Coordination & Support action +ec:h2020toas @=@ Horizon 2020 - Type of Actions @=@ ECSEL-IA @=@ ECSEL Innovation Action +ec:h2020toas @=@ Horizon 2020 - Type of Actions @=@ ECSEL-RIA @=@ ECSEL Research and Innovation Actions +ec:h2020toas @=@ Horizon 2020 - Type of Actions @=@ ERA-NET-Cofund @=@ ERA-NET Cofund +ec:h2020toas @=@ Horizon 2020 - Type of Actions @=@ ERC-POC-LS @=@ ERC Proof of Concept Lump Sum Pilot +ec:h2020toas @=@ Horizon 2020 - Type of Actions @=@ ERC-SyG @=@ ERC Synergy Grant +ec:h2020toas @=@ Horizon 2020 - Type of Actions @=@ ERC-LVG @=@ ERC low value grant +ec:h2020toas @=@ Horizon 2020 - Type of Actions @=@ H2020-EEN-SGA @=@ Enterprise Europe Network - Specific Grant Agreement +ec:h2020toas @=@ Horizon 2020 - Type of Actions @=@ MSCA-ITN-EID @=@ European Industrial Doctorates +ec:h2020toas @=@ Horizon 2020 - Type of Actions @=@ MSCA-ITN-EJD @=@ European Joint Doctorates +ec:h2020toas @=@ Horizon 2020 - Type of Actions @=@ MSCA-ITN-ETN @=@ European Training Networks +ec:h2020toas @=@ Horizon 2020 - Type of Actions @=@ FCH2-IA @=@ FCH2 Innovation action +ec:h2020toas @=@ Horizon 2020 - Type of Actions @=@ FCH2-RIA @=@ FCH2 Research and Innovation action +ec:h2020toas @=@ Horizon 2020 - Type of Actions @=@ MSCA-COFUND-FP @=@ Fellowship programmes +ec:h2020toas @=@ Horizon 2020 - Type of Actions @=@ MSCA-IF-GF @=@ Global Fellowships +ec:h2020toas @=@ Horizon 2020 - Type of Actions @=@ IMI2-CSA @=@ IMI2 Coordination & support action +ec:h2020toas @=@ Horizon 2020 - Type of Actions @=@ IMI2-RIA @=@ IMI2 Research and Innovation action +ec:h2020toas @=@ Horizon 2020 - Type of Actions @=@ Shift2Rail-IA-LS @=@ Innovation Action Lump-Sum +ec:h2020toas @=@ Horizon 2020 - Type of Actions @=@ IA-LS @=@ Innovation Action Lump-Sum +ec:h2020toas @=@ Horizon 2020 - Type of Actions @=@ IA @=@ Innovation action +ec:h2020toas @=@ Horizon 2020 - Type of Actions @=@ Shift2Rail-IA @=@ Innovation action +ec:h2020toas @=@ Horizon 2020 - Type of Actions @=@ PCP @=@ Pre-Commercial Procurement +ec:h2020toas @=@ Horizon 2020 - Type of Actions @=@ ERC-POC @=@ Proof of Concept Grant +ec:h2020toas @=@ Horizon 2020 - Type of Actions @=@ PPI @=@ Public Procurement of Innovative Solutions +ec:h2020toas @=@ Horizon 2020 - Type of Actions @=@ MSCA-IF-EF-RI @=@ RI – Reintegration panel +ec:h2020toas @=@ Horizon 2020 - Type of Actions @=@ MSCA-RISE @=@ RISE +ec:h2020toas @=@ Horizon 2020 - Type of Actions @=@ Shift2Rail-RIA-LS @=@ Research and Innovation Action Lump-Sum +ec:h2020toas @=@ Horizon 2020 - Type of Actions @=@ Shift2Rail-RIA @=@ Research and Innovation action +ec:h2020toas @=@ Horizon 2020 - Type of Actions @=@ RIA @=@ Research and Innovation action +ec:h2020toas @=@ Horizon 2020 - Type of Actions @=@ RIA-LS @=@ Research and Innovation action Lump Sum +ec:h2020toas @=@ Horizon 2020 - Type of Actions @=@ SESAR-CSA @=@ SESAR: Coordination and Support Action +ec:h2020toas @=@ Horizon 2020 - Type of Actions @=@ SESAR-IA @=@ SESAR: Innovation action +ec:h2020toas @=@ Horizon 2020 - Type of Actions @=@ SESAR-RIA @=@ SESAR: Research and Innovation action +ec:h2020toas @=@ Horizon 2020 - Type of Actions @=@ SGA-RIA @=@ SGA Research and Innovation action +ec:h2020toas @=@ Horizon 2020 - Type of Actions @=@ SME-2b @=@ SME Instrument (grant only and blended finance) +ec:h2020toas @=@ Horizon 2020 - Type of Actions @=@ SME-1 @=@ SME instrument phase 1 +ec:h2020toas @=@ Horizon 2020 - Type of Actions @=@ SME-2 @=@ SME instrument phase 2 +ec:h2020toas @=@ Horizon 2020 - Type of Actions @=@ Shift2Rail-CSA @=@ Shift2Rail - Coordination and Support action +ec:h2020toas @=@ Horizon 2020 - Type of Actions @=@ MSCA-IF-EF-SE @=@ Society and Enterprise panel +ec:h2020toas @=@ Horizon 2020 - Type of Actions @=@ SGA-CSA @=@ Specific Grant agreement and Coordination and Support Action +ec:h2020toas @=@ Horizon 2020 - Type of Actions @=@ MSCA-IF-EF-ST @=@ Standard EF +ec:h2020toas @=@ Horizon 2020 - Type of Actions @=@ ERC-STG @=@ Starting Grant +ec:h2020toas @=@ Horizon 2020 - Type of Actions @=@ MSCA-SNLS @=@ Grant to identified beneficiary - Coordination and support actions (MSCA-Special Needs lump sum) +wt:contractTypes @=@ wt:contractTypes @=@ UNKNOWN @=@ UNKNOWN +dnet:countries @=@ dnet:countries @=@ AF @=@ Afghanistan +dnet:countries @=@ dnet:countries @=@ AL @=@ Albania +dnet:countries @=@ dnet:countries @=@ DZ @=@ Algeria +dnet:countries @=@ dnet:countries @=@ AS @=@ American Samoa +dnet:countries @=@ dnet:countries @=@ AD @=@ Andorra +dnet:countries @=@ dnet:countries @=@ AO @=@ Angola +dnet:countries @=@ dnet:countries @=@ AI @=@ Anguilla +dnet:countries @=@ dnet:countries @=@ AQ @=@ Antarctica +dnet:countries @=@ dnet:countries @=@ AG @=@ Antigua and Barbuda +dnet:countries @=@ dnet:countries @=@ AR @=@ Argentina +dnet:countries @=@ dnet:countries @=@ AM @=@ Armenia +dnet:countries @=@ dnet:countries @=@ AW @=@ Aruba +dnet:countries @=@ dnet:countries @=@ AU @=@ Australia +dnet:countries @=@ dnet:countries @=@ AT @=@ Austria +dnet:countries @=@ dnet:countries @=@ AZ @=@ Azerbaijan +dnet:countries @=@ dnet:countries @=@ BS @=@ Bahamas +dnet:countries @=@ dnet:countries @=@ BH @=@ Bahrain +dnet:countries @=@ dnet:countries @=@ BD @=@ Bangladesh +dnet:countries @=@ dnet:countries @=@ BB @=@ Barbados +dnet:countries @=@ dnet:countries @=@ BY @=@ Belarus +dnet:countries @=@ dnet:countries @=@ BE @=@ Belgium +dnet:countries @=@ dnet:countries @=@ BZ @=@ Belize +dnet:countries @=@ dnet:countries @=@ BJ @=@ Benin +dnet:countries @=@ dnet:countries @=@ BM @=@ Bermuda +dnet:countries @=@ dnet:countries @=@ BT @=@ Bhutan +dnet:countries @=@ dnet:countries @=@ BO @=@ Bolivia +dnet:countries @=@ dnet:countries @=@ BQ @=@ Bonaire, Sint Eustatius and Saba +dnet:countries @=@ dnet:countries @=@ BA @=@ Bosnia and Herzegovina +dnet:countries @=@ dnet:countries @=@ BW @=@ Botswana +dnet:countries @=@ dnet:countries @=@ BV @=@ Bouvet Island +dnet:countries @=@ dnet:countries @=@ BR @=@ Brazil +dnet:countries @=@ dnet:countries @=@ IO @=@ British Indian Ocean Territory +dnet:countries @=@ dnet:countries @=@ BN @=@ Brunei Darussalam +dnet:countries @=@ dnet:countries @=@ BG @=@ Bulgaria +dnet:countries @=@ dnet:countries @=@ BF @=@ Burkina Faso +dnet:countries @=@ dnet:countries @=@ BI @=@ Burundi +dnet:countries @=@ dnet:countries @=@ KH @=@ Cambodia +dnet:countries @=@ dnet:countries @=@ CM @=@ Cameroon +dnet:countries @=@ dnet:countries @=@ CA @=@ Canada +dnet:countries @=@ dnet:countries @=@ CV @=@ Cape Verde +dnet:countries @=@ dnet:countries @=@ KY @=@ Cayman Islands +dnet:countries @=@ dnet:countries @=@ CF @=@ Central African Republic +dnet:countries @=@ dnet:countries @=@ TD @=@ Chad +dnet:countries @=@ dnet:countries @=@ CL @=@ Chile +dnet:countries @=@ dnet:countries @=@ CN @=@ China (People's Republic of) +dnet:countries @=@ dnet:countries @=@ CX @=@ Christmas Island +dnet:countries @=@ dnet:countries @=@ CC @=@ Cocos (Keeling) Islands +dnet:countries @=@ dnet:countries @=@ CO @=@ Colombia +dnet:countries @=@ dnet:countries @=@ KM @=@ Comoros +dnet:countries @=@ dnet:countries @=@ CG @=@ Congo +dnet:countries @=@ dnet:countries @=@ CD @=@ Congo (Democratic Republic of) +dnet:countries @=@ dnet:countries @=@ CK @=@ Cook Islands +dnet:countries @=@ dnet:countries @=@ CR @=@ Costa Rica +dnet:countries @=@ dnet:countries @=@ CI @=@ Cote d'Ivoire +dnet:countries @=@ dnet:countries @=@ HR @=@ Croatia +dnet:countries @=@ dnet:countries @=@ CU @=@ Cuba +dnet:countries @=@ dnet:countries @=@ CW @=@ Curaçao +dnet:countries @=@ dnet:countries @=@ CY @=@ Cyprus +dnet:countries @=@ dnet:countries @=@ CZ @=@ Czech Republic +dnet:countries @=@ dnet:countries @=@ DK @=@ Denmark +dnet:countries @=@ dnet:countries @=@ DJ @=@ Djibouti +dnet:countries @=@ dnet:countries @=@ DM @=@ Dominica +dnet:countries @=@ dnet:countries @=@ DO @=@ Dominican Republic +dnet:countries @=@ dnet:countries @=@ EC @=@ Ecuador +dnet:countries @=@ dnet:countries @=@ EG @=@ Egypt +dnet:countries @=@ dnet:countries @=@ SV @=@ El Salvador +dnet:countries @=@ dnet:countries @=@ GQ @=@ Equatorial Guinea +dnet:countries @=@ dnet:countries @=@ ER @=@ Eritrea +dnet:countries @=@ dnet:countries @=@ EE @=@ Estonia +dnet:countries @=@ dnet:countries @=@ ET @=@ Ethiopia +dnet:countries @=@ dnet:countries @=@ EU @=@ European Union +dnet:countries @=@ dnet:countries @=@ FK @=@ Falkland Islands (Malvinas) +dnet:countries @=@ dnet:countries @=@ FO @=@ Faroe Islands +dnet:countries @=@ dnet:countries @=@ FJ @=@ Fiji +dnet:countries @=@ dnet:countries @=@ FI @=@ Finland +dnet:countries @=@ dnet:countries @=@ MK @=@ Former Yugoslav Republic of Macedonia +dnet:countries @=@ dnet:countries @=@ FR @=@ France +dnet:countries @=@ dnet:countries @=@ GF @=@ French Guiana +dnet:countries @=@ dnet:countries @=@ PF @=@ French Polynesia +dnet:countries @=@ dnet:countries @=@ TF @=@ French Southern Territories +dnet:countries @=@ dnet:countries @=@ GA @=@ Gabon +dnet:countries @=@ dnet:countries @=@ GM @=@ Gambia +dnet:countries @=@ dnet:countries @=@ GE @=@ Georgia +dnet:countries @=@ dnet:countries @=@ DE @=@ Germany +dnet:countries @=@ dnet:countries @=@ GH @=@ Ghana +dnet:countries @=@ dnet:countries @=@ GI @=@ Gibraltar +dnet:countries @=@ dnet:countries @=@ GR @=@ Greece +dnet:countries @=@ dnet:countries @=@ GL @=@ Greenland +dnet:countries @=@ dnet:countries @=@ GD @=@ Grenada +dnet:countries @=@ dnet:countries @=@ GP @=@ Guadeloupe +dnet:countries @=@ dnet:countries @=@ GU @=@ Guam +dnet:countries @=@ dnet:countries @=@ GT @=@ Guatemala +dnet:countries @=@ dnet:countries @=@ GG @=@ Guernsey +dnet:countries @=@ dnet:countries @=@ GN @=@ Guinea +dnet:countries @=@ dnet:countries @=@ GW @=@ Guinea-Bissau +dnet:countries @=@ dnet:countries @=@ GY @=@ Guyana +dnet:countries @=@ dnet:countries @=@ HT @=@ Haiti +dnet:countries @=@ dnet:countries @=@ HM @=@ Heard Island and McDonald Islands +dnet:countries @=@ dnet:countries @=@ VA @=@ Holy See (Vatican City State) +dnet:countries @=@ dnet:countries @=@ HN @=@ Honduras +dnet:countries @=@ dnet:countries @=@ HK @=@ Hong Kong +dnet:countries @=@ dnet:countries @=@ HU @=@ Hungary +dnet:countries @=@ dnet:countries @=@ IS @=@ Iceland +dnet:countries @=@ dnet:countries @=@ IN @=@ India +dnet:countries @=@ dnet:countries @=@ ID @=@ Indonesia +dnet:countries @=@ dnet:countries @=@ IR @=@ Iran (Islamic Republic of) +dnet:countries @=@ dnet:countries @=@ IQ @=@ Iraq +dnet:countries @=@ dnet:countries @=@ IE @=@ Ireland +dnet:countries @=@ dnet:countries @=@ IM @=@ Isle of Man +dnet:countries @=@ dnet:countries @=@ IL @=@ Israel +dnet:countries @=@ dnet:countries @=@ IT @=@ Italy +dnet:countries @=@ dnet:countries @=@ JM @=@ Jamaica +dnet:countries @=@ dnet:countries @=@ JP @=@ Japan +dnet:countries @=@ dnet:countries @=@ JE @=@ Jersey +dnet:countries @=@ dnet:countries @=@ JO @=@ Jordan +dnet:countries @=@ dnet:countries @=@ KZ @=@ Kazakhstan +dnet:countries @=@ dnet:countries @=@ KE @=@ Kenya +dnet:countries @=@ dnet:countries @=@ KI @=@ Kiribati +dnet:countries @=@ dnet:countries @=@ KR @=@ Korea (Republic of) +dnet:countries @=@ dnet:countries @=@ KP @=@ Korea, Democatric People's Republic of +dnet:countries @=@ dnet:countries @=@ XK @=@ Kosovo * UN resolution +dnet:countries @=@ dnet:countries @=@ KW @=@ Kuwait +dnet:countries @=@ dnet:countries @=@ KG @=@ Kyrgyzstan +dnet:countries @=@ dnet:countries @=@ LA @=@ Lao (People's Democratic Republic) +dnet:countries @=@ dnet:countries @=@ LV @=@ Latvia +dnet:countries @=@ dnet:countries @=@ LB @=@ Lebanon +dnet:countries @=@ dnet:countries @=@ LS @=@ Lesotho +dnet:countries @=@ dnet:countries @=@ LR @=@ Liberia +dnet:countries @=@ dnet:countries @=@ LY @=@ Libyan Arab Jamahiriya +dnet:countries @=@ dnet:countries @=@ LI @=@ Liechtenstein +dnet:countries @=@ dnet:countries @=@ LT @=@ Lithuania +dnet:countries @=@ dnet:countries @=@ LU @=@ Luxembourg +dnet:countries @=@ dnet:countries @=@ MO @=@ Macao +dnet:countries @=@ dnet:countries @=@ MG @=@ Madagascar +dnet:countries @=@ dnet:countries @=@ MW @=@ Malawi +dnet:countries @=@ dnet:countries @=@ MY @=@ Malaysia +dnet:countries @=@ dnet:countries @=@ MV @=@ Maldives +dnet:countries @=@ dnet:countries @=@ ML @=@ Mali +dnet:countries @=@ dnet:countries @=@ MT @=@ Malta +dnet:countries @=@ dnet:countries @=@ MH @=@ Marshall Islands +dnet:countries @=@ dnet:countries @=@ MQ @=@ Martinique +dnet:countries @=@ dnet:countries @=@ MR @=@ Mauritania +dnet:countries @=@ dnet:countries @=@ MU @=@ Mauritius +dnet:countries @=@ dnet:countries @=@ YT @=@ Mayotte +dnet:countries @=@ dnet:countries @=@ MX @=@ Mexico +dnet:countries @=@ dnet:countries @=@ FM @=@ Micronesia, Federated States of +dnet:countries @=@ dnet:countries @=@ MD @=@ Moldova (Republic of) +dnet:countries @=@ dnet:countries @=@ MN @=@ Mongolia +dnet:countries @=@ dnet:countries @=@ ME @=@ Montenegro +dnet:countries @=@ dnet:countries @=@ MS @=@ Montserrat +dnet:countries @=@ dnet:countries @=@ MA @=@ Morocco +dnet:countries @=@ dnet:countries @=@ MZ @=@ Mozambique +dnet:countries @=@ dnet:countries @=@ MM @=@ Myanmar +dnet:countries @=@ dnet:countries @=@ NA @=@ Namibia +dnet:countries @=@ dnet:countries @=@ NR @=@ Nauru +dnet:countries @=@ dnet:countries @=@ NP @=@ Nepal +dnet:countries @=@ dnet:countries @=@ NL @=@ Netherlands +dnet:countries @=@ dnet:countries @=@ AN @=@ Netherlands Antilles +dnet:countries @=@ dnet:countries @=@ NC @=@ New Caledonia +dnet:countries @=@ dnet:countries @=@ NZ @=@ New Zealand +dnet:countries @=@ dnet:countries @=@ NI @=@ Nicaragua +dnet:countries @=@ dnet:countries @=@ NE @=@ Niger +dnet:countries @=@ dnet:countries @=@ NG @=@ Nigeria +dnet:countries @=@ dnet:countries @=@ NU @=@ Niue +dnet:countries @=@ dnet:countries @=@ NF @=@ Norfolk Island +dnet:countries @=@ dnet:countries @=@ MP @=@ Northern Mariana Islands +dnet:countries @=@ dnet:countries @=@ NO @=@ Norway +dnet:countries @=@ dnet:countries @=@ OC @=@ Oceania +dnet:countries @=@ dnet:countries @=@ OM @=@ Oman +dnet:countries @=@ dnet:countries @=@ PK @=@ Pakistan +dnet:countries @=@ dnet:countries @=@ PW @=@ Palau +dnet:countries @=@ dnet:countries @=@ PS @=@ Palestinian-administered areas +dnet:countries @=@ dnet:countries @=@ PA @=@ Panama +dnet:countries @=@ dnet:countries @=@ PG @=@ Papua New Guinea +dnet:countries @=@ dnet:countries @=@ PY @=@ Paraguay +dnet:countries @=@ dnet:countries @=@ PE @=@ Peru +dnet:countries @=@ dnet:countries @=@ PH @=@ Philippines +dnet:countries @=@ dnet:countries @=@ PN @=@ Pitcairn +dnet:countries @=@ dnet:countries @=@ PL @=@ Poland +dnet:countries @=@ dnet:countries @=@ PT @=@ Portugal +dnet:countries @=@ dnet:countries @=@ PR @=@ Puerto Rico +dnet:countries @=@ dnet:countries @=@ QA @=@ Qatar +dnet:countries @=@ dnet:countries @=@ RO @=@ Romania +dnet:countries @=@ dnet:countries @=@ RU @=@ Russian Federation +dnet:countries @=@ dnet:countries @=@ RW @=@ Rwanda +dnet:countries @=@ dnet:countries @=@ RE @=@ Réunion +dnet:countries @=@ dnet:countries @=@ SH @=@ Saint Helena, Ascension and Tristan da Cunha +dnet:countries @=@ dnet:countries @=@ KN @=@ Saint Kitts and Nevis +dnet:countries @=@ dnet:countries @=@ LC @=@ Saint Lucia +dnet:countries @=@ dnet:countries @=@ MF @=@ Saint Martin (French Part) +dnet:countries @=@ dnet:countries @=@ PM @=@ Saint Pierre and Miquelon +dnet:countries @=@ dnet:countries @=@ VC @=@ Saint Vincent and the Grenadines +dnet:countries @=@ dnet:countries @=@ BL @=@ Saint-Barthélemy +dnet:countries @=@ dnet:countries @=@ WS @=@ Samoa +dnet:countries @=@ dnet:countries @=@ SM @=@ San Marino +dnet:countries @=@ dnet:countries @=@ SA @=@ Saudi Arabia +dnet:countries @=@ dnet:countries @=@ SN @=@ Senegal +dnet:countries @=@ dnet:countries @=@ RS @=@ Serbia +dnet:countries @=@ dnet:countries @=@ CS @=@ Serbia and Montenegro +dnet:countries @=@ dnet:countries @=@ SC @=@ Seychelles +dnet:countries @=@ dnet:countries @=@ SL @=@ Sierra Leone +dnet:countries @=@ dnet:countries @=@ SG @=@ Singapore +dnet:countries @=@ dnet:countries @=@ SX @=@ Sint Maarten (Dutch Part) +dnet:countries @=@ dnet:countries @=@ SK @=@ Slovakia +dnet:countries @=@ dnet:countries @=@ SI @=@ Slovenia +dnet:countries @=@ dnet:countries @=@ SB @=@ Solomon Islands +dnet:countries @=@ dnet:countries @=@ SO @=@ Somalia +dnet:countries @=@ dnet:countries @=@ ZA @=@ South Africa +dnet:countries @=@ dnet:countries @=@ GS @=@ South Georgia and the South Sandwich Islands +dnet:countries @=@ dnet:countries @=@ SS @=@ South Sudan +dnet:countries @=@ dnet:countries @=@ ES @=@ Spain +dnet:countries @=@ dnet:countries @=@ LK @=@ Sri Lanka +dnet:countries @=@ dnet:countries @=@ SD @=@ Sudan +dnet:countries @=@ dnet:countries @=@ SR @=@ Suriname +dnet:countries @=@ dnet:countries @=@ SJ @=@ Svalbard and Jan Mayen +dnet:countries @=@ dnet:countries @=@ SZ @=@ Swaziland +dnet:countries @=@ dnet:countries @=@ SE @=@ Sweden +dnet:countries @=@ dnet:countries @=@ CH @=@ Switzerland +dnet:countries @=@ dnet:countries @=@ SY @=@ Syrian Arab Republic +dnet:countries @=@ dnet:countries @=@ ST @=@ São Tomé and Príncipe +dnet:countries @=@ dnet:countries @=@ TW @=@ Taiwan +dnet:countries @=@ dnet:countries @=@ TJ @=@ Tajikistan +dnet:countries @=@ dnet:countries @=@ TZ @=@ Tanzania (United Republic of) +dnet:countries @=@ dnet:countries @=@ TH @=@ Thailand +dnet:countries @=@ dnet:countries @=@ TL @=@ Timor-Leste +dnet:countries @=@ dnet:countries @=@ TG @=@ Togo +dnet:countries @=@ dnet:countries @=@ TK @=@ Tokelau +dnet:countries @=@ dnet:countries @=@ TO @=@ Tonga +dnet:countries @=@ dnet:countries @=@ TT @=@ Trinidad and Tobago +dnet:countries @=@ dnet:countries @=@ TN @=@ Tunisia +dnet:countries @=@ dnet:countries @=@ TR @=@ Turkey +dnet:countries @=@ dnet:countries @=@ TM @=@ Turkmenistan +dnet:countries @=@ dnet:countries @=@ TC @=@ Turks and Caicos Islands +dnet:countries @=@ dnet:countries @=@ TV @=@ Tuvalu +dnet:countries @=@ dnet:countries @=@ UNKNOWN @=@ UNKNOWN +dnet:countries @=@ dnet:countries @=@ UG @=@ Uganda +dnet:countries @=@ dnet:countries @=@ UA @=@ Ukraine +dnet:countries @=@ dnet:countries @=@ AE @=@ United Arab Emirates +dnet:countries @=@ dnet:countries @=@ GB @=@ United Kingdom +dnet:countries @=@ dnet:countries @=@ US @=@ United States +dnet:countries @=@ dnet:countries @=@ UM @=@ United States Minor Outlying Islands +dnet:countries @=@ dnet:countries @=@ UY @=@ Uruguay +dnet:countries @=@ dnet:countries @=@ UZ @=@ Uzbekistan +dnet:countries @=@ dnet:countries @=@ VU @=@ Vanuatu +dnet:countries @=@ dnet:countries @=@ VE @=@ Venezuela +dnet:countries @=@ dnet:countries @=@ VN @=@ Viet Nam +dnet:countries @=@ dnet:countries @=@ VG @=@ Virgin Islands (British) +dnet:countries @=@ dnet:countries @=@ VI @=@ Virgin Islands, U.S. +dnet:countries @=@ dnet:countries @=@ WF @=@ Wallis and Futuna +dnet:countries @=@ dnet:countries @=@ EH @=@ Western Sahara +dnet:countries @=@ dnet:countries @=@ YE @=@ Yemen +dnet:countries @=@ dnet:countries @=@ YU @=@ Yugoslavia +dnet:countries @=@ dnet:countries @=@ ZM @=@ Zambia +dnet:countries @=@ dnet:countries @=@ ZW @=@ Zimbabwe +dnet:countries @=@ dnet:countries @=@ AX @=@ Åland Islands +dnet:datasourceCompatibilityLevel @=@ dnet:datasourceCompatibilityLevel @=@ openaire2.0 @=@ OpenAIRE 2.0 (EC funding) +dnet:datasourceCompatibilityLevel @=@ dnet:datasourceCompatibilityLevel @=@ driver-openaire2.0 @=@ OpenAIRE 2.0+ (DRIVER OA, EC funding) +dnet:datasourceCompatibilityLevel @=@ dnet:datasourceCompatibilityLevel @=@ openaire3.0 @=@ OpenAIRE 3.0 (OA, funding) +dnet:datasourceCompatibilityLevel @=@ dnet:datasourceCompatibilityLevel @=@ openaire4.0 @=@ OpenAIRE 4.0 (inst.&thematic. repo.) +dnet:datasourceCompatibilityLevel @=@ dnet:datasourceCompatibilityLevel @=@ driver @=@ OpenAIRE Basic (DRIVER OA) +dnet:datasourceCompatibilityLevel @=@ dnet:datasourceCompatibilityLevel @=@ openaire2.0_data @=@ OpenAIRE Data (funded, referenced datasets) +dnet:datasourceCompatibilityLevel @=@ dnet:datasourceCompatibilityLevel @=@ hostedBy @=@ collected from a compatible aggregator +dnet:datasourceCompatibilityLevel @=@ dnet:datasourceCompatibilityLevel @=@ UNKNOWN @=@ not available +dnet:datasourceCompatibilityLevel @=@ dnet:datasourceCompatibilityLevel @=@ native @=@ proprietary +dnet:datasourceCompatibilityLevel @=@ dnet:datasourceCompatibilityLevel @=@ notCompatible @=@ under validation +dnet:datasourceCompatibilityLevel @=@ dnet:datasourceCompatibilityLevel @=@ openaire-cris_1.1 @=@ OpenAIRE CRIS v1.1 +fct:funding_relations @=@ fct:funding_relations @=@ fct:hasParentFunding @=@ fct:hasParentFunding +dnet:protocols @=@ dnet:protocols @=@ HTTPWithFileName @=@ HTTPWithFileName +dnet:protocols @=@ dnet:protocols @=@ NetCDF @=@ NetCDF +dnet:protocols @=@ dnet:protocols @=@ OpenDAP @=@ OpenDAP +dnet:protocols @=@ dnet:protocols @=@ schemaorg @=@ Schema.org +dnet:protocols @=@ dnet:protocols @=@ UNKNOWN @=@ UNKNOWN +dnet:protocols @=@ dnet:protocols @=@ api @=@ api +dnet:protocols @=@ dnet:protocols @=@ dataciteESPlugins @=@ dataciteESPlugins +dnet:protocols @=@ dnet:protocols @=@ datasetsbyjournal @=@ datasetsbyjournal +dnet:protocols @=@ dnet:protocols @=@ datasetsbyproject @=@ datasetsbyproject +dnet:protocols @=@ dnet:protocols @=@ excelFile @=@ excelFile +dnet:protocols @=@ dnet:protocols @=@ file @=@ file +dnet:protocols @=@ dnet:protocols @=@ fileGzip @=@ fileGzip +dnet:protocols @=@ dnet:protocols @=@ files_by_rpc @=@ files_by_rpc +dnet:protocols @=@ dnet:protocols @=@ files_from_mdstore @=@ files_from_mdstore +dnet:protocols @=@ dnet:protocols @=@ files_from_metadata @=@ files_from_metadata +dnet:protocols @=@ dnet:protocols @=@ filesystem @=@ filesystem +dnet:protocols @=@ dnet:protocols @=@ ftp @=@ ftp +dnet:protocols @=@ dnet:protocols @=@ gristProjects @=@ gristProjects +dnet:protocols @=@ dnet:protocols @=@ gtr2Projects @=@ gtr2Projects +dnet:protocols @=@ dnet:protocols @=@ http @=@ http +dnet:protocols @=@ dnet:protocols @=@ httpCSV @=@ httpCSV +dnet:protocols @=@ dnet:protocols @=@ httpList @=@ httpList +dnet:protocols @=@ dnet:protocols @=@ jdbc @=@ jdbc +dnet:protocols @=@ dnet:protocols @=@ oai @=@ oai +dnet:protocols @=@ dnet:protocols @=@ oai_sets @=@ oai_sets +dnet:protocols @=@ dnet:protocols @=@ other @=@ other +dnet:protocols @=@ dnet:protocols @=@ re3data @=@ re3data +dnet:protocols @=@ dnet:protocols @=@ rest @=@ rest +dnet:protocols @=@ dnet:protocols @=@ rest_json2xml @=@ rest_json2xml +dnet:protocols @=@ dnet:protocols @=@ sftp @=@ sftp +dnet:protocols @=@ dnet:protocols @=@ soap @=@ soap +dnet:protocols @=@ dnet:protocols @=@ sparql @=@ sparql +dnet:protocols @=@ dnet:protocols @=@ sword @=@ sword +dnet:protocols @=@ dnet:protocols @=@ targz @=@ targz +dnet:protocols @=@ dnet:protocols @=@ remoteMdstore @=@ remoteMdstore +wt:funding_typologies @=@ Wellcome Trust: Funding Typologies @=@ wt:fundingStream @=@ Wellcome Trust: Funding Stream +dnet:externalReference_typologies @=@ dnet:externalReference_typologies @=@ accessionNumber @=@ accessionNumber +dnet:externalReference_typologies @=@ dnet:externalReference_typologies @=@ dataset @=@ dataset +dnet:externalReference_typologies @=@ dnet:externalReference_typologies @=@ software @=@ software +datacite:id_typologies @=@ datacite:id_typologies @=@ ARK @=@ ARK +datacite:id_typologies @=@ datacite:id_typologies @=@ DOI @=@ DOI +datacite:id_typologies @=@ datacite:id_typologies @=@ EAN13 @=@ EAN13 +datacite:id_typologies @=@ datacite:id_typologies @=@ EISSN @=@ EISSN +datacite:id_typologies @=@ datacite:id_typologies @=@ Handle @=@ Handle +datacite:id_typologies @=@ datacite:id_typologies @=@ ISBN @=@ ISBN +datacite:id_typologies @=@ datacite:id_typologies @=@ ISSN @=@ ISSN +datacite:id_typologies @=@ datacite:id_typologies @=@ ISTC @=@ ISTC +datacite:id_typologies @=@ datacite:id_typologies @=@ LISSN @=@ LISSN +datacite:id_typologies @=@ datacite:id_typologies @=@ LSID @=@ LSID +datacite:id_typologies @=@ datacite:id_typologies @=@ PURL @=@ PURL +datacite:id_typologies @=@ datacite:id_typologies @=@ UNKNOWN @=@ UNKNOWN +datacite:id_typologies @=@ datacite:id_typologies @=@ UPC @=@ UPC +datacite:id_typologies @=@ datacite:id_typologies @=@ URL @=@ URL +datacite:id_typologies @=@ datacite:id_typologies @=@ URN @=@ URN +dnet:pid_types @=@ dnet:pid_types @=@ actrn @=@ ACTRN Identifier +dnet:pid_types @=@ dnet:pid_types @=@ nct @=@ ClinicalTrials.gov Identifier +dnet:pid_types @=@ dnet:pid_types @=@ euctr @=@ EU Clinical Trials Register +dnet:pid_types @=@ dnet:pid_types @=@ epo_id @=@ European Patent Office application ID +dnet:pid_types @=@ dnet:pid_types @=@ gsk @=@ GSK Identifier +dnet:pid_types @=@ dnet:pid_types @=@ GeoPass @=@ Geographic Location-Password Scheme +dnet:pid_types @=@ dnet:pid_types @=@ GBIF @=@ Global Biodiversity Information Facility +dnet:pid_types @=@ dnet:pid_types @=@ isrctn @=@ ISRCTN Identifier +dnet:pid_types @=@ dnet:pid_types @=@ ISNI @=@ International Standard Name Identifier +dnet:pid_types @=@ dnet:pid_types @=@ jprn @=@ JPRN Identifier +dnet:pid_types @=@ dnet:pid_types @=@ mag_id @=@ Microsoft Academic Graph Identifier +dnet:pid_types @=@ dnet:pid_types @=@ oai @=@ Open Archives Initiative +dnet:pid_types @=@ dnet:pid_types @=@ orcid @=@ Open Researcher and Contributor ID +dnet:pid_types @=@ dnet:pid_types @=@ PANGAEA @=@ PANGAEA +dnet:pid_types @=@ dnet:pid_types @=@ epo_nr_epodoc @=@ Patent application number in EPODOC format +dnet:pid_types @=@ dnet:pid_types @=@ UNKNOWN @=@ UNKNOWN +dnet:pid_types @=@ dnet:pid_types @=@ VIAF @=@ Virtual International Authority File +dnet:pid_types @=@ dnet:pid_types @=@ arXiv @=@ arXiv +dnet:pid_types @=@ dnet:pid_types @=@ doi @=@ doi +dnet:pid_types @=@ dnet:pid_types @=@ grid @=@ grid +dnet:pid_types @=@ dnet:pid_types @=@ info:eu-repo/dai @=@ info:eu-repo/dai +dnet:pid_types @=@ dnet:pid_types @=@ orcidworkid @=@ orcid workid +dnet:pid_types @=@ dnet:pid_types @=@ pmc @=@ pmc +dnet:pid_types @=@ dnet:pid_types @=@ pmid @=@ pmid +dnet:pid_types @=@ dnet:pid_types @=@ urn @=@ urn +dnet:pid_types @=@ dnet:pid_types @=@ who @=@ WHO Identifier +dnet:pid_types @=@ dnet:pid_types @=@ drks @=@ DRKS Identifier +dnet:pid_types @=@ dnet:pid_types @=@ handle @=@ Handle +dnet:topic_types @=@ dnet:topic_types @=@ ENRICH/MISSING/SUBJECT/ACM @=@ An ACM classification term that can be associated to your publications +dnet:topic_types @=@ dnet:topic_types @=@ ENRICH/MISSING/SUBJECT/ARXIV @=@ An ARXIV classification term that can be associated to your publications +dnet:topic_types @=@ dnet:topic_types @=@ ENRICH/MISSING/SUBJECT/DDC @=@ A Dewey Decimal classification term (DDC) that can be associated to your publications +dnet:topic_types @=@ dnet:topic_types @=@ ENRICH/MISSING/SUBJECT/JEL @=@ A Journal of Economic Literature (JEL) classification term that can be associated to your publications +dnet:topic_types @=@ dnet:topic_types @=@ ENRICH/MISSING/OPENACCESS_VERSION @=@ An Open Access versions of your publications +dnet:topic_types @=@ dnet:topic_types @=@ ENRICH/MISSING/DATASET/IS_REFERENCED_BY @=@ A dataset referenced by your records +dnet:topic_types @=@ dnet:topic_types @=@ ENRICH/MISSING/DATASET/REFERENCES @=@ A dataset that refers to your records +dnet:topic_types @=@ dnet:topic_types @=@ ENRICH/MISSING/DATASET/IS_RELATED_TO @=@ A dataset related to your records +dnet:topic_types @=@ dnet:topic_types @=@ ENRICH/MISSING/DATASET/IS_SUPPLEMENTED_TO @=@ A dataset that supplements your records +dnet:topic_types @=@ dnet:topic_types @=@ ENRICH/MISSING/PUBLICATION/IS_RELATED_TO @=@ A publication related to your records +dnet:topic_types @=@ dnet:topic_types @=@ ENRICH/MISSING/PUBLICATION/REFERENCES @=@ A publication referenced by your records +dnet:topic_types @=@ dnet:topic_types @=@ ENRICH/MISSING/PUBLICATION/IS_REFERENCED_BY @=@ A publication that refers to your records +dnet:topic_types @=@ dnet:topic_types @=@ ENRICH/MISSING/PUBLICATION/IS_SUPPLEMENTED_BY @=@ A publication that is supplemented by your records +dnet:topic_types @=@ dnet:topic_types @=@ ENRICH/MISSING/PUBLICATION/IS_SUPPLEMENTED_TO @=@ A publication that supplements your records +dnet:topic_types @=@ dnet:topic_types @=@ ENRICH/MISSING/SOFTWARE @=@ A software referred by your records +dnet:topic_types @=@ dnet:topic_types @=@ ENRICH/MORE/OPENACCESS_VERSION @=@ Another Open Access version of a publication +dnet:topic_types @=@ dnet:topic_types @=@ ENRICH/MORE/PID @=@ Another persistent identifier associated to your publications +dnet:topic_types @=@ dnet:topic_types @=@ ENRICH/MISSING/SUBJECT/MESHEUROPMC @=@ A classification term from the Medical Subject Headings (MeSH) that can be associated to your publications +dnet:topic_types @=@ dnet:topic_types @=@ ENRICH/MISSING/ABSTRACT @=@ An abstract describing among your publications +dnet:topic_types @=@ dnet:topic_types @=@ ENRICH/MISSING/PUBLICATION_DATE @=@ A date of publication missing in your content +dnet:topic_types @=@ dnet:topic_types @=@ ENRICH/MISSING/PID @=@ A persistent identifier associated to your publications +dnet:topic_types @=@ dnet:topic_types @=@ ENRICH/MORE/SUBJECT/ACM @=@ Another ACM classification term that can be associated to your publications +dnet:topic_types @=@ dnet:topic_types @=@ ENRICH/MORE/SUBJECT/ARXIV @=@ Another ARXIV classification term that can be associated to your publications +dnet:topic_types @=@ dnet:topic_types @=@ ENRICH/MORE/SUBJECT/DDC @=@ Another Dewey Decimal classification term (DDC) that can be associated to your publications +dnet:topic_types @=@ dnet:topic_types @=@ ENRICH/MORE/SUBJECT/JEL @=@ Another Journal of Economic Literature (JEL) classification term that can be associated to your publications +dnet:topic_types @=@ dnet:topic_types @=@ ENRICH/MORE/SUBJECT/MESHEUROPMC @=@ Another classification term from the Medical Subject Headings (MeSH) that can be associated to your publications +dnet:topic_types @=@ dnet:topic_types @=@ ENRICH/MISSING/PROJECT @=@ A project reference that can be associated to your publications +dnet:topic_types @=@ dnet:topic_types @=@ ENRICH/MISSING/DATASET/IS_SUPPLEMENTED_BY @=@ A dataset that is supplemented by your records +dnet:topic_types @=@ dnet:topic_types @=@ ENRICH/MISSING/AUTHOR/ORCID @=@ An Open Researcher and Contributor ID (ORCID) that can be associated to an author of your publications +dnet:review_levels @=@ dnet:review_levels @=@ 0000 @=@ Unknown +dnet:review_levels @=@ dnet:review_levels @=@ 0002 @=@ nonPeerReviewed +dnet:review_levels @=@ dnet:review_levels @=@ 0001 @=@ peerReviewed \ No newline at end of file diff --git a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/MigrateDbEntitiesApplication.java b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/MigrateDbEntitiesApplication.java index db1a2ef57..7ff06e428 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/MigrateDbEntitiesApplication.java +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/MigrateDbEntitiesApplication.java @@ -55,9 +55,9 @@ import org.slf4j.LoggerFactory; import eu.dnetlib.dhp.application.ArgumentApplicationParser; import eu.dnetlib.dhp.common.DbClient; +import eu.dnetlib.dhp.common.vocabulary.VocabularyGroup; import eu.dnetlib.dhp.oa.graph.raw.common.AbstractMigrationApplication; import eu.dnetlib.dhp.oa.graph.raw.common.VerifyNsPrefixPredicate; -import eu.dnetlib.dhp.common.vocabulary.VocabularyGroup; import eu.dnetlib.dhp.schema.oaf.Context; import eu.dnetlib.dhp.schema.oaf.DataInfo; import eu.dnetlib.dhp.schema.oaf.Dataset; diff --git a/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/raw/GenerateEntitiesApplicationTest.java b/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/raw/GenerateEntitiesApplicationTest.java index 8293faac4..83303ae8e 100644 --- a/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/raw/GenerateEntitiesApplicationTest.java +++ b/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/raw/GenerateEntitiesApplicationTest.java @@ -15,8 +15,8 @@ import org.junit.jupiter.api.extension.ExtendWith; import org.mockito.Mock; import org.mockito.junit.jupiter.MockitoExtension; -import eu.dnetlib.dhp.oa.graph.clean.CleaningFunctionTest; import eu.dnetlib.dhp.common.vocabulary.VocabularyGroup; +import eu.dnetlib.dhp.oa.graph.clean.CleaningFunctionTest; import eu.dnetlib.dhp.schema.common.ModelConstants; import eu.dnetlib.dhp.schema.oaf.*; import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException; diff --git a/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/raw/MappersTest.java b/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/raw/MappersTest.java index 35a829c71..dbfa6290d 100644 --- a/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/raw/MappersTest.java +++ b/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/raw/MappersTest.java @@ -21,8 +21,8 @@ import org.mockito.junit.jupiter.MockitoExtension; import com.fasterxml.jackson.databind.ObjectMapper; -import eu.dnetlib.dhp.oa.graph.clean.CleaningFunctionTest; import eu.dnetlib.dhp.common.vocabulary.VocabularyGroup; +import eu.dnetlib.dhp.oa.graph.clean.CleaningFunctionTest; import eu.dnetlib.dhp.schema.common.ModelConstants; import eu.dnetlib.dhp.schema.oaf.Author; import eu.dnetlib.dhp.schema.oaf.Dataset;