Compare commits

...

46 Commits

Author SHA1 Message Date
Michele Artini 6e380993d6 added filter for typenorm field 2024-03-12 10:09:38 +01:00
Michele Artini f9416ab2aa minor 2024-03-12 09:20:37 +01:00
Michele Artini 3e56b88a5f base transformation rule test 2024-03-11 14:48:37 +01:00
Michele Artini dfb05ebedb unitTest 2024-03-11 11:44:04 +01:00
Michele Artini 341c3f798e xslt 2024-03-11 10:50:17 +01:00
Michele Artini 4145065481 mapping for ODF 2024-03-08 15:31:08 +01:00
Michele Artini 5e9102d404 xslt 2024-03-08 14:41:25 +01:00
Michele Artini dc9d642e66 xslt 2024-03-08 12:37:54 +01:00
Michele Artini 932173287a added a base vocabulary 2024-03-08 11:52:58 +01:00
Michele Artini 3b5163d8e3 fixed a xpath 2024-03-08 08:23:07 +01:00
Michele Artini 6648d710a3 xslt 2024-03-07 13:06:41 +01:00
Michele Artini a059747f16 xslt 2024-03-07 10:57:41 +01:00
Michele Artini 1e34585213 file renaming 2024-03-07 09:52:49 +01:00
Michele Artini 108478b778 xslt 2024-03-07 09:34:16 +01:00
Michele Artini 6500151c90 initial version of the BASE Transformation rule 2024-03-05 14:18:57 +01:00
Michele Artini af58cd726e initial version of the BASE Transformation rule 2024-03-05 11:48:36 +01:00
Michele Artini efbb6c37d6 updated sql for base 2024-03-05 09:37:04 +01:00
Michele Artini b206e9a30b sql to create the BASE datasource 2024-03-04 15:13:34 +01:00
Michele Artini db6f774394 selection of the new plugin 2024-03-04 10:43:40 +01:00
Michele Artini 9506d80ddc changed sql to select accepted datasources 2024-03-04 08:25:40 +01:00
Michele Artini c2b6841eb0 opendoar datasource filter 2024-03-01 15:32:56 +01:00
Michele Artini be7f327e88 opendoar datasource filter 2024-03-01 13:38:36 +01:00
Michele Artini 32f4d6f691 reports for types 2024-03-01 11:43:37 +01:00
Michele Artini 71204a8056 some fields in stats 2024-02-29 10:17:31 +01:00
Michele Artini 5ddbef3a5b new stats 2024-02-28 14:34:09 +01:00
Michele Artini 04dd31139b separator in sql query 2024-02-28 11:12:25 +01:00
Michele Artini 3d14bef381 OpenDoar reports 2024-02-28 10:51:13 +01:00
Michele Artini f8cf7ffbcb stats 2024-02-22 14:01:11 +01:00
Michele Artini d2b7541583 fixed a problem with Dataset model 2024-02-16 11:36:46 +01:00
Michele Artini 8ffdd9747d added id to BaseRecordInfo 2024-02-15 14:27:50 +01:00
Michele Artini da65728afe produce a parquet file 2024-02-15 14:04:17 +01:00
Michele Artini e254720377 fixed path reports 2024-02-15 08:52:28 +01:00
Michele Artini 8d85c1e97e used a parser STAX 2024-02-15 08:21:52 +01:00
Michele Artini b42e2b4d61 fixed log class 2024-02-14 15:52:31 +01:00
Michele Artini 773346f638 increased memory 2024-02-14 14:40:27 +01:00
Michele Artini 2e11197142 removed invalid deletion 2024-02-14 11:59:30 +01:00
Michele Artini ddd6a7ceb3 minor fixes 2024-02-14 11:39:37 +01:00
Michele Artini 963a2500be new reports in hadoop job 2024-02-14 10:37:39 +01:00
Michele Artini 4b1ecad4e2 prepared a job to analyze the BASE records 2024-02-13 13:48:26 +01:00
Michele Artini dd7350ecf2 fixed a problem with xpaths 2024-02-13 08:36:42 +01:00
Michele Artini 265bfd364d refactoing 2024-02-12 15:35:36 +01:00
Michele Artini 16766c514e refactoring 2024-02-12 12:19:57 +01:00
Michele Artini 5add433b74 partial refactoring 2024-02-09 14:33:04 +01:00
Michele Artini c974c75f83 partial refactoring 2024-02-09 12:36:20 +01:00
Michele Artini c6db6335b9 prepare filtering for base import 2024-02-06 15:10:29 +01:00
Michele Artini abcd81bba0 first implementation of the collection plugin for BASE 2024-02-05 15:19:41 +01:00
26 changed files with 2361 additions and 35 deletions

View File

@ -19,6 +19,7 @@ import org.slf4j.LoggerFactory;
import eu.dnetlib.dhp.aggregation.common.ReporterCallback;
import eu.dnetlib.dhp.aggregation.common.ReportingJob;
import eu.dnetlib.dhp.collection.plugin.CollectorPlugin;
import eu.dnetlib.dhp.collection.plugin.base.BaseCollectorPlugin;
import eu.dnetlib.dhp.collection.plugin.file.FileCollectorPlugin;
import eu.dnetlib.dhp.collection.plugin.file.FileGZipCollectorPlugin;
import eu.dnetlib.dhp.collection.plugin.mongodb.MDStoreCollectorPlugin;
@ -57,7 +58,7 @@ public class CollectorWorker extends ReportingJob {
public void collect() throws UnknownCollectorPluginException, CollectorException, IOException {
final String outputPath = mdStoreVersion.getHdfsPath() + SEQUENCE_FILE_NAME;
final String outputPath = this.mdStoreVersion.getHdfsPath() + SEQUENCE_FILE_NAME;
log.info("outputPath path is {}", outputPath);
final CollectorPlugin plugin = getCollectorPlugin();
@ -67,36 +68,36 @@ public class CollectorWorker extends ReportingJob {
try (SequenceFile.Writer writer = SequenceFile
.createWriter(
fileSystem.getConf(),
SequenceFile.Writer.file(new Path(outputPath)),
SequenceFile.Writer.keyClass(IntWritable.class),
SequenceFile.Writer.valueClass(Text.class),
this.fileSystem.getConf(), SequenceFile.Writer.file(new Path(outputPath)), SequenceFile.Writer
.keyClass(IntWritable.class),
SequenceFile.Writer
.valueClass(Text.class),
SequenceFile.Writer.compression(SequenceFile.CompressionType.BLOCK, new DeflateCodec()))) {
final IntWritable key = new IntWritable(counter.get());
final Text value = new Text();
plugin
.collect(api, report)
.forEach(
content -> {
key.set(counter.getAndIncrement());
value.set(content);
try {
writer.append(key, value);
} catch (Throwable e) {
throw new RuntimeException(e);
}
});
} catch (Throwable e) {
report.put(e.getClass().getName(), e.getMessage());
.collect(this.api, this.report)
.forEach(content -> {
key.set(counter.getAndIncrement());
value.set(content);
try {
writer.append(key, value);
} catch (final Throwable e) {
throw new RuntimeException(e);
}
});
} catch (final Throwable e) {
this.report.put(e.getClass().getName(), e.getMessage());
throw new CollectorException(e);
} finally {
shutdown();
report.ongoing(counter.longValue(), counter.longValue());
this.report.ongoing(counter.longValue(), counter.longValue());
}
}
private void scheduleReport(AtomicInteger counter) {
private void scheduleReport(final AtomicInteger counter) {
schedule(new ReporterCallback() {
@Override
public Long getCurrent() {
return counter.longValue();
@ -111,31 +112,33 @@ public class CollectorWorker extends ReportingJob {
private CollectorPlugin getCollectorPlugin() throws UnknownCollectorPluginException {
switch (CollectorPlugin.NAME.valueOf(api.getProtocol())) {
switch (CollectorPlugin.NAME.valueOf(this.api.getProtocol())) {
case oai:
return new OaiCollectorPlugin(clientParams);
return new OaiCollectorPlugin(this.clientParams);
case rest_json2xml:
return new RestCollectorPlugin(clientParams);
return new RestCollectorPlugin(this.clientParams);
case file:
return new FileCollectorPlugin(fileSystem);
return new FileCollectorPlugin(this.fileSystem);
case fileGzip:
return new FileGZipCollectorPlugin(fileSystem);
return new FileGZipCollectorPlugin(this.fileSystem);
case baseDump:
return new BaseCollectorPlugin(this.fileSystem);
case other:
final CollectorPlugin.NAME.OTHER_NAME plugin = Optional
.ofNullable(api.getParams().get("other_plugin_type"))
.ofNullable(this.api.getParams().get("other_plugin_type"))
.map(CollectorPlugin.NAME.OTHER_NAME::valueOf)
.orElseThrow(() -> new IllegalArgumentException("invalid other_plugin_type"));
switch (plugin) {
case mdstore_mongodb_dump:
return new MongoDbDumpCollectorPlugin(fileSystem);
return new MongoDbDumpCollectorPlugin(this.fileSystem);
case mdstore_mongodb:
return new MDStoreCollectorPlugin();
default:
throw new UnknownCollectorPluginException("plugin is not managed: " + plugin);
}
default:
throw new UnknownCollectorPluginException("protocol is not managed: " + api.getProtocol());
throw new UnknownCollectorPluginException("protocol is not managed: " + this.api.getProtocol());
}
}

View File

@ -10,7 +10,8 @@ import eu.dnetlib.dhp.common.collection.CollectorException;
public interface CollectorPlugin {
enum NAME {
oai, other, rest_json2xml, file, fileGzip;
oai, other, rest_json2xml, file, fileGzip, baseDump;
public enum OTHER_NAME {
mdstore_mongodb_dump, mdstore_mongodb

View File

@ -0,0 +1,379 @@
package eu.dnetlib.dhp.collection.plugin.base;
import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession;
import static org.apache.spark.sql.functions.col;
import static org.apache.spark.sql.functions.count;
import java.sql.SQLException;
import java.util.ArrayList;
import java.util.LinkedHashSet;
import java.util.List;
import java.util.Optional;
import java.util.Set;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.commons.io.IOUtils;
import org.apache.commons.lang3.ObjectUtils;
import org.apache.commons.lang3.StringUtils;
import org.apache.commons.lang3.math.NumberUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.LongWritable;
import org.apache.hadoop.io.SequenceFile;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.io.compress.DeflateCodec;
import org.apache.spark.SparkConf;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Encoders;
import org.apache.spark.sql.SaveMode;
import org.apache.spark.sql.SparkSession;
import org.dom4j.Document;
import org.dom4j.DocumentException;
import org.dom4j.DocumentHelper;
import org.dom4j.Element;
import org.dom4j.Node;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
import eu.dnetlib.dhp.common.DbClient;
import eu.dnetlib.dhp.common.aggregation.AggregatorReport;
import scala.Tuple2;
public class BaseAnalyzerJob {
private static final String BASE_DUMP = "BASE_DUMP";
private static final Logger log = LoggerFactory.getLogger(BaseAnalyzerJob.class);
public static void main(final String[] args) throws Exception {
final String jsonConfiguration = IOUtils
.toString(
BaseAnalyzerJob.class
.getResourceAsStream("/eu/dnetlib/dhp/collection/plugin/base/action_set_parameters.json"));
final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration);
parser.parseArgument(args);
final Boolean isSparkSessionManaged = Optional
.ofNullable(parser.get("isSparkSessionManaged"))
.map(Boolean::valueOf)
.orElse(Boolean.TRUE);
log.info("isSparkSessionManaged: {}", isSparkSessionManaged);
final String inputPath = parser.get("inputPath");
log.info("inputPath: {}", inputPath);
final String dataPath = parser.get("dataPath");
log.info("dataPath {}: ", dataPath);
final String outputPath = parser.get("outputPath");
log.info("outputPath {}: ", outputPath);
final String opendoarPath = parser.get("opendoarPath");
log.info("opendoarPath {}: ", opendoarPath);
final String typesReportPath = parser.get("typesReportPath");
log.info("typesReportPath {}: ", typesReportPath);
final int fromStep = Integer.parseInt(parser.get("fromStep"));
log.info("fromStep {}: ", fromStep);
final String dbUrl = parser.get("postgresUrl");
log.info("postgresUrl {}: ", dbUrl);
final String dbUser = parser.get("postgresUser");
log.info("postgresUser {}: ", dbUser);
final String dbPassword = parser.get("postgresPassword");
log.info("postgresPassword {}: ", dbPassword);
final SparkConf conf = new SparkConf();
runWithSparkSession(conf, isSparkSessionManaged, spark -> {
if (fromStep <= 0) {
log
.info(
"\n**************************************\n* EXECUTING STEP 0: LoadRecords\n**************************************");
loadRecords(inputPath, dataPath);
log
.info(
"\n**************************************\n* EXECUTING STEP 0: DONE\n**************************************");
}
if (fromStep <= 1) {
log
.info(
"\n**************************************\n* EXECUTING STEP 1: Base Report\n**************************************");
generateReport(spark, dataPath, outputPath);
log
.info(
"\n**************************************\n* EXECUTING STEP 1: DONE\n**************************************");
}
if (fromStep <= 2) {
log
.info(
"\n**************************************\n* EXECUTING STEP 2: OpenDOAR Report\n**************************************");
generateOpenDoarReport(spark, outputPath, opendoarPath, loadOpenDoarStats(dbUrl, dbUser, dbPassword));
log
.info(
"\n**************************************\n* EXECUTING STEP 2: DONE\n**************************************");
}
if (fromStep <= 3) {
log
.info(
"\n**************************************\n* EXECUTING STEP 3: Type Vocabulary Report\n**************************************");
generateVocTypeReport(spark, outputPath, typesReportPath);
log
.info(
"\n**************************************\n* EXECUTING STEP 3: DONE\n**************************************");
}
});
}
private static void generateVocTypeReport(final SparkSession spark,
final String reportPath,
final String typesReportPath) {
spark
.read()
.parquet(reportPath)
.as(Encoders.bean(BaseRecordInfo.class))
.flatMap(rec -> {
final List<Tuple2<String, String>> list = new ArrayList<>();
for (final String t1 : rec.getTypes()) {
if (t1.startsWith("TYPE_NORM:")) {
for (final String t2 : rec.getTypes()) {
if (t2.startsWith("TYPE:")) {
list
.add(
new Tuple2<>(StringUtils.substringAfter(t1, "TYPE_NORM:").trim(),
StringUtils.substringAfter(t2, "TYPE:").trim()));
}
}
}
}
return list.iterator();
}, Encoders.tuple(Encoders.STRING(), Encoders.STRING()))
.distinct()
.write()
.mode(SaveMode.Overwrite)
.format("parquet")
.save(typesReportPath);
}
private static void generateOpenDoarReport(final SparkSession spark,
final String reportPath,
final String opendoarPath,
final List<OpenDoarRepoStatus> repos) {
final Dataset<OpenDoarRepoStatus> fromDB = spark.createDataset(repos, Encoders.bean(OpenDoarRepoStatus.class));
final Dataset<OpenDoarRepoStatus> fromBASE = spark
.read()
.parquet(reportPath)
.selectExpr("explode(collections) as collection")
.where("isnotnull(collection.opendoarId) and character_length(collection.opendoarId)>0")
.selectExpr("concat('opendoar____::',collection.opendoarId) as id")
.groupBy(col("id"))
.agg(count(col("id")))
.map(row -> {
final OpenDoarRepoStatus repo = new OpenDoarRepoStatus();
repo.setId(row.getString(0));
repo.getAggregations().put(BASE_DUMP, row.getLong(1));
repo.setBaseCount(row.getLong(1));
repo.setOpenaireCount(0);
repo.setHighCompliance(false);
return repo;
}, Encoders.bean(OpenDoarRepoStatus.class));
fromDB
.joinWith(fromBASE, fromDB.col("id").equalTo(fromBASE.col("id")), "full_outer")
.map(t -> merge(t._1, t._2), Encoders.bean(OpenDoarRepoStatus.class))
.write()
.mode(SaveMode.Overwrite)
.format("parquet")
.save(opendoarPath);
}
private static OpenDoarRepoStatus merge(final OpenDoarRepoStatus r1, final OpenDoarRepoStatus r2) {
if (r1 == null) {
return r2;
}
if (r2 == null) {
return r1;
}
final OpenDoarRepoStatus r = new OpenDoarRepoStatus();
r.setId(ObjectUtils.firstNonNull(r1.getId(), r2.getId()));
r.setJurisdiction(ObjectUtils.firstNonNull(r1.getJurisdiction(), r2.getJurisdiction()));
r.getAggregations().putAll(r1.getAggregations());
r.getAggregations().putAll(r2.getAggregations());
r.setHighCompliance(r1.isHighCompliance() || r2.isHighCompliance());
r.setBaseCount(Math.max(r1.getBaseCount(), r2.getBaseCount()));
r.setOpenaireCount(Math.max(r1.getOpenaireCount(), r2.getOpenaireCount()));
return r;
}
private static List<OpenDoarRepoStatus> loadOpenDoarStats(final String dbUrl,
final String dbUser,
final String dbPassword) throws Exception {
final List<OpenDoarRepoStatus> repos = new ArrayList<>();
try (DbClient dbClient = new DbClient(dbUrl, dbUser, dbPassword)) {
final String sql = IOUtils
.toString(
BaseAnalyzerJob.class
.getResourceAsStream(
"/eu/dnetlib/dhp/collection/plugin/base/sql/opendoar-aggregation-status.sql"));
dbClient.processResults(sql, row -> {
try {
final OpenDoarRepoStatus repo = new OpenDoarRepoStatus();
repo.setId(row.getString("id"));
repo.setJurisdiction(row.getString("jurisdiction"));
repo.setBaseCount(0);
repo.setHighCompliance(false);
long sum = 0;
for (final String s : (String[]) row.getArray("aggregations").getArray()) {
final String api = StringUtils.substringBefore(s, "@@@");
final long count = NumberUtils.toLong(StringUtils.substringAfter(s, "@@@"), 0);
sum += count;
repo.getAggregations().put(api, count);
// This should recognize the HIGH Compliances: openaire*X.Y*
if (s.contains("compliance: openaire")) {
repo.setHighCompliance(true);
}
}
repo.setOpenaireCount(sum);
repos.add(repo);
log.info("# FOUND OPENDOAR (DB): " + repo.getId());
} catch (final SQLException e) {
log.error("Error in SQL", e);
throw new RuntimeException("Error in SQL", e);
}
});
}
return repos;
}
private static void loadRecords(final String inputPath, final String outputPath) throws Exception {
try (final FileSystem fs = FileSystem.get(new Configuration());
final AggregatorReport report = new AggregatorReport()) {
final AtomicLong recordsCounter = new AtomicLong(0);
final LongWritable key = new LongWritable();
final Text value = new Text();
try (final SequenceFile.Writer writer = SequenceFile
.createWriter(
fs.getConf(), SequenceFile.Writer.file(new Path(outputPath)), SequenceFile.Writer
.keyClass(LongWritable.class),
SequenceFile.Writer
.valueClass(Text.class),
SequenceFile.Writer.compression(SequenceFile.CompressionType.BLOCK, new DeflateCodec()))) {
final BaseCollectorIterator iteraror = new BaseCollectorIterator(fs, new Path(inputPath), report);
while (iteraror.hasNext()) {
final String record = iteraror.next();
final long i = recordsCounter.incrementAndGet();
if ((i % 10000) == 0) {
log.info("# Loaded records: " + i);
}
key.set(i);
value.set(record);
try {
writer.append(key, value);
} catch (final Throwable e1) {
throw new RuntimeException(e1);
}
}
log.info("# COMPLETED - Loaded records: " + recordsCounter.get());
}
}
}
private static void generateReport(final SparkSession spark,
final String inputPath,
final String targetPath) throws Exception {
final JavaRDD<BaseRecordInfo> rdd = JavaSparkContext
.fromSparkContext(spark.sparkContext())
.sequenceFile(inputPath, LongWritable.class, Text.class)
.map(s -> s._2.toString())
.map(BaseAnalyzerJob::extractInfo);
spark
.createDataset(rdd.rdd(), Encoders.bean(BaseRecordInfo.class))
.write()
.mode(SaveMode.Overwrite)
.format("parquet")
.save(targetPath);
}
protected static BaseRecordInfo extractInfo(final String s) {
try {
final Document record = DocumentHelper.parseText(s);
final BaseRecordInfo info = new BaseRecordInfo();
final Set<String> paths = new LinkedHashSet<>();
final Set<String> types = new LinkedHashSet<>();
final List<BaseCollectionInfo> colls = new ArrayList<>();
for (final Object o : record.selectNodes("//*|//@*")) {
paths.add(((Node) o).getPath());
if (o instanceof Element) {
final Element n = (Element) o;
final String nodeName = n.getName();
if ("collection".equals(nodeName)) {
final String collName = n.getText().trim();
if (StringUtils.isNotBlank(collName)) {
final BaseCollectionInfo coll = new BaseCollectionInfo();
coll.setId(collName);
coll.setOpendoarId(n.valueOf("@opendoar_id").trim());
coll.setRorId(n.valueOf("@ror_id").trim());
colls.add(coll);
}
} else if ("type".equals(nodeName)) {
types.add("TYPE: " + n.getText().trim());
} else if ("typenorm".equals(nodeName)) {
types.add("TYPE_NORM: " + n.getText().trim());
}
}
}
info.setId(record.valueOf("//*[local-name() = 'header']/*[local-name() = 'identifier']").trim());
info.getTypes().addAll(types);
info.getPaths().addAll(paths);
info.setCollections(colls);
return info;
} catch (final DocumentException e) {
throw new RuntimeException(e);
}
}
}

View File

@ -0,0 +1,38 @@
package eu.dnetlib.dhp.collection.plugin.base;
import java.io.Serializable;
public class BaseCollectionInfo implements Serializable {
private static final long serialVersionUID = 5766333937429419647L;
private String id;
private String opendoarId;
private String rorId;
public String getId() {
return this.id;
}
public void setId(final String id) {
this.id = id;
}
public String getOpendoarId() {
return this.opendoarId;
}
public void setOpendoarId(final String opendoarId) {
this.opendoarId = opendoarId;
}
public String getRorId() {
return this.rorId;
}
public void setRorId(final String rorId) {
this.rorId = rorId;
}
}

View File

@ -0,0 +1,171 @@
package eu.dnetlib.dhp.collection.plugin.base;
import java.io.BufferedInputStream;
import java.io.ByteArrayInputStream;
import java.io.InputStream;
import java.io.StringWriter;
import java.util.Iterator;
import java.util.concurrent.BlockingQueue;
import java.util.concurrent.LinkedBlockingQueue;
import javax.xml.stream.XMLEventReader;
import javax.xml.stream.XMLEventWriter;
import javax.xml.stream.XMLInputFactory;
import javax.xml.stream.XMLOutputFactory;
import javax.xml.stream.events.EndElement;
import javax.xml.stream.events.StartElement;
import javax.xml.stream.events.XMLEvent;
import org.apache.commons.compress.archivers.tar.TarArchiveEntry;
import org.apache.commons.compress.archivers.tar.TarArchiveInputStream;
import org.apache.commons.compress.compressors.CompressorInputStream;
import org.apache.commons.compress.compressors.CompressorStreamFactory;
import org.apache.commons.io.IOUtils;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import eu.dnetlib.dhp.common.aggregation.AggregatorReport;
public class BaseCollectorIterator implements Iterator<String> {
private String nextElement;
private final BlockingQueue<String> queue = new LinkedBlockingQueue<>(100);
private static final Logger log = LoggerFactory.getLogger(BaseCollectorIterator.class);
private static final String END_ELEM = "__END__";
public BaseCollectorIterator(final FileSystem fs, final Path filePath, final AggregatorReport report) {
new Thread(() -> importHadoopFile(fs, filePath, report)).start();
try {
this.nextElement = this.queue.take();
} catch (final InterruptedException e) {
throw new RuntimeException(e);
}
}
protected BaseCollectorIterator(final String resourcePath, final AggregatorReport report) {
new Thread(() -> importTestFile(resourcePath, report)).start();
try {
this.nextElement = this.queue.take();
} catch (final InterruptedException e) {
throw new RuntimeException(e);
}
}
@Override
public synchronized boolean hasNext() {
return (this.nextElement != null) & !END_ELEM.equals(this.nextElement);
}
@Override
public synchronized String next() {
try {
return END_ELEM.equals(this.nextElement) ? null : this.nextElement;
} finally {
try {
this.nextElement = this.queue.take();
} catch (final InterruptedException e) {
throw new RuntimeException(e);
}
}
}
private void importHadoopFile(final FileSystem fs, final Path filePath, final AggregatorReport report) {
log.info("I start to read the TAR stream");
try (InputStream origInputStream = fs.open(filePath);
final TarArchiveInputStream tarInputStream = new TarArchiveInputStream(origInputStream)) {
importTarStream(tarInputStream, report);
} catch (final Throwable e) {
throw new RuntimeException("Error processing BASE records", e);
}
}
private void importTestFile(final String resourcePath, final AggregatorReport report) {
try (final InputStream origInputStream = BaseCollectorIterator.class.getResourceAsStream(resourcePath);
final TarArchiveInputStream tarInputStream = new TarArchiveInputStream(origInputStream)) {
importTarStream(tarInputStream, report);
} catch (final Throwable e) {
throw new RuntimeException("Error processing BASE records", e);
}
}
private void importTarStream(final TarArchiveInputStream tarInputStream, final AggregatorReport report) {
long count = 0;
final XMLInputFactory xmlInputFactory = XMLInputFactory.newInstance();
final XMLOutputFactory xmlOutputFactory = XMLOutputFactory.newInstance();
try {
TarArchiveEntry entry;
while ((entry = (TarArchiveEntry) tarInputStream.getNextEntry()) != null) {
final String name = entry.getName();
if (!entry.isDirectory() && name.contains("ListRecords") && name.endsWith(".bz2")) {
log.info("Processing file (BZIP): " + name);
final byte[] bzipData = new byte[(int) entry.getSize()];
IOUtils.readFully(tarInputStream, bzipData);
try (InputStream bzipIs = new ByteArrayInputStream(bzipData);
final BufferedInputStream bzipBis = new BufferedInputStream(bzipIs);
final CompressorInputStream bzipInput = new CompressorStreamFactory()
.createCompressorInputStream(bzipBis)) {
final XMLEventReader reader = xmlInputFactory.createXMLEventReader(bzipInput);
XMLEventWriter eventWriter = null;
StringWriter xmlWriter = null;
while (reader.hasNext()) {
final XMLEvent nextEvent = reader.nextEvent();
if (nextEvent.isStartElement()) {
final StartElement startElement = nextEvent.asStartElement();
if ("record".equals(startElement.getName().getLocalPart())) {
xmlWriter = new StringWriter();
eventWriter = xmlOutputFactory.createXMLEventWriter(xmlWriter);
}
}
if (eventWriter != null) {
eventWriter.add(nextEvent);
}
if (nextEvent.isEndElement()) {
final EndElement endElement = nextEvent.asEndElement();
if ("record".equals(endElement.getName().getLocalPart())) {
eventWriter.flush();
eventWriter.close();
this.queue.put(xmlWriter.toString());
eventWriter = null;
xmlWriter = null;
count++;
}
}
}
}
}
}
this.queue.put(END_ELEM); // TO INDICATE THE END OF THE QUEUE
} catch (final Throwable e) {
log.error("Error processing BASE records", e);
report.put(e.getClass().getName(), e.getMessage());
throw new RuntimeException("Error processing BASE records", e);
} finally {
log.info("Total records (written in queue): " + count);
}
}
}

View File

@ -0,0 +1,144 @@
package eu.dnetlib.dhp.collection.plugin.base;
import java.io.IOException;
import java.sql.SQLException;
import java.util.HashSet;
import java.util.Iterator;
import java.util.Optional;
import java.util.Set;
import java.util.Spliterator;
import java.util.Spliterators;
import java.util.stream.Stream;
import java.util.stream.StreamSupport;
import org.apache.commons.io.IOUtils;
import org.apache.commons.lang3.StringUtils;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.dom4j.Document;
import org.dom4j.DocumentException;
import org.dom4j.DocumentHelper;
import org.dom4j.Node;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import eu.dnetlib.dhp.collection.ApiDescriptor;
import eu.dnetlib.dhp.collection.plugin.CollectorPlugin;
import eu.dnetlib.dhp.collection.plugin.file.AbstractSplittedRecordPlugin;
import eu.dnetlib.dhp.common.DbClient;
import eu.dnetlib.dhp.common.aggregation.AggregatorReport;
import eu.dnetlib.dhp.common.collection.CollectorException;
public class BaseCollectorPlugin implements CollectorPlugin {
private final FileSystem fs;
private static final Logger log = LoggerFactory.getLogger(AbstractSplittedRecordPlugin.class);
// MAPPING AND FILTERING ARE DEFINED HERE:
// https://docs.google.com/document/d/1Aj-ZAV11b44MCrAAUCPiS2TUlXb6PnJEu1utCMAcCOU/edit
public BaseCollectorPlugin(final FileSystem fs) {
this.fs = fs;
}
@Override
public Stream<String> collect(final ApiDescriptor api, final AggregatorReport report) throws CollectorException {
// get path to file
final Path filePath = Optional
.ofNullable(api.getBaseUrl())
.map(Path::new)
.orElseThrow(() -> new CollectorException("missing baseUrl"));
final String dbUrl = api.getParams().get("dbUrl");
final String dbUser = api.getParams().get("dbUser");
final String dbPassword = api.getParams().get("dbPassword");
final String acceptedNormTypesString = api.getParams().get("acceptedNormTypes");
log.info("baseUrl: {}", filePath);
log.info("dbUrl: {}", dbUrl);
log.info("dbUser: {}", dbUser);
log.info("dbPassword: {}", "***");
log.info("acceptedNormTypes: {}", acceptedNormTypesString);
try {
if (!this.fs.exists(filePath)) { throw new CollectorException("path does not exist: " + filePath); }
} catch (final Throwable e) {
throw new CollectorException(e);
}
final Set<String> acceptedOpendoarIds = findAcceptedOpendoarIds(dbUrl, dbUser, dbPassword);
final Set<String> acceptedNormTypes = new HashSet<>();
if (StringUtils.isNotBlank(acceptedNormTypesString)) {
for (final String s : StringUtils.split(acceptedNormTypesString, ",")) {
if (StringUtils.isNotBlank(s)) {
acceptedNormTypes.add(s.trim());
}
}
}
final Iterator<String> iterator = new BaseCollectorIterator(this.fs, filePath, report);
final Spliterator<String> spliterator = Spliterators.spliteratorUnknownSize(iterator, Spliterator.ORDERED);
return StreamSupport
.stream(spliterator, false)
.filter(doc -> filterXml(doc, acceptedOpendoarIds, acceptedNormTypes));
}
private Set<String> findAcceptedOpendoarIds(final String dbUrl, final String dbUser, final String dbPassword)
throws CollectorException {
final Set<String> accepted = new HashSet<>();
try (final DbClient dbClient = new DbClient(dbUrl, dbUser, dbPassword)) {
final String sql = IOUtils
.toString(BaseAnalyzerJob.class
.getResourceAsStream("/eu/dnetlib/dhp/collection/plugin/base/sql/opendoar-accepted.sql"));
dbClient.processResults(sql, row -> {
try {
final String dsId = row.getString("id");
log.info("Accepted Datasource: " + dsId);
accepted.add(dsId);
} catch (final SQLException e) {
log.error("Error in SQL", e);
throw new RuntimeException("Error in SQL", e);
}
});
} catch (final IOException e) {
log.error("Error accessong SQL", e);
throw new CollectorException("Error accessong SQL", e);
}
log.info("Accepted Datasources (TOTAL): " + accepted.size());
return accepted;
}
protected static boolean filterXml(final String xml,
final Set<String> acceptedOpendoarIds,
final Set<String> acceptedNormTypes) {
try {
final Document doc = DocumentHelper.parseText(xml);
final String id = doc.valueOf("//*[local-name()='collection']/@opendoar_id").trim();
if (StringUtils.isBlank(id) || !acceptedOpendoarIds.contains("opendoar____::" + id)) { return false; }
if (acceptedNormTypes.isEmpty()) { return true; }
for (final Object s : doc.selectNodes("//*[local-name()='typenorm']")) {
if (acceptedNormTypes.contains(((Node) s).getText().trim())) { return true; }
}
return false;
} catch (final DocumentException e) {
log.error("Error parsing document", e);
throw new RuntimeException("Error parsing document", e);
}
}
}

View File

@ -0,0 +1,49 @@
package eu.dnetlib.dhp.collection.plugin.base;
import java.io.Serializable;
import java.util.ArrayList;
import java.util.List;
public class BaseRecordInfo implements Serializable {
private static final long serialVersionUID = -8848232018350074593L;
private String id;
private List<BaseCollectionInfo> collections = new ArrayList<>();
private List<String> paths = new ArrayList<>();
private List<String> types = new ArrayList<>();
public String getId() {
return this.id;
}
public void setId(final String id) {
this.id = id;
}
public List<String> getPaths() {
return this.paths;
}
public void setPaths(final List<String> paths) {
this.paths = paths;
}
public List<String> getTypes() {
return this.types;
}
public void setTypes(final List<String> types) {
this.types = types;
}
public List<BaseCollectionInfo> getCollections() {
return this.collections;
}
public void setCollections(final List<BaseCollectionInfo> collections) {
this.collections = collections;
}
}

View File

@ -0,0 +1,71 @@
package eu.dnetlib.dhp.collection.plugin.base;
import java.io.Serializable;
import java.util.HashMap;
import java.util.Map;
public class OpenDoarRepoStatus implements Serializable {
private static final long serialVersionUID = 4832658700366871160L;
private String id;
private String jurisdiction;
private boolean highCompliance = false;
private long baseCount = 0;
private long openaireCount = 0;
private Map<String, Long> aggregations = new HashMap<>();
public String getId() {
return this.id;
}
public void setId(final String id) {
this.id = id;
}
public String getJurisdiction() {
return this.jurisdiction;
}
public void setJurisdiction(final String jurisdiction) {
this.jurisdiction = jurisdiction;
}
public Map<String, Long> getAggregations() {
return this.aggregations;
}
public void setAggregations(final Map<String, Long> aggregations) {
this.aggregations = aggregations;
}
public boolean isHighCompliance() {
return this.highCompliance;
}
public void setHighCompliance(final boolean highCompliance) {
this.highCompliance = highCompliance;
}
public long getOpenaireCount() {
return this.openaireCount;
}
public void setOpenaireCount(final long openaireCount) {
this.openaireCount = openaireCount;
}
public long getBaseCount() {
return this.baseCount;
}
public void setBaseCount(final long baseCount) {
this.baseCount = baseCount;
}
}

View File

@ -0,0 +1,56 @@
[
{
"paramName": "i",
"paramLongName": "inputPath",
"paramDescription": "the path of the BASE dump",
"paramRequired": true
},
{
"paramName": "d",
"paramLongName": "dataPath",
"paramDescription": "the path of the loaded records",
"paramRequired": true
},
{
"paramName": "o",
"paramLongName": "outputPath",
"paramDescription": "the path of the generated the report",
"paramRequired": true
},
{
"paramName": "od",
"paramLongName": "opendoarPath",
"paramDescription": "the path of the generated the OpenDOAR report",
"paramRequired": true
},
{
"paramName": "t",
"paramLongName": "typesReportPath",
"paramDescription": "the path of the generated the types report",
"paramRequired": true
},
{
"paramName": "f",
"paramLongName": "fromStep",
"paramDescription": "the initial step (numeric, 0 for ALL STEPS)",
"paramRequired": true
},
{
"paramName": "pgurl",
"paramLongName": "postgresUrl",
"paramDescription": "postgres url, example: jdbc:postgresql://localhost:5432/testdb",
"paramRequired": true
},
{
"paramName": "pguser",
"paramLongName": "postgresUser",
"paramDescription": "postgres user",
"paramRequired": false
},
{
"paramName": "pgpasswd",
"paramLongName": "postgresPassword",
"paramDescription": "postgres password",
"paramRequired": false
}
]

View File

@ -0,0 +1,58 @@
<configuration>
<property>
<name>jobTracker</name>
<value>yarnRM</value>
</property>
<property>
<name>nameNode</name>
<value>hdfs://nameservice1</value>
</property>
<property>
<name>oozie.use.system.libpath</name>
<value>true</value>
</property>
<property>
<name>oozie.action.sharelib.for.spark</name>
<value>spark2</value>
</property>
<property>
<name>hive_metastore_uris</name>
<value>thrift://iis-cdh5-test-m3.ocean.icm.edu.pl:9083</value>
</property>
<property>
<name>spark2YarnHistoryServerAddress</name>
<value>http://iis-cdh5-test-gw.ocean.icm.edu.pl:18089</value>
</property>
<property>
<name>spark2ExtraListeners</name>
<value>com.cloudera.spark.lineage.NavigatorAppListener</value>
</property>
<property>
<name>spark2SqlQueryExecutionListeners</name>
<value>com.cloudera.spark.lineage.NavigatorQueryListener</value>
</property>
<property>
<name>oozie.launcher.mapreduce.user.classpath.first</name>
<value>true</value>
</property>
<property>
<name>sparkExecutorNumber</name>
<value>4</value>
</property>
<property>
<name>spark2EventLogDir</name>
<value>/user/spark/spark2ApplicationHistory</value>
</property>
<property>
<name>sparkDriverMemory</name>
<value>15G</value>
</property>
<property>
<name>sparkExecutorMemory</name>
<value>10G</value>
</property>
<property>
<name>sparkExecutorCores</name>
<value>1</value>
</property>
</configuration>

View File

@ -0,0 +1,79 @@
<workflow-app name="Analyze_BASE_Records" xmlns="uri:oozie:workflow:0.5">
<parameters>
<property>
<name>baseInputPath</name>
<description>the path of the BASE dump</description>
</property>
<property>
<name>baseDataPath</name>
<description>the path where to store BASE records</description>
</property>
<property>
<name>baseReportsPath</name>
<description>path where to store the reports</description>
</property>
<property>
<name>baseOpenDoarReportsPath</name>
<description>path where to store the OpenDOAR reports</description>
</property>
<property>
<name>baseTypesReportPath</name>
<description>path of the generated the types report</description>
</property>
<property>
<name>postgresURL</name>
<description>the postgres URL to access to the database</description>
</property>
<property>
<name>postgresUser</name>
<description>the user postgres</description>
</property>
<property>
<name>postgresPassword</name>
<description>the password postgres</description>
</property>
<property>
<name>baseFromStep</name>
<description>the initial step (numeric, 0 for ALL STEPS)</description>
</property>
</parameters>
<start to="analyzeBaseRecords"/>
<kill name="Kill">
<message>Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}]</message>
</kill>
<action name="analyzeBaseRecords">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn</master>
<mode>cluster</mode>
<name>AnalyzeBaseRecords</name>
<class>eu.dnetlib.dhp.collection.plugin.base.BaseAnalyzerJob</class>
<jar>dhp-aggregation-${projectVersion}.jar</jar>
<spark-opts>
--executor-cores=${sparkExecutorCores}
--executor-memory=${sparkExecutorMemory}
--driver-memory=${sparkDriverMemory}
--conf spark.extraListeners=${spark2ExtraListeners}
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
--conf spark.sql.shuffle.partitions=3840
</spark-opts>
<arg>--inputPath</arg><arg>${baseInputPath}</arg>
<arg>--dataPath</arg><arg>${baseDataPath}</arg>
<arg>--outputPath</arg><arg>${baseReportsPath}</arg>
<arg>--opendoarPath</arg><arg>${baseOpenDoarReportsPath}</arg>
<arg>--typesReportPath</arg><arg>${baseTypesReportPath}</arg>
<arg>--postgresUrl</arg><arg>${postgresURL}</arg>
<arg>--postgresUser</arg><arg>${postgresUser}</arg>
<arg>--postgresPassword</arg><arg>${postgresPassword}</arg>
<arg>--fromStep</arg><arg>${baseFromStep}</arg>
</spark>
<ok to="End"/>
<error to="Kill"/>
</action>
<end name="End"/>
</workflow-app>

View File

@ -0,0 +1,102 @@
BEGIN;
INSERT INTO dsm_services(
_dnet_resource_identifier_,
id,
officialname,
englishname,
namespaceprefix,
websiteurl,
logourl,
platform,
contactemail,
collectedfrom,
provenanceaction,
_typology_to_remove_,
eosc_type,
eosc_datasource_type,
research_entity_types,
thematic
) VALUES (
'openaire____::base_search',
'openaire____::base_search',
'Bielefeld Academic Search Engine (BASE)',
'Bielefeld Academic Search Engine (BASE)',
'base_search_',
'https://www.base-search.net',
'https://www.base-search.net/about/download/logo_224x57_white.gif',
'BASE',
'openaire-helpdesk@uni-bielefeld.de',
'infrastruct_::openaire',
'user:insert',
'aggregator::pubsrepository::unknown',
'Data Source',
'Aggregator',
ARRAY['Research Products'],
false
);
INSERT INTO dsm_service_organization(
_dnet_resource_identifier_,
organization,
service
) VALUES (
'fairsharing_::org::214@@openaire____::base_search',
'fairsharing_::org::214',
'openaire____::base_search'
);
INSERT INTO dsm_api(
_dnet_resource_identifier_,
id,
service,
protocol,
baseurl,
metadata_identifier_path
) VALUES (
'api_________::openaire____::base_search::dump',
'api_________::openaire____::base_search::dump',
'openaire____::base_search',
'baseDump',
'/user/michele.artini/base-import/base_oaipmh_dump-current.tar',
'//*[local-name()=''header'']/*[local-name()=''identifier'']'
);
INSERT INTO dsm_apiparams(
_dnet_resource_identifier_,
api,
param,
value
) VALUES (
'api_________::openaire____::base_search::dump@@dbUrl',
'api_________::openaire____::base_search::dump',
'dbUrl',
'jdbc:postgresql://postgresql.services.openaire.eu:5432/dnet_openaireplus'
);
INSERT INTO dsm_apiparams(
_dnet_resource_identifier_,
api,
param,
value
) VALUES (
'api_________::openaire____::base_search::dump@@dbUser',
'api_________::openaire____::base_search::dump',
'dbUser',
'dnet'
);
INSERT INTO dsm_apiparams(
_dnet_resource_identifier_,
api,
param,
value
) VALUES (
'api_________::openaire____::base_search::dump@@dbPassword',
'api_________::openaire____::base_search::dump',
'dbPassword',
'***'
);
COMMIT;

View File

@ -0,0 +1,7 @@
select s.id as id
from dsm_services s
where collectedfrom = 'openaire____::opendoar'
and jurisdiction = 'Institutional'
and s.id not in (
select service from dsm_api where coalesce(compatibility_override, compatibility) like '%openaire%' or last_collection_total > 0
);

View File

@ -0,0 +1,11 @@
select
s.id as id,
s.jurisdiction as jurisdiction,
array_remove(array_agg(a.id || ' (compliance: ' || coalesce(a.compatibility_override, a.compatibility, 'UNKNOWN') || ')@@@' || coalesce(a.last_collection_total, 0)), NULL) as aggregations
from
dsm_services s
join dsm_api a on (s.id = a.service)
where
collectedfrom = 'openaire____::opendoar'
group by
s.id;

View File

@ -0,0 +1,180 @@
<RESOURCE_PROFILE>
<HEADER>
<RESOURCE_IDENTIFIER value="c67911d6-9988-4a3b-b965-7d39bdd4a31d_Vm9jYWJ1bGFyeURTUmVzb3VyY2VzL1ZvY2FidWxhcnlEU1Jlc291cmNlVHlwZQ==" />
<RESOURCE_TYPE value="VocabularyDSResourceType" />
<RESOURCE_KIND value="VocabularyDSResources" />
<RESOURCE_URI value="" />
<DATE_OF_CREATION value="2024-02-13T11:15:48+00:00" />
</HEADER>
<BODY>
<CONFIGURATION>
<VOCABULARY_NAME code="base:normalized_types">base:normalized_types</VOCABULARY_NAME>
<VOCABULARY_DESCRIPTION>base:normalized_types</VOCABULARY_DESCRIPTION>
<TERMS>
<TERM native_name="Text" code="Text" english_name="Text" encoding="BASE">
<SYNONYMS>
<SYNONYM term="1" encoding="BASE" />
</SYNONYMS>
<RELATIONS />
</TERM>
<TERM native_name="Book" code="Book" english_name="Book" encoding="BASE">
<SYNONYMS>
<SYNONYM term="11" encoding="BASE" />
</SYNONYMS>
<RELATIONS />
</TERM>
<TERM native_name="Book part" code="Book part" english_name="Book part" encoding="BASE">
<SYNONYMS>
<SYNONYM term="111" encoding="BASE" />
</SYNONYMS>
<RELATIONS />
</TERM>
<TERM native_name="Journal/Newspaper" code="Journal/Newspaper" english_name="Journal/Newspaper" encoding="BASE">
<SYNONYMS>
<SYNONYM term="12" encoding="BASE" />
</SYNONYMS>
<RELATIONS />
</TERM>
<TERM native_name="Article contribution" code="Article contribution" english_name="Article contribution" encoding="BASE">
<SYNONYMS>
<SYNONYM term="121" encoding="BASE" />
</SYNONYMS>
<RELATIONS />
</TERM>
<TERM native_name="Other non-article" code="Other non-article" english_name="Other non-article" encoding="BASE">
<SYNONYMS>
<SYNONYM term="122" encoding="BASE" />
</SYNONYMS>
<RELATIONS />
</TERM>
<TERM native_name="Conference object" code="Conference object" english_name="Conference object" encoding="BASE">
<SYNONYMS>
<SYNONYM term="13" encoding="BASE" />
</SYNONYMS>
<RELATIONS />
</TERM>
<TERM native_name="Report" code="Report" english_name="Report" encoding="BASE">
<SYNONYMS>
<SYNONYM term="14" encoding="BASE" />
</SYNONYMS>
<RELATIONS />
</TERM>
<TERM native_name="Review" code="Review" english_name="Review" encoding="BASE">
<SYNONYMS>
<SYNONYM term="15" encoding="BASE" />
</SYNONYMS>
<RELATIONS />
</TERM>
<TERM native_name="Course material" code="Course material" english_name="Course material" encoding="BASE">
<SYNONYMS>
<SYNONYM term="16" encoding="BASE" />
</SYNONYMS>
<RELATIONS />
</TERM>
<TERM native_name="Lecture" code="Lecture" english_name="Lecture" encoding="BASE">
<SYNONYMS>
<SYNONYM term="17" encoding="BASE" />
</SYNONYMS>
<RELATIONS />
</TERM>
<TERM native_name="Thesis" code="Thesis" english_name="Thesis" encoding="BASE">
<SYNONYMS>
<SYNONYM term="18" encoding="BASE" />
</SYNONYMS>
<RELATIONS />
</TERM>
<TERM native_name="Bachelor's thesis" code="Bachelor's thesis" english_name="Bachelor's thesis" encoding="BASE">
<SYNONYMS>
<SYNONYM term="181" encoding="BASE" />
</SYNONYMS>
<RELATIONS />
</TERM>
<TERM native_name="Master's thesis" code="Master's thesis" english_name="Master's thesis" encoding="BASE">
<SYNONYMS>
<SYNONYM term="182" encoding="BASE" />
</SYNONYMS>
<RELATIONS />
</TERM>
<TERM native_name="Doctoral and postdoctoral thesis" code="Doctoral and postdoctoral thesis" english_name="Doctoral and postdoctoral thesis" encoding="BASE">
<SYNONYMS>
<SYNONYM term="183" encoding="BASE" />
</SYNONYMS>
<RELATIONS />
</TERM>
<TERM native_name="Manuscript" code="Manuscript" english_name="Manuscript" encoding="BASE">
<SYNONYMS>
<SYNONYM term="19" encoding="BASE" />
</SYNONYMS>
<RELATIONS />
</TERM>
<TERM native_name="Patent" code="Patent" english_name="Patent" encoding="BASE">
<SYNONYMS>
<SYNONYM term="1A" encoding="BASE" />
</SYNONYMS>
<RELATIONS />
</TERM>
<TERM native_name="Musical notation" code="Musical notation" english_name="Musical notation" encoding="BASE">
<SYNONYMS>
<SYNONYM term="2" encoding="BASE" />
</SYNONYMS>
<RELATIONS />
</TERM>
<TERM native_name="Map" code="Map" english_name="Map" encoding="BASE">
<SYNONYMS>
<SYNONYM term="3" encoding="BASE" />
</SYNONYMS>
<RELATIONS />
</TERM>
<TERM native_name="Audio" code="Audio" english_name="Audio" encoding="BASE">
<SYNONYMS>
<SYNONYM term="4" encoding="BASE" />
</SYNONYMS>
<RELATIONS />
</TERM>
<TERM native_name="Image/Video" code="Image/Video" english_name="Image/Video" encoding="BASE">
<SYNONYMS>
<SYNONYM term="5" encoding="BASE" />
</SYNONYMS>
<RELATIONS />
</TERM>
<TERM native_name="Still image" code="Still image" english_name="Still image" encoding="BASE">
<SYNONYMS>
<SYNONYM term="51" encoding="BASE" />
</SYNONYMS>
<RELATIONS />
</TERM>
<TERM native_name="Moving image/Video" code="Moving image/Video" english_name="Moving image/Video" encoding="BASE">
<SYNONYMS>
<SYNONYM term="52" encoding="BASE" />
</SYNONYMS>
<RELATIONS />
</TERM>
<TERM native_name="Software" code="Software" english_name="Software" encoding="BASE">
<SYNONYMS>
<SYNONYM term="6" encoding="BASE" />
</SYNONYMS>
<RELATIONS />
</TERM>
<TERM native_name="Dataset" code="Dataset" english_name="Dataset" encoding="BASE">
<SYNONYMS>
<SYNONYM term="7" encoding="BASE" />
</SYNONYMS>
<RELATIONS />
</TERM>
<TERM native_name="Unknown" code="Unknown" english_name="Unknown" encoding="BASE">
<SYNONYMS>
<SYNONYM term="F" encoding="BASE" />
</SYNONYMS>
<RELATIONS />
</TERM>
</TERMS>
</CONFIGURATION>
<STATUS>
<LAST_UPDATE value="2013-11-18T10:46:36Z" />
</STATUS>
<SECURITY_PARAMETERS>String</SECURITY_PARAMETERS>
</BODY>
</RESOURCE_PROFILE>

View File

@ -0,0 +1,298 @@
<RESOURCE_PROFILE>
<HEADER>
<RESOURCE_IDENTIFIER value="" />
<RESOURCE_TYPE value="TransformationRuleDSResourceType" />
<RESOURCE_KIND value="TransformationRuleDSResources" />
<RESOURCE_URI value="" />
<DATE_OF_CREATION value="2024-03-05T11:23:00+00:00" />
</HEADER>
<BODY>
<CONFIGURATION>
<SOURCE_METADATA_FORMAT interpretation="cleaned" layout="store" name="dc" />
<SINK_METADATA_FORMAT name="oaf_hbase" />
<IMPORTED />
<SCRIPT>
<TITLE>xslt_base2oaf_hadoop</TITLE>
<CODE>
<xsl:stylesheet xmlns:oaire="http://namespace.openaire.eu/schema/oaire/" xmlns:dateCleaner="http://eu/dnetlib/transform/dateISO"
xmlns:base_dc="http://oai.base-search.net/base_dc/"
xmlns:datacite="http://datacite.org/schema/kernel-4" xmlns:dr="http://www.driver-repository.eu/namespace/dr" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
xmlns:xsl="http://www.w3.org/1999/XSL/Transform" xmlns:vocabulary="http://eu/dnetlib/transform/clean" xmlns:oaf="http://namespace.openaire.eu/oaf"
xmlns:oai="http://www.openarchives.org/OAI/2.0/" xmlns:dri="http://www.driver-repository.eu/namespace/dri" xmlns:xs="http://www.w3.org/2001/XMLSchema" xmlns:dc="http://purl.org/dc/elements/1.1/"
exclude-result-prefixes="xsl vocabulary dateCleaner base_dc" version="2.0">
<xsl:param name="varOfficialName" />
<xsl:param name="varDataSourceId" />
<xsl:param name="varFP7" select="'corda_______::'" />
<xsl:param name="varH2020" select="'corda__h2020::'" />
<xsl:param name="repoCode" select="substring-before(//*[local-name() = 'header']/*[local-name()='recordIdentifier'], ':')" />
<xsl:param name="index" select="0" />
<xsl:param name="transDate" select="current-dateTime()" />
<xsl:template name="terminate">
<xsl:message terminate="yes">
record is not compliant, transformation is interrupted.
</xsl:message>
</xsl:template>
<xsl:template match="/">
<record>
<xsl:apply-templates select="//*[local-name() = 'header']" />
<!-- TO EVALUATE
base_dc:authod_id
base_dc:authod_id/base_dc:creator_id
base_dc:authod_id/base_dc:creator_name
example:
<dc:creator>ALBU, Svetlana</dc:creator>
<base_dc:authod_id>
<base_dc:creator_name>ALBU, Svetlana</base_dc:creator_name>
<base_dc:creator_id>https://orcid.org/0000-0002-8648-950X</base_dc:creator_id>
</base_dc:authod_id>
-->
<!-- NOT USED
base_dc:global_id (I used oai:identifier)
base_dc:collection/text()
base_dc:continent
base_dc:country
base_dc:year (I used dc:date)
dc:coverage
dc:language (I used base_dc:lang)
base_dc:link (I used dc:identifier)
-->
<xsl:variable name="varBaseNormType" select="vocabulary:clean(//base_dc:typenorm, 'base:normalized_types')" />
<metadata>
<xsl:call-template name="allElements">
<xsl:with-param name="sourceElement" select="//dc:title" />
<xsl:with-param name="targetElement" select="'dc:title'" />
</xsl:call-template>
<xsl:call-template name="allElements">
<xsl:with-param name="sourceElement" select="//dc:creator/replace(., '^(.*)\|.*$', '$1')" />
<xsl:with-param name="targetElement" select="'dc:creator'" />
</xsl:call-template>
<xsl:call-template name="allElements">
<xsl:with-param name="sourceElement" select="//dc:contributor" />
<xsl:with-param name="targetElement" select="'dc:contributor'" />
</xsl:call-template>
<xsl:call-template name="allElements">
<xsl:with-param name="sourceElement" select="//dc:description" />
<xsl:with-param name="targetElement" select="'dc:description'" />
</xsl:call-template>
<xsl:call-template name="allElements">
<xsl:with-param name="sourceElement" select="//dc:subject" />
<xsl:with-param name="targetElement" select="'dc:subject'" />
</xsl:call-template>
<!-- TODO: I'm not sure if this is the correct encoding -->
<xsl:for-each select="//base_dc:classcode|//base_dc:autoclasscode">
<dc:subject><xsl:value-of select="concat(@type, ':', .)" /></dc:subject>
</xsl:for-each>
<!-- END TODO -->
<xsl:call-template name="allElements">
<xsl:with-param name="sourceElement" select="//dc:publisher" />
<xsl:with-param name="targetElement" select="'dc:publisher'" />
</xsl:call-template>
<xsl:call-template name="allElements">
<xsl:with-param name="sourceElement" select="//dc:format" />
<xsl:with-param name="targetElement" select="'dc:format'" />
</xsl:call-template>
<dc:type>
<xsl:value-of select="$varBaseNormType" />
</dc:type>
<xsl:call-template name="allElements">
<xsl:with-param name="sourceElement" select="//dc:type" />
<xsl:with-param name="targetElement" select="'dc:type'" />
</xsl:call-template>
<xsl:call-template name="allElements">
<xsl:with-param name="sourceElement" select="//dc:source" />
<xsl:with-param name="targetElement" select="'dc:source'" />
</xsl:call-template>
<dc:language>
<xsl:value-of select="vocabulary:clean( //base_dc:lang, 'dnet:languages')" />
</dc:language>
<xsl:call-template name="allElements">
<xsl:with-param name="sourceElement" select="//dc:rights" />
<xsl:with-param name="targetElement" select="'dc:rights'" />
</xsl:call-template>
<xsl:call-template name="allElements">
<xsl:with-param name="sourceElement" select="//dc:relation" />
<xsl:with-param name="targetElement" select="'dc:relation'" />
</xsl:call-template>
<xsl:if test="not(//dc:identifier[starts-with(., 'http')])">
<xsl:call-template name="terminate" />
</xsl:if>
<xsl:call-template name="allElements">
<xsl:with-param name="sourceElement" select="//dc:identifier[starts-with(., 'http')]" />
<xsl:with-param name="targetElement" select="'dc:identifier'" />
</xsl:call-template>
<xsl:for-each select="//dc:relation">
<xsl:if test="matches(normalize-space(.), '(info:eu-repo/grantagreement/ec/fp7/)(\d\d\d\d\d\d)(.*)', 'i')">
<oaf:projectid>
<xsl:value-of select="concat($varFP7, replace(normalize-space(.), '(info:eu-repo/grantagreement/ec/fp7/)(\d\d\d\d\d\d)(.*)', '$2', 'i'))" />
</oaf:projectid>
</xsl:if>
<xsl:if test="matches(normalize-space(.), '(info:eu-repo/grantagreement/ec/h2020/)(\d\d\d\d\d\d)(.*)', 'i')">
<oaf:projectid>
<xsl:value-of select="concat($varH2020, replace(normalize-space(.), '(info:eu-repo/grantagreement/ec/h2020/)(\d\d\d\d\d\d)(.*)', '$2', 'i'))" />
</oaf:projectid>
</xsl:if>
</xsl:for-each>
<dr:CobjCategory>
<xsl:variable name="varCobjCategory" select="vocabulary:clean($varBaseNormType, 'dnet:publication_resource')" />
<xsl:variable name="varSuperType" select="vocabulary:clean($varCobjCategory, 'dnet:result_typologies')" />
<xsl:attribute name="type" select="$varSuperType" />
<xsl:value-of select="$varCobjCategory" />
</dr:CobjCategory>
<oaf:accessrights>
<xsl:choose>
<xsl:when test="//base_dc:oa[.='1']">OPEN</xsl:when>
<xsl:when test="//base_dc:rightsnorm">
<xsl:value-of select="vocabulary:clean(//base_dc:rightsnorm, 'dnet:access_modes')" />
</xsl:when>
<xsl:when test="//dc:rights">
<xsl:value-of select="vocabulary:clean( //dc:rights, 'dnet:access_modes')" />
</xsl:when>
<xsl:otherwise>UNKNOWN</xsl:otherwise>
</xsl:choose>
</oaf:accessrights>
<xsl:for-each select="//base_dc:doi">
<oaf:identifier identifierType="doi">
<xsl:value-of select="." />
</oaf:identifier>
</xsl:for-each>
<xsl:for-each select="distinct-values(//dc:identifier[starts-with(., 'http') and (not(contains(., '://dx.doi.org/') or contains(., '://doi.org/') or contains(., '://hdl.handle.net/')))])">
<oaf:identifier identifierType="url">
<xsl:value-of select="." />
</oaf:identifier>
</xsl:for-each>
<xsl:for-each select="distinct-values(//dc:identifier[starts-with(., 'http') and contains(., '://hdl.handle.net/')]/substring-after(., 'hdl.handle.net/'))">
<oaf:identifier identifierType="handle">
<xsl:value-of select="." />
</oaf:identifier>
</xsl:for-each>
<xsl:for-each select="distinct-values(//dc:identifier[starts-with(., 'urn:nbn:nl:') or starts-with(., 'URN:NBN:NL:')])">
<oaf:identifier identifierType='urn'>
<xsl:value-of select="." />
</oaf:identifier>
</xsl:for-each>
<oaf:identifier identifierType="oai-original">
<xsl:value-of
select="//*[local-name() = 'about']/*[local-name() = 'provenance']//*[local-name() = 'originDescription' and not(./*[local-name() = 'originDescription'])]/*[local-name() = 'identifier']" />
</oaf:identifier>
<oaf:hostedBy>
<xsl:attribute name="name">
<xsl:value-of select="//base_dc:collname" />
</xsl:attribute>
<xsl:attribute name="id">
<xsl:value-of select="concat('opendoar____::', //base_dc:collection/@opendoar_id)" />
</xsl:attribute>
</oaf:hostedBy>
<oaf:collectedFrom>
<xsl:attribute name="name">
<xsl:value-of select="$varOfficialName" />
</xsl:attribute>
<xsl:attribute name="id">
<xsl:value-of select="$varDataSourceId" />
</xsl:attribute>
</oaf:collectedFrom>
<oaf:dateAccepted>
<xsl:value-of select="dateCleaner:dateISO( //dc:date[1] )" />
</oaf:dateAccepted>
<xsl:if test="//base_dc:oa[.='1']">
<xsl:for-each select="//dc:relation[starts-with(., 'http')]">
<oaf:fulltext>
<xsl:value-of select="normalize-space(.)" />
</oaf:fulltext>
</xsl:for-each>
</xsl:if>
<xsl:for-each select="//base_dc:collection/@ror_id">
<oaf:relation relType="resultOrganization"
subRelType="affiliation"
relClass="hasAuthorInstitution"
targetType="organization">
<xsl:choose>
<xsl:when test="contains(.,'https://ror.org/')">
<xsl:value-of select="concat('ror_________::', normalize-space(.))" />
</xsl:when>
<xsl:otherwise>
<xsl:value-of select="concat('ror_________::https://ror.org/', normalize-space(.))" />
</xsl:otherwise>
</xsl:choose>
</oaf:relation>
</xsl:for-each>
</metadata>
<xsl:copy-of select="//*[local-name() = 'about']" />
</record>
</xsl:template>
<xsl:template name="allElements">
<xsl:param name="sourceElement" />
<xsl:param name="targetElement" />
<xsl:for-each select="$sourceElement">
<xsl:element name="{$targetElement}">
<xsl:value-of select="normalize-space(.)" />
</xsl:element>
</xsl:for-each>
</xsl:template>
<xsl:template match="//*[local-name() = 'header']">
<xsl:if test="//oai:header/@status='deleted'">
<xsl:call-template name="terminate" />
</xsl:if>
<xsl:copy>
<xsl:apply-templates select="node()|@*" />
<xsl:element name="dr:dateOfTransformation">
<xsl:value-of select="$transDate" />
</xsl:element>
</xsl:copy>
</xsl:template>
<xsl:template match="node()|@*">
<xsl:copy>
<xsl:apply-templates select="node()|@*" />
</xsl:copy>
</xsl:template>
</xsl:stylesheet>
</CODE>
</SCRIPT>
</CONFIGURATION>
<STATUS />
<SECURITY_PARAMETERS />
</BODY>
</RESOURCE_PROFILE>

View File

@ -0,0 +1,322 @@
<RESOURCE_PROFILE>
<HEADER>
<RESOURCE_IDENTIFIER value="2ad0cdd9-c96c-484c-8b0e-ed56d86891fe_VHJhbnNmb3JtYXRpb25SdWxlRFNSZXNvdXJjZXMvVHJhbnNmb3JtYXRpb25SdWxlRFNSZXNvdXJjZVR5cGU=" />
<RESOURCE_TYPE value="TransformationRuleDSResourceType" />
<RESOURCE_KIND value="TransformationRuleDSResources" />
<RESOURCE_URI value="" />
<DATE_OF_CREATION value="2024-03-05T11:23:00+00:00" />
</HEADER>
<BODY>
<CONFIGURATION>
<SOURCE_METADATA_FORMAT interpretation="cleaned" layout="store" name="dc" />
<SINK_METADATA_FORMAT name="odf_hbase" />
<IMPORTED />
<SCRIPT>
<TITLE>xslt_base2odf_hadoop</TITLE>
<CODE>
<xsl:stylesheet xmlns:oaire="http://namespace.openaire.eu/schema/oaire/" xmlns:dateCleaner="http://eu/dnetlib/transform/dateISO" xmlns:base_dc="http://oai.base-search.net/base_dc/"
xmlns:datacite="http://datacite.org/schema/kernel-4" xmlns:dr="http://www.driver-repository.eu/namespace/dr" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
xmlns:xsl="http://www.w3.org/1999/XSL/Transform" xmlns:vocabulary="http://eu/dnetlib/transform/clean" xmlns:oaf="http://namespace.openaire.eu/oaf"
xmlns:oai="http://www.openarchives.org/OAI/2.0/" xmlns:dri="http://www.driver-repository.eu/namespace/dri" xmlns:xs="http://www.w3.org/2001/XMLSchema" xmlns:dc="http://purl.org/dc/elements/1.1/"
exclude-result-prefixes="xsl vocabulary dateCleaner base_dc" version="2.0">
<xsl:param name="varOfficialName" />
<xsl:param name="varDataSourceId" />
<xsl:param name="varFP7" select="'corda_______::'" />
<xsl:param name="varH2020" select="'corda__h2020::'" />
<xsl:param name="repoCode" select="substring-before(//*[local-name() = 'header']/*[local-name()='recordIdentifier'], ':')" />
<xsl:param name="index" select="0" />
<xsl:param name="transDate" select="current-dateTime()" />
<xsl:template name="terminate">
<xsl:message terminate="yes">
record is not compliant, transformation is interrupted.
</xsl:message>
</xsl:template>
<xsl:template match="/">
<record>
<xsl:apply-templates select="//*[local-name() = 'header']" />
<!-- NOT USED
base_dc:global_id (I used oai:identifier)
base_dc:collection/text()
base_dc:continent
base_dc:country
dc:coverage
dc:source
dc:relation
dc:type (I used //base_dc:typenorm)
dc:language (I used base_dc:lang)
base_dc:link (I used dc:identifier)
-->
<xsl:variable name="varBaseNormType" select="vocabulary:clean(//base_dc:typenorm, 'base:normalized_types')" />
<metadata>
<datacite:resource>
<xsl:for-each select="//base_dc:doi">
<datacite:identifier identifierType="DOI">
<xsl:value-of select="." />
</datacite:identifier>
</xsl:for-each>
<datacite:alternateIdentifiers>
<xsl:for-each
select="distinct-values(//dc:identifier[starts-with(., 'http') and (not(contains(., '://dx.doi.org/') or contains(., '://doi.org/') or contains(., '://hdl.handle.net/')))])">
<datacite:identifier alternateIdentifierType="url">
<xsl:value-of select="." />
</datacite:identifier>
</xsl:for-each>
<xsl:for-each select="distinct-values(//dc:identifier[starts-with(., 'http') and contains(., '://hdl.handle.net/')]/substring-after(., 'hdl.handle.net/'))">
<datacite:identifier alternateIdentifierType="handle">
<xsl:value-of select="." />
</datacite:identifier>
</xsl:for-each>
<xsl:for-each select="distinct-values(//dc:identifier[starts-with(., 'urn:nbn:nl:') or starts-with(., 'URN:NBN:NL:')])">
<datacite:identifier alternateIdentifierType='urn'>
<xsl:value-of select="." />
</datacite:identifier>
</xsl:for-each>
<datacite:identifier alternateIdentifierType="oai-original">
<xsl:value-of
select="//*[local-name() = 'about']/*[local-name() = 'provenance']//*[local-name() = 'originDescription' and not(./*[local-name() = 'originDescription'])]/*[local-name() = 'identifier']" />
</datacite:identifier>
</datacite:alternateIdentifiers>
<datacite:relatedIdentifiers />
<datacite:resourceType><xsl:value-of select="$varBaseNormType" /></datacite:resourceType>
<datacite:titles>
<xsl:for-each select="//dc:title">
<datacite:title>
<xsl:value-of select="normalize-space(.)" />
</datacite:title>
</xsl:for-each>
</datacite:titles>
<datacite:creators>
<xsl:for-each select="//dc:creator">
<xsl:variable name="author" select="normalize-space(.)" />
<datacite:creator>
<datacite:creatorName>
<xsl:value-of select="$author" />
</datacite:creatorName>
<xsl:for-each select="//base_dc:authod_id[normalize-space(./base_dc:creator_name) = $author]/base_dc:creator_id ">
<xsl:if test="contains(.,'https://orcid.org/')">
<nameIdentifier schemeURI="https://orcid.org/" nameIdentifierScheme="ORCID">
<xsl:value-of select="substring-after(., 'https://orcid.org/')" />
</nameIdentifier>
</xsl:if>
</xsl:for-each>
</datacite:creator>
</xsl:for-each>
</datacite:creators>
<datacite:contributors>
<xsl:for-each select="//dc:contributor">
<datacite:contributor>
<datacite:contributorName>
<xsl:value-of select="normalize-space(.)" />
</datacite:contributorName>
</datacite:contributor>
</xsl:for-each>
</datacite:contributors>
<datacite:descriptions>
<xsl:for-each select="//dc:description">
<datacite:description descriptionType="Abstract">
<xsl:value-of select="normalize-space(.)" />
</datacite:description>
</xsl:for-each>
</datacite:descriptions>
<datacite:subjects>
<xsl:for-each select="//dc:subject">
<datacite:subject>
<xsl:value-of select="normalize-space(.)" />
</datacite:subject>
</xsl:for-each>
<xsl:for-each select="//base_dc:classcode|//base_dc:autoclasscode">
<datacite:subject subjectScheme="{@type}" classificationCode="{normalize-space(.)}">
<!-- TODO the value should be obtained by the Code -->
<xsl:value-of select="normalize-space(.)" />
</datacite:subject>
</xsl:for-each>
</datacite:subjects>
<datacite:publisher>
<xsl:value-of select="normalize-space(//dc:publisher)" />
</datacite:publisher>
<datacite:publicationYear>
<xsl:value-of select="normalize-space(//base_dc:year)" />
</datacite:publicationYear>
<datacite:formats>
<xsl:for-each select="//dc:format">
<datacite:format>
<xsl:value-of select="normalize-space(.)" />
</datacite:format>
</xsl:for-each>
</datacite:formats>
<datacite:language>
<xsl:value-of select="vocabulary:clean( //base_dc:lang, 'dnet:languages')" />
</datacite:language>
<oaf:accessrights>
<xsl:if test="//base_dc:oa[.='1']">
<datacite:rights rightsURI="http://purl.org/coar/access_right/c_abf2">open access</datacite:rights>
</xsl:if>
<xsl:for-each select="//dc:rights|//base_dc:rightsnorm">
<datacite:rights><xsl:value-of select="vocabulary:clean(., 'dnet:access_modes')" /></datacite:rights>
</xsl:for-each>
</oaf:accessrights>
</datacite:resource>
<xsl:for-each select="//dc:relation">
<xsl:if test="matches(normalize-space(.), '(info:eu-repo/grantagreement/ec/fp7/)(\d\d\d\d\d\d)(.*)', 'i')">
<oaf:projectid>
<xsl:value-of select="concat($varFP7, replace(normalize-space(.), '(info:eu-repo/grantagreement/ec/fp7/)(\d\d\d\d\d\d)(.*)', '$2', 'i'))" />
</oaf:projectid>
</xsl:if>
<xsl:if test="matches(normalize-space(.), '(info:eu-repo/grantagreement/ec/h2020/)(\d\d\d\d\d\d)(.*)', 'i')">
<oaf:projectid>
<xsl:value-of select="concat($varH2020, replace(normalize-space(.), '(info:eu-repo/grantagreement/ec/h2020/)(\d\d\d\d\d\d)(.*)', '$2', 'i'))" />
</oaf:projectid>
</xsl:if>
</xsl:for-each>
<dr:CobjCategory>
<xsl:variable name="varCobjCategory" select="vocabulary:clean($varBaseNormType, 'dnet:publication_resource')" />
<xsl:variable name="varSuperType" select="vocabulary:clean($varCobjCategory, 'dnet:result_typologies')" />
<xsl:attribute name="type" select="$varSuperType" />
<xsl:value-of select="$varCobjCategory" />
</dr:CobjCategory>
<oaf:accessrights>
<xsl:choose>
<xsl:when test="//base_dc:oa[.='1']">OPEN</xsl:when>
<xsl:when test="//base_dc:rightsnorm">
<xsl:value-of select="vocabulary:clean(//base_dc:rightsnorm, 'dnet:access_modes')" />
</xsl:when>
<xsl:when test="//dc:rights">
<xsl:value-of select="vocabulary:clean( //dc:rights, 'dnet:access_modes')" />
</xsl:when>
<xsl:otherwise>UNKNOWN</xsl:otherwise>
</xsl:choose>
</oaf:accessrights>
<xsl:for-each select="//base_dc:doi">
<oaf:identifier identifierType="doi">
<xsl:value-of select="." />
</oaf:identifier>
</xsl:for-each>
<xsl:for-each
select="distinct-values(//dc:identifier[starts-with(., 'http') and ( not(contains(., '://dx.doi.org/') or contains(., '://doi.org/') or contains(., '://hdl.handle.net/')))])">
<oaf:identifier identifierType="url">
<xsl:value-of select="." />
</oaf:identifier>
</xsl:for-each>
<xsl:for-each select="distinct-values(//dc:identifier[starts-with(., 'http') and contains(., '://hdl.handle.net/')]/substring-after(., 'hdl.handle.net/'))">
<oaf:identifier identifierType="handle">
<xsl:value-of select="." />
</oaf:identifier>
</xsl:for-each>
<xsl:for-each select="distinct-values(//dc:identifier[starts-with(., 'urn:nbn:nl:') or starts-with(., 'URN:NBN:NL:')])">
<oaf:identifier identifierType='urn'>
<xsl:value-of select="." />
</oaf:identifier>
</xsl:for-each>
<oaf:identifier identifierType="oai-original">
<xsl:value-of
select="//*[local-name() = 'about']/*[local-name() = 'provenance']//*[local-name() = 'originDescription' and not(./*[local-name() = 'originDescription'])]/*[local-name() = 'identifier']" />
</oaf:identifier>
<oaf:hostedBy>
<xsl:attribute name="name">
<xsl:value-of select="//base_dc:collname" />
</xsl:attribute>
<xsl:attribute name="id">
<xsl:value-of select="concat('opendoar____::', //base_dc:collection/@opendoar_id)" />
</xsl:attribute>
</oaf:hostedBy>
<oaf:collectedFrom>
<xsl:attribute name="name">
<xsl:value-of select="$varOfficialName" />
</xsl:attribute>
<xsl:attribute name="id">
<xsl:value-of select="$varDataSourceId" />
</xsl:attribute>
</oaf:collectedFrom>
<oaf:dateAccepted>
<xsl:value-of select="dateCleaner:dateISO( //dc:date[1] )" />
</oaf:dateAccepted>
<xsl:if test="//base_dc:oa[.='1']">
<xsl:for-each select="//dc:relation[starts-with(., 'http')]">
<oaf:fulltext>
<xsl:value-of select="normalize-space(.)" />
</oaf:fulltext>
</xsl:for-each>
</xsl:if>
<xsl:for-each select="//base_dc:collection/@ror_id">
<oaf:relation relType="resultOrganization" subRelType="affiliation" relClass="hasAuthorInstitution" targetType="organization">
<xsl:choose>
<xsl:when test="contains(.,'https://ror.org/')">
<xsl:value-of select="concat('ror_________::', normalize-space(.))" />
</xsl:when>
<xsl:otherwise>
<xsl:value-of select="concat('ror_________::https://ror.org/', normalize-space(.))" />
</xsl:otherwise>
</xsl:choose>
</oaf:relation>
</xsl:for-each>
</metadata>
<xsl:copy-of select="//*[local-name() = 'about']" />
</record>
</xsl:template>
<xsl:template match="//*[local-name() = 'header']">
<xsl:if test="//oai:header/@status='deleted'">
<xsl:call-template name="terminate" />
</xsl:if>
<xsl:copy>
<xsl:apply-templates select="node()|@*" />
<xsl:element name="dr:dateOfTransformation">
<xsl:value-of select="$transDate" />
</xsl:element>
</xsl:copy>
</xsl:template>
<xsl:template match="node()|@*">
<xsl:copy>
<xsl:apply-templates select="node()|@*" />
</xsl:copy>
</xsl:template>
</xsl:stylesheet>
</CODE>
</SCRIPT>
</CONFIGURATION>
<STATUS />
<SECURITY_PARAMETERS />
</BODY>
</RESOURCE_PROFILE>

View File

@ -2,7 +2,7 @@ package eu.dnetlib.dhp.sx.bio
import eu.dnetlib.dhp.application.ArgumentApplicationParser
import eu.dnetlib.dhp.collection.CollectionUtils
import eu.dnetlib.dhp.common.Constants.{MDSTORE_DATA_PATH,MDSTORE_SIZE_PATH}
import eu.dnetlib.dhp.common.Constants.{MDSTORE_DATA_PATH, MDSTORE_SIZE_PATH}
import eu.dnetlib.dhp.schema.mdstore.MDStoreVersion
import eu.dnetlib.dhp.schema.oaf.Oaf
import eu.dnetlib.dhp.sx.bio.BioDBToOAF.ScholixResolved
@ -11,6 +11,7 @@ import org.apache.spark.SparkConf
import org.apache.spark.sql.{Encoder, Encoders, SparkSession}
import org.slf4j.{Logger, LoggerFactory}
import eu.dnetlib.dhp.utils.DHPUtils.{MAPPER, writeHdfsFile}
object SparkTransformBioDatabaseToOAF {
def main(args: Array[String]): Unit = {

View File

@ -0,0 +1,134 @@
package eu.dnetlib.dhp.collection.plugin.base;
import static org.junit.jupiter.api.Assertions.assertEquals;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
import java.util.Set;
import java.util.concurrent.atomic.AtomicInteger;
import org.apache.commons.io.IOUtils;
import org.apache.commons.lang3.StringUtils;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Encoders;
import org.apache.spark.sql.SparkSession;
import org.dom4j.Attribute;
import org.dom4j.Document;
import org.dom4j.DocumentHelper;
import org.dom4j.Element;
import org.dom4j.Node;
import org.junit.jupiter.api.Disabled;
import org.junit.jupiter.api.Test;
import com.fasterxml.jackson.databind.ObjectMapper;
import eu.dnetlib.dhp.common.aggregation.AggregatorReport;
@Disabled
public class BaseCollectorIteratorTest {
@Test
void testImportFile() throws Exception {
long count = 0;
final BaseCollectorIterator iterator = new BaseCollectorIterator("base-sample.tar", new AggregatorReport());
final Map<String, Map<String, String>> collections = new HashMap<>();
final Map<String, AtomicInteger> fields = new HashMap<>();
final Set<String> types = new HashSet<>();
while (iterator.hasNext()) {
final Document record = DocumentHelper.parseText(iterator.next());
count++;
if ((count % 1000) == 0) {
System.out.println("# Read records: " + count);
}
// System.out.println(record.asXML());
for (final Object o : record.selectNodes("//*|//@*")) {
final String path = ((Node) o).getPath();
if (fields.containsKey(path)) {
fields.get(path).incrementAndGet();
} else {
fields.put(path, new AtomicInteger(1));
}
if (o instanceof Element) {
final Element n = (Element) o;
if ("collection".equals(n.getName())) {
final String collName = n.getText().trim();
if (StringUtils.isNotBlank(collName) && !collections.containsKey(collName)) {
final Map<String, String> collAttrs = new HashMap<>();
for (final Object ao : n.attributes()) {
collAttrs.put(((Attribute) ao).getName(), ((Attribute) ao).getValue());
}
collections.put(collName, collAttrs);
}
} else if ("type".equals(n.getName())) {
types.add(n.getText().trim());
}
}
}
}
final ObjectMapper mapper = new ObjectMapper();
for (final Entry<String, Map<String, String>> e : collections.entrySet()) {
System.out.println(e.getKey() + ": " + mapper.writeValueAsString(e.getValue()));
}
for (final Entry<String, AtomicInteger> e : fields.entrySet()) {
System.out.println(e.getKey() + ": " + e.getValue().get());
}
System.out.println("TYPES: ");
for (final String s : types) {
System.out.println(s);
}
assertEquals(30000, count);
}
@Test
public void testParquet() throws Exception {
final String xml = IOUtils.toString(getClass().getResourceAsStream("record.xml"));
final SparkSession spark = SparkSession.builder().master("local[*]").getOrCreate();
final List<BaseRecordInfo> ls = new ArrayList<>();
for (int i = 0; i < 10; i++) {
ls.add(BaseAnalyzerJob.extractInfo(xml));
}
final JavaRDD<BaseRecordInfo> rdd = JavaSparkContext
.fromSparkContext(spark.sparkContext())
.parallelize(ls);
final Dataset<BaseRecordInfo> df = spark
.createDataset(rdd.rdd(), Encoders.bean(BaseRecordInfo.class));
df.printSchema();
df.show(false);
}
}

View File

@ -0,0 +1,31 @@
package eu.dnetlib.dhp.collection.plugin.base;
import static org.junit.jupiter.api.Assertions.assertFalse;
import static org.junit.jupiter.api.Assertions.assertTrue;
import java.util.Arrays;
import java.util.HashSet;
import java.util.Set;
import org.apache.commons.io.IOUtils;
import org.junit.jupiter.api.Test;
class BaseCollectorPluginTest {
@Test
void testFilterXml() throws Exception {
final String xml = IOUtils.toString(getClass().getResourceAsStream("record.xml"));
final Set<String> validIds = new HashSet<>(Arrays.asList("opendoar____::1234", "opendoar____::4567"));
final Set<String> validTypes = new HashSet<>(Arrays.asList("1", "121"));
final Set<String> validTypes2 = new HashSet<>(Arrays.asList("1", "11"));
assertTrue(BaseCollectorPlugin.filterXml(xml, validIds, validTypes));
assertTrue(BaseCollectorPlugin.filterXml(xml, validIds, new HashSet<>()));
assertFalse(BaseCollectorPlugin.filterXml(xml, new HashSet<>(), validTypes));
assertFalse(BaseCollectorPlugin.filterXml(xml, validIds, validTypes2));
}
}

View File

@ -0,0 +1,77 @@
package eu.dnetlib.dhp.collection.plugin.base;
import java.io.IOException;
import org.apache.commons.io.IOUtils;
import org.apache.spark.SparkConf;
import org.apache.spark.util.LongAccumulator;
import org.dom4j.io.SAXReader;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Disabled;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.extension.ExtendWith;
import org.mockito.junit.jupiter.MockitoExtension;
import eu.dnetlib.dhp.aggregation.AbstractVocabularyTest;
import eu.dnetlib.dhp.aggregation.common.AggregationCounter;
import eu.dnetlib.dhp.schema.mdstore.MetadataRecord;
import eu.dnetlib.dhp.schema.mdstore.Provenance;
import eu.dnetlib.dhp.transformation.xslt.XSLTTransformationFunction;
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException;
@Disabled
@ExtendWith(MockitoExtension.class)
public class BaseTransfomationTest extends AbstractVocabularyTest {
private SparkConf sparkConf;
@BeforeEach
public void setUp() throws IOException, ISLookUpException {
setUpVocabulary();
this.sparkConf = new SparkConf();
this.sparkConf.setMaster("local[*]");
this.sparkConf.set("spark.driver.host", "localhost");
this.sparkConf.set("spark.ui.enabled", "false");
}
@Test
void testBase2ODF() throws Exception {
final MetadataRecord mr = new MetadataRecord();
mr.setProvenance(new Provenance("DSID", "DSNAME", "PREFIX"));
mr.setBody(IOUtils.toString(getClass().getResourceAsStream("record.xml")));
final XSLTTransformationFunction tr = loadTransformationRule("xml/base2odf.transformationRule.xml");
final MetadataRecord result = tr.call(mr);
System.out.println(result.getBody());
}
@Test
void testBase2OAF() throws Exception {
final MetadataRecord mr = new MetadataRecord();
mr.setProvenance(new Provenance("DSID", "DSNAME", "PREFIX"));
mr.setBody(IOUtils.toString(getClass().getResourceAsStream("record.xml")));
final XSLTTransformationFunction tr = loadTransformationRule("xml/base2oaf.transformationRule.xml");
final MetadataRecord result = tr.call(mr);
System.out.println(result.getBody());
}
private XSLTTransformationFunction loadTransformationRule(final String path) throws Exception {
final String xslt = new SAXReader()
.read(this.getClass().getResourceAsStream(path))
.selectSingleNode("//CODE/*")
.asXML();
final LongAccumulator la = new LongAccumulator();
return new XSLTTransformationFunction(new AggregationCounter(la, la, la), xslt, 0, this.vocabularies);
}
}

View File

@ -0,0 +1,58 @@
<record>
<header xmlns="http://www.openarchives.org/OAI/2.0/">
<identifier>ftdoajarticles:oai:doaj.org/article:e2d5b5126b2d4e479933cc7f9a9ae0c1</identifier>
<datestamp>2022-12-31T11:48:55Z</datestamp>
</header>
<metadata xmlns="http://www.openarchives.org/OAI/2.0/" xmlns:base_dc="http://oai.base-search.net/base_dc/" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xmlns:dc="http://purl.org/dc/elements/1.1/">
<base_dc:dc xsi:schemaLocation="http://oai.base-search.net/base_dc/ http://oai.base-search.net/base_dc/base_dc.xsd">
<base_dc:global_id>ftdoajarticles:oai:doaj.org/article:e2d5b5126b2d4e479933cc7f9a9ae0c1</base_dc:global_id>
<base_dc:continent>cww</base_dc:continent>
<base_dc:country>org</base_dc:country>
<base_dc:collection opendoar_id="1234" ror_id="ror1234">ftdoajarticles</base_dc:collection>
<base_dc:collname>TEST REPO</base_dc:collname>
<dc:title>Assessment of cultural heritage: the legislative and methodological framework of Russian Federation</dc:title>
<dc:creator>ALBU, Svetlana</dc:creator>
<dc:creator>LEȘAN, Anna</dc:creator>
<dc:subject>architectural heritage</dc:subject>
<dc:subject>evaluation of architectural heritage</dc:subject>
<dc:subject>types of values</dc:subject>
<dc:subject>experience of russian federation</dc:subject>
<dc:subject>Social Sciences</dc:subject>
<dc:subject>H</dc:subject>
<dc:description>Architectural heritage is the real estate inheritance by population of a country becoming an extremely valuable and specific category, preserving and capitalizing on those assets requires considerable effort. The state does not have sufficient means to maintain and preserve cultural heritage, as a result it is included in the civil circuit. The transfer of property right or of some partial rights over the architectural patrimony is accompanied by the necessity to estimate the value of goods. In this article, the authors examine the experience of Russian Federation (one of the largest countries with a huge architectural heritage) on the legislative framework of architectural and methodological heritage of architectural heritage assessment. The particularities of cultural assets valuation compared to other categories of real estate are examined, as well as the methodological aspects (types of values, methods applied in valuation, approaches according to the purpose of valuation) regarding the valuation of real estate with architectural value in Russian Federation.</dc:description>
<dc:publisher>Technical University of Moldova</dc:publisher>
<dc:date>2020-09-01T00:00:00Z</dc:date>
<base_dc:year>2020</base_dc:year>
<dc:type>article</dc:type>
<base_dc:typenorm>121</base_dc:typenorm>
<dc:identifier>https://doi.org/10.5281/zenodo.3971988</dc:identifier>
<dc:identifier>https://doaj.org/article/e2d5b5126b2d4e479933cc7f9a9ae0c1</dc:identifier>
<base_dc:link>https://doi.org/10.5281/zenodo.3971988</base_dc:link>
<dc:source>Journal of Social Sciences, Vol 3, Iss 3, Pp 134-143 (2020)</dc:source>
<dc:language>EN</dc:language>
<dc:language>FR</dc:language>
<dc:language>RO</dc:language>
<dc:relation>http://ibn.idsi.md/sites/default/files/imag_file/JSS-3-2020_134-143.pdf</dc:relation>
<dc:relation>https://doaj.org/toc/2587-3490</dc:relation>
<dc:relation>https://doaj.org/toc/2587-3504</dc:relation>
<dc:relation>doi:10.5281/zenodo.3971988</dc:relation>
<dc:relation>2587-3490</dc:relation>
<dc:relation>2587-3504</dc:relation>
<dc:relation>https://doaj.org/article/e2d5b5126b2d4e479933cc7f9a9ae0c1</dc:relation>
<base_dc:autoclasscode type="ddc">720</base_dc:autoclasscode>
<base_dc:authod_id>
<base_dc:creator_name>ALBU, Svetlana</base_dc:creator_name>
<base_dc:creator_id>https://orcid.org/0000-0002-8648-950X</base_dc:creator_id>
</base_dc:authod_id>
<base_dc:authod_id>
<base_dc:creator_name>LEȘAN, Anna</base_dc:creator_name>
<base_dc:creator_id>https://orcid.org/0000-0003-3284-0525</base_dc:creator_id>
</base_dc:authod_id>
<base_dc:doi>https://doi.org/10.5281/zenodo.3971988</base_dc:doi>
<base_dc:oa>1</base_dc:oa>
<base_dc:lang>eng</base_dc:lang>
<base_dc:lang>fre</base_dc:lang>
<base_dc:lang>rum</base_dc:lang>
</base_dc:dc>
</metadata>
</record>

View File

@ -1496,4 +1496,30 @@ cnr:institutes @=@ __CDS131__ @=@ IBE - Istituto per la BioEconomia
cnr:institutes @=@ https://ror.org/0263zy895 @=@ CDS132
cnr:institutes @=@ https://ror.org/0263zy895 @=@ SCITEC - Istituto di Scienze e Tecnologie Chimiche \"Giulio Natta\"
cnr:institutes @=@ __CDS133__ @=@ CDS133
cnr:institutes @=@ __CDS133__ @=@ STEMS - Istituto di Scienze e Tecnologie per l'Energia e la Mobilità Sostenibili
cnr:institutes @=@ __CDS133__ @=@ STEMS - Istituto di Scienze e Tecnologie per l'Energia e la Mobilità Sostenibili
base:normalized_types @=@ Text @=@ 1
base:normalized_types @=@ Book @=@ 11
base:normalized_types @=@ Book part @=@ 111
base:normalized_types @=@ Journal/Newspaper @=@ 12
base:normalized_types @=@ Article contribution @=@ 121
base:normalized_types @=@ Other non-article @=@ 122
base:normalized_types @=@ Conference object @=@ 13
base:normalized_types @=@ Report @=@ 14
base:normalized_types @=@ Review @=@ 15
base:normalized_types @=@ Course material @=@ 16
base:normalized_types @=@ Lecture @=@ 17
base:normalized_types @=@ Thesis @=@ 18
base:normalized_types @=@ Bachelor's thesis @=@ 181
base:normalized_types @=@ Master's thesis @=@ 182
base:normalized_types @=@ Doctoral and postdoctoral thesis @=@ 183
base:normalized_types @=@ Manuscript @=@ 19
base:normalized_types @=@ Patent @=@ 1A
base:normalized_types @=@ Musical notation @=@ 2
base:normalized_types @=@ Map @=@ 3
base:normalized_types @=@ Audio @=@ 4
base:normalized_types @=@ Image/Video @=@ 5
base:normalized_types @=@ Still image @=@ 51
base:normalized_types @=@ Moving image/Video @=@ 52
base:normalized_types @=@ Software @=@ 6
base:normalized_types @=@ Dataset @=@ 7
base:normalized_types @=@ Unknown @=@ F

View File

@ -1210,4 +1210,29 @@ cnr:institutes @=@ cnr:institutes @=@ __CDS130__ @=@ __CDS130__
cnr:institutes @=@ cnr:institutes @=@ __CDS131__ @=@ __CDS131__
cnr:institutes @=@ cnr:institutes @=@ https://ror.org/0263zy895 @=@ https://ror.org/0263zy895
cnr:institutes @=@ cnr:institutes @=@ __CDS133__ @=@ __CDS133__
base:normalized_types @=@ base:normalized_types @=@ Text @=@ Text
base:normalized_types @=@ base:normalized_types @=@ Book @=@ Book
base:normalized_types @=@ base:normalized_types @=@ Book part @=@ Book part
base:normalized_types @=@ base:normalized_types @=@ Journal/Newspaper @=@ Journal/Newspaper
base:normalized_types @=@ base:normalized_types @=@ Article contribution @=@ Article contribution
base:normalized_types @=@ base:normalized_types @=@ Other non-article @=@ Other non-article
base:normalized_types @=@ base:normalized_types @=@ Conference object @=@ Conference object
base:normalized_types @=@ base:normalized_types @=@ Report @=@ Report
base:normalized_types @=@ base:normalized_types @=@ Review @=@ Review
base:normalized_types @=@ base:normalized_types @=@ Course material @=@ Course material
base:normalized_types @=@ base:normalized_types @=@ Lecture @=@ Lecture
base:normalized_types @=@ base:normalized_types @=@ Thesis @=@ Thesis
base:normalized_types @=@ base:normalized_types @=@ Bachelor's thesis @=@ Bachelor's thesis
base:normalized_types @=@ base:normalized_types @=@ Master's thesis @=@ Master's thesis
base:normalized_types @=@ base:normalized_types @=@ Doctoral and postdoctoral thesis @=@ Doctoral and postdoctoral thesis
base:normalized_types @=@ base:normalized_types @=@ Manuscript @=@ Manuscript
base:normalized_types @=@ base:normalized_types @=@ Patent @=@ Patent
base:normalized_types @=@ base:normalized_types @=@ Musical notation @=@ Musical notation
base:normalized_types @=@ base:normalized_types @=@ Map @=@ Map
base:normalized_types @=@ base:normalized_types @=@ Audio @=@ Audio
base:normalized_types @=@ base:normalized_types @=@ Image/Video @=@ Image/Video
base:normalized_types @=@ base:normalized_types @=@ Still image @=@ Still image
base:normalized_types @=@ base:normalized_types @=@ Moving image/Video @=@ Moving image/Video
base:normalized_types @=@ base:normalized_types @=@ Software @=@ Software
base:normalized_types @=@ base:normalized_types @=@ Dataset @=@ Dataset
base:normalized_types @=@ base:normalized_types @=@ Unknown @=@ Unknown

View File

@ -23,10 +23,15 @@ class CrossrefMappingTest {
val mapper = new ObjectMapper()
@Test
def testMissingAuthorParser():Unit = {
val json: String = Source.fromInputStream(getClass.getResourceAsStream("/eu/dnetlib/doiboost/crossref/s41567-022-01757-y.json")).mkString
def testMissingAuthorParser(): Unit = {
val json: String = Source
.fromInputStream(getClass.getResourceAsStream("/eu/dnetlib/doiboost/crossref/s41567-022-01757-y.json"))
.mkString
val result = Crossref2Oaf.convert(json)
result.filter(o => o.isInstanceOf[Publication]).map(p=> p.asInstanceOf[Publication]).foreach(p =>assertTrue(p.getAuthor.size()>0))
result
.filter(o => o.isInstanceOf[Publication])
.map(p => p.asInstanceOf[Publication])
.foreach(p => assertTrue(p.getAuthor.size() > 0))
}
@Test