Compare commits

...

7 Commits

16 changed files with 311 additions and 40 deletions

View File

@ -47,17 +47,14 @@ public class DispatchEntitiesSparkJob {
String outputPath = parser.get("outputPath"); String outputPath = parser.get("outputPath");
log.info("outputPath: {}", outputPath); log.info("outputPath: {}", outputPath);
boolean filterInvisible = Boolean.valueOf(parser.get("filterInvisible")); boolean filterInvisible = Boolean.parseBoolean(parser.get("filterInvisible"));
log.info("filterInvisible: {}", filterInvisible); log.info("filterInvisible: {}", filterInvisible);
SparkConf conf = new SparkConf(); SparkConf conf = new SparkConf();
runWithSparkSession( runWithSparkSession(
conf, conf,
isSparkSessionManaged, isSparkSessionManaged,
spark -> { spark -> dispatchEntities(spark, inputPath, outputPath, filterInvisible));
HdfsSupport.remove(outputPath, spark.sparkContext().hadoopConfiguration());
dispatchEntities(spark, inputPath, outputPath, filterInvisible);
});
} }
private static void dispatchEntities( private static void dispatchEntities(
@ -72,7 +69,9 @@ public class DispatchEntitiesSparkJob {
String entityType = entry.getKey(); String entityType = entry.getKey();
Class<?> clazz = entry.getValue(); Class<?> clazz = entry.getValue();
final String entityPath = outputPath + "/" + entityType;
if (!entityType.equalsIgnoreCase("relation")) { if (!entityType.equalsIgnoreCase("relation")) {
HdfsSupport.remove(entityPath, spark.sparkContext().hadoopConfiguration());
Dataset<Row> entityDF = spark Dataset<Row> entityDF = spark
.read() .read()
.schema(Encoders.bean(clazz).schema()) .schema(Encoders.bean(clazz).schema())
@ -91,7 +90,7 @@ public class DispatchEntitiesSparkJob {
.write() .write()
.mode(SaveMode.Overwrite) .mode(SaveMode.Overwrite)
.option("compression", "gzip") .option("compression", "gzip")
.json(outputPath + "/" + entityType); .json(entityPath);
} }
}); });
} }

View File

@ -7,7 +7,7 @@ import org.apache.spark.sql.catalyst.expressions.Literal
import org.apache.spark.sql.expressions._ import org.apache.spark.sql.expressions._
import org.apache.spark.sql.functions.{col, lit, udf} import org.apache.spark.sql.functions.{col, lit, udf}
import org.apache.spark.sql.types._ import org.apache.spark.sql.types._
import org.apache.spark.sql.{Column, Dataset, Row, functions} import org.apache.spark.sql.{Column, Dataset, Row, SaveMode, functions}
import java.util.function.Predicate import java.util.function.Predicate
import java.util.stream.Collectors import java.util.stream.Collectors
@ -91,10 +91,19 @@ case class SparkDeduper(conf: DedupConfig) extends Serializable {
val df_with_blocks = df_with_clustering_keys val df_with_blocks = df_with_clustering_keys
// filter out rows with position exceeding the maxqueuesize parameter // filter out rows with position exceeding the maxqueuesize parameter
.filter(col("position").leq(conf.getWf.getQueueMaxSize)) //.filter(col("position").leq(conf.getWf.getQueueMaxSize))
.groupBy("clustering", "key") .groupBy(col("clustering"), col("key"), functions.floor(col("position").divide(lit(conf.getWf.getQueueMaxSize))))
.agg(functions.collect_set(functions.struct(model.schema.fieldNames.map(col): _*)).as("block")) .agg(functions.collect_set(functions.struct(model.schema.fieldNames.map(col): _*)).as("block"))
.filter(functions.size(new Column("block")).gt(1)) .filter(functions.size(new Column("block")).gt(1))
.union(
df_with_clustering_keys
// create small blocks of records on "the border" of maxqueuesize: getSlidingWindowSize/2 elements before and after
.filter(col("position").mod(conf.getWf.getQueueMaxSize).lt(conf.getWf.getSlidingWindowSize/2) ||
col("position").mod(conf.getWf.getQueueMaxSize).gt(conf.getWf.getQueueMaxSize - (conf.getWf.getSlidingWindowSize/2)))
.groupBy(col("clustering"), col("key"), functions.floor((col("position") + lit(conf.getWf.getSlidingWindowSize/2)).divide(lit(conf.getWf.getQueueMaxSize))))
.agg(functions.collect_set(functions.struct(model.schema.fieldNames.map(col): _*)).as("block"))
.filter(functions.size(new Column("block")).gt(1))
)
df_with_blocks df_with_blocks
} }

View File

@ -1,6 +1,23 @@
package eu.dnetlib.pace.util; package eu.dnetlib.pace.util;
/*
* Diff Match and Patch
* Copyright 2018 The diff-match-patch Authors.
* https://github.com/google/diff-match-patch
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
/* /*
* Diff Match and Patch * Diff Match and Patch
* Copyright 2018 The diff-match-patch Authors. * Copyright 2018 The diff-match-patch Authors.

View File

@ -1,4 +1,9 @@
{ {
"ETHZ.UNIGENF": {
"openaire_id": "opendoar____::1400",
"datacite_name": "Uni Genf",
"official_name": "Archive ouverte UNIGE"
},
"GESIS.RKI": { "GESIS.RKI": {
"openaire_id": "re3data_____::r3d100010436", "openaire_id": "re3data_____::r3d100010436",
"datacite_name": "Forschungsdatenzentrum am Robert Koch Institut", "datacite_name": "Forschungsdatenzentrum am Robert Koch Institut",

View File

@ -93,18 +93,13 @@ public class SparkWhitelistSimRels extends AbstractSparkAction {
Dataset<Row> entities = spark Dataset<Row> entities = spark
.read() .read()
.textFile(DedupUtility.createEntityPath(graphBasePath, subEntity)) .textFile(DedupUtility.createEntityPath(graphBasePath, subEntity))
.repartition(numPartitions) .select(functions.get_json_object(new Column("value"), dedupConf.getWf().getIdPath()).as("id") )
.withColumn("id", functions.get_json_object(new Column("value"), dedupConf.getWf().getIdPath())); .distinct();
Dataset<Row> whiteListRels1 = whiteListRels
.join(entities, entities.col("id").equalTo(whiteListRels.col("from")), "inner")
.select("from", "to");
Dataset<Row> whiteListRels2 = whiteListRels1 Dataset<Relation> whiteListSimRels = whiteListRels
.join(entities, whiteListRels1.col("to").equalTo(entities.col("id")), "inner") .join(entities, entities.col("id").equalTo(whiteListRels.col("from")), "semi")
.select("from", "to"); .join(entities, functions.col("to").equalTo(entities.col("id")), "semi")
Dataset<Relation> whiteListSimRels = whiteListRels2
.map( .map(
(MapFunction<Row, Relation>) r -> DedupUtility (MapFunction<Row, Relation>) r -> DedupUtility
.createSimRel(r.getString(0), r.getString(1), entity), .createSimRel(r.getString(0), r.getString(1), entity),

View File

@ -8,8 +8,12 @@ import scala.collection.JavaConversions;
object GraphProcessor { object GraphProcessor {
def findCCs(vertexes: RDD[(VertexId, String)], edges: RDD[Edge[String]], maxIterations: Int, cut:Int): RDD[ConnectedComponent] = { def findCCs(vertexes: RDD[(VertexId, String)], edges: RDD[Edge[String]], maxIterations: Int, cut:Int): RDD[ConnectedComponent] = {
val graph: Graph[String, String] = Graph(vertexes, edges).partitionBy(PartitionStrategy.RandomVertexCut) //TODO remember to remove partitionby
val cc = graph.connectedComponents(maxIterations).vertices val (cc, didConverge, iterations) = com.kwartile.lib.cc.ConnectedComponent
.run(edges.map{ e => Seq(e.srcId, e.dstId).toList}, 50)
// val graph: Graph[String, String] = Graph(vertexes, edges).partitionBy(PartitionStrategy.RandomVertexCut) //TODO remember to remove partitionby
// val cc = graph.connectedComponents(maxIterations).vertices
val joinResult = vertexes.leftOuterJoin(cc).map { val joinResult = vertexes.leftOuterJoin(cc).map {
case (id, (openaireId, cc)) => { case (id, (openaireId, cc)) => {

View File

@ -127,6 +127,7 @@
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
--conf spark.sql.shuffle.partitions=15000 --conf spark.sql.shuffle.partitions=15000
--conf spark.network.timeout=600
</spark-opts> </spark-opts>
<arg>--graphBasePath</arg><arg>${graphBasePath}</arg> <arg>--graphBasePath</arg><arg>${graphBasePath}</arg>
<arg>--isLookUpUrl</arg><arg>${isLookUpUrl}</arg> <arg>--isLookUpUrl</arg><arg>${isLookUpUrl}</arg>
@ -154,6 +155,7 @@
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
--conf spark.sql.shuffle.partitions=15000 --conf spark.sql.shuffle.partitions=15000
--conf spark.network.timeout=600
</spark-opts> </spark-opts>
<arg>--graphBasePath</arg><arg>${graphBasePath}</arg> <arg>--graphBasePath</arg><arg>${graphBasePath}</arg>
<arg>--isLookUpUrl</arg><arg>${isLookUpUrl}</arg> <arg>--isLookUpUrl</arg><arg>${isLookUpUrl}</arg>
@ -182,6 +184,7 @@
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
--conf spark.sql.shuffle.partitions=15000 --conf spark.sql.shuffle.partitions=15000
--conf spark.network.timeout=600
</spark-opts> </spark-opts>
<arg>--graphBasePath</arg><arg>${graphBasePath}</arg> <arg>--graphBasePath</arg><arg>${graphBasePath}</arg>
<arg>--workingPath</arg><arg>${workingPath}</arg> <arg>--workingPath</arg><arg>${workingPath}</arg>
@ -209,6 +212,7 @@
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
--conf spark.sql.shuffle.partitions=15000 --conf spark.sql.shuffle.partitions=15000
--conf spark.network.timeout=600
</spark-opts> </spark-opts>
<arg>--graphBasePath</arg><arg>${graphBasePath}</arg> <arg>--graphBasePath</arg><arg>${graphBasePath}</arg>
<arg>--workingPath</arg><arg>${workingPath}</arg> <arg>--workingPath</arg><arg>${workingPath}</arg>
@ -236,6 +240,7 @@
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
--conf spark.sql.shuffle.partitions=15000 --conf spark.sql.shuffle.partitions=15000
--conf spark.network.timeout=600
</spark-opts> </spark-opts>
<arg>--graphBasePath</arg><arg>${graphBasePath}</arg> <arg>--graphBasePath</arg><arg>${graphBasePath}</arg>
<arg>--workingPath</arg><arg>${workingPath}</arg> <arg>--workingPath</arg><arg>${workingPath}</arg>
@ -263,6 +268,7 @@
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
--conf spark.sql.shuffle.partitions=3840 --conf spark.sql.shuffle.partitions=3840
--conf spark.network.timeout=600
</spark-opts> </spark-opts>
<arg>--graphBasePath</arg><arg>${graphBasePath}</arg> <arg>--graphBasePath</arg><arg>${graphBasePath}</arg>
<arg>--workingPath</arg><arg>${workingPath}</arg> <arg>--workingPath</arg><arg>${workingPath}</arg>
@ -289,6 +295,7 @@
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
--conf spark.sql.shuffle.partitions=10000 --conf spark.sql.shuffle.partitions=10000
--conf spark.network.timeout=600
</spark-opts> </spark-opts>
<arg>--graphBasePath</arg><arg>${graphBasePath}</arg> <arg>--graphBasePath</arg><arg>${graphBasePath}</arg>
<arg>--workingPath</arg><arg>${workingPath}</arg> <arg>--workingPath</arg><arg>${workingPath}</arg>
@ -315,6 +322,7 @@
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
--conf spark.sql.shuffle.partitions=10000 --conf spark.sql.shuffle.partitions=10000
--conf spark.network.timeout=600
</spark-opts> </spark-opts>
<arg>--graphBasePath</arg><arg>${graphBasePath}</arg> <arg>--graphBasePath</arg><arg>${graphBasePath}</arg>
<arg>--workingPath</arg><arg>${workingPath}</arg> <arg>--workingPath</arg><arg>${workingPath}</arg>

View File

@ -49,7 +49,7 @@ public class DownloadCsvTest {
@Test @Test
void getUnibiFileTest() throws CollectorException, IOException, ClassNotFoundException { void getUnibiFileTest() throws CollectorException, IOException, ClassNotFoundException {
String fileURL = "https://pub.uni-bielefeld.de/download/2944717/2944718/issn_gold_oa_version_4.csv"; String fileURL = "https://pub.uni-bielefeld.de/download/2944717/2944718/issn_gold_oa_version_5.csv";
final String outputFile = workingDir + "/unibi_gold.json"; final String outputFile = workingDir + "/unibi_gold.json";
new DownloadCSV() new DownloadCSV()

View File

@ -1067,6 +1067,28 @@ class MappersTest {
System.out.println("***************"); System.out.println("***************");
} }
@Test
public void testD4ScienceTraining() throws IOException {
final String xml = IOUtils
.toString(Objects.requireNonNull(getClass().getResourceAsStream("d4science-1-training.xml")));
final List<Oaf> list = new OdfToOafMapper(vocs, false, true).processMdRecord(xml);
final OtherResearchProduct trainingMaterial = (OtherResearchProduct) list.get(0);
System.out.println("***************");
System.out.println(new ObjectMapper().writeValueAsString(trainingMaterial));
System.out.println("***************");
}
@Test
public void testD4ScienceDataset() throws IOException {
final String xml = IOUtils
.toString(Objects.requireNonNull(getClass().getResourceAsStream("d4science-2-dataset.xml")));
final List<Oaf> list = new OdfToOafMapper(vocs, false, true).processMdRecord(xml);
final Dataset trainingMaterial = (Dataset) list.get(0);
System.out.println("***************");
System.out.println(new ObjectMapper().writeValueAsString(trainingMaterial));
System.out.println("***************");
}
@Test @Test
void testNotWellFormed() throws IOException { void testNotWellFormed() throws IOException {
final String xml = IOUtils final String xml = IOUtils

View File

@ -0,0 +1,93 @@
<?xml version="1.0" encoding="UTF-8"?>
<oai:record xmlns:dr="http://www.driver-repository.eu/namespace/dr"
xmlns:dri="http://www.driver-repository.eu/namespace/dri"
xmlns:oaf="http://namespace.openaire.eu/oaf" xmlns:oai="http://www.openarchives.org/OAI/2.0/">
<oai:header>
<dri:objIdentifier>alessia_____::104c2d4ba8878c16fa824dce5b1bea57</dri:objIdentifier>
<dri:recordIdentifier>12d8f77e-d66f-46f5-8d88-af7db23bc4c9</dri:recordIdentifier>
<dri:dateOfCollection>2023-09-08T10:12:35.864+02:00</dri:dateOfCollection>
<oaf:datasourceprefix>alessia_____</oaf:datasourceprefix>
<dr:dateOfTransformation>2023-09-08T11:31:45.692+02:00</dr:dateOfTransformation>
</oai:header>
<oai:metadata>
<datacite:resource
xmlns:datacite="http://datacite.org/schema/kernel-4"
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://datacite.org/schema/kernel-4 http://schema.datacite.org/meta/kernel-4/metadata.xsd">
<datacite:identifier identifierType="URL">http://data.d4science.org/ctlg/ResourceCatalogue/visual_analytics_for_data_scientists</datacite:identifier>
<datacite:alternateIdentifiers/>
<datacite:creators>
<datacite:creator>
<datacite:creatorName>BRAGHIERI MARCO</datacite:creatorName>
</datacite:creator>
</datacite:creators>
<datacite:titles>
<datacite:title>Visual Analytics for Data Scientists</datacite:title>
</datacite:titles>
<datacite:publisher>SoBigData++</datacite:publisher>
<datacite:publicationYear/>
<datacite:dates>
<datacite:date dateType="Issued"/>
</datacite:dates>
<datacite:resourceType resourceTypeGeneral="TrainingMaterial">TrainingMaterial</datacite:resourceType>
<datacite:descriptions>
<datacite:description descriptionType="Abstract">Participants to this module shall
- Learn the principles and rules underlying the design of visual data
representations and human-computer interactions
- Understand, adapt and apply representative visual analytics methods and systems for diverse types
of data and problems
- Analyse and evaluate the structure and properties
of data to select or devise appropriate methods for data exploration
- Combine visualization, interactive techniques, and computational
processing to develop practical data analysis for problem solving
(This teaching material on Visual Analytics for Data Scientists is part of a MSc module at City University London).
The author did not intend to violate any copyright on figures or content. In case you are the legal owner of any copyrighted content, please contact info@sobigdata.eu and we will immediately remove it</datacite:description>
</datacite:descriptions>
<datacite:subjects>
<datacite:subject>Visual analytics</datacite:subject>
</datacite:subjects>
<datacite:formats>
<datacite:format>Slides</datacite:format>
<datacite:format>Other</datacite:format>
<datacite:format>PDF</datacite:format>
<datacite:format>PDF</datacite:format>
<datacite:format>PDF</datacite:format>
<datacite:format>PDF</datacite:format>
<datacite:format>PDF</datacite:format>
<datacite:format>PDF</datacite:format>
<datacite:format>PDF</datacite:format>
<datacite:format>PDF</datacite:format>
<datacite:format>PDF</datacite:format>
<datacite:format>PDF</datacite:format>
<datacite:format>ZIP</datacite:format>
</datacite:formats>
</datacite:resource>
<oaf:accessrights>OPEN</oaf:accessrights>
<dr:CobjCategory type="other">0010</dr:CobjCategory>
<oaf:dateAccepted/>
<oaf:hostedBy id="alessia_____::alessia" name="Alessia"/>
<oaf:collectedFrom id="alessia_____::alessia" name="Alessia"/>
<oaf:license>other-open</oaf:license>
<oaf:projectid>corda__h2020::871042</oaf:projectid>
</oai:metadata>
<about xmlns:dc="http://purl.org/dc/elements/1.1/"
xmlns:prov="http://www.openarchives.org/OAI/2.0/provenance" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
<provenance xmlns="http://www.openarchives.org/OAI/2.0/provenance" xsi:schemaLocation="http://www.openarchives.org/OAI/2.0/provenance http://www.openarchives.org/OAI/2.0/provenance.xsd">
<originDescription altered="true" harvestDate="2023-09-08T10:12:35.864+02:00">
<baseURL>https%3A%2F%2Fapi.d4science.org%2Fcatalogue%2Fitems</baseURL>
<identifier/>
<datestamp/>
<metadataNamespace/>
</originDescription>
</provenance>
<oaf:datainfo>
<oaf:inferred>false</oaf:inferred>
<oaf:deletedbyinference>false</oaf:deletedbyinference>
<oaf:trust>0.9</oaf:trust>
<oaf:inferenceprovenance/>
<oaf:provenanceaction classid="sysimport:crosswalk"
classname="Harvested" schemeid="dnet:provenanceActions" schemename="dnet:provenanceActions"/>
</oaf:datainfo>
</about>
</oai:record>

View File

@ -0,0 +1,72 @@
<?xml version="1.0" encoding="UTF-8"?>
<oai:record xmlns:dr="http://www.driver-repository.eu/namespace/dr"
xmlns:dri="http://www.driver-repository.eu/namespace/dri"
xmlns:oaf="http://namespace.openaire.eu/oaf" xmlns:oai="http://www.openarchives.org/OAI/2.0/">
<oai:header>
<dri:objIdentifier>alessia_____::028879484548f4e1c630e1c503e35231</dri:objIdentifier>
<dri:recordIdentifier>4fed018e-c2ff-4afa-b7b5-1ca1beebf850</dri:recordIdentifier>
<dri:dateOfCollection>2023-09-08T12:14:27.615+02:00</dri:dateOfCollection>
<oaf:datasourceprefix>alessia_____</oaf:datasourceprefix>
<dr:dateOfTransformation>2023-09-08T12:14:51.7+02:00</dr:dateOfTransformation>
</oai:header>
<oai:metadata>
<datacite:resource
xmlns:datacite="http://datacite.org/schema/kernel-4"
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://datacite.org/schema/kernel-4 http://schema.datacite.org/meta/kernel-4/metadata.xsd">
<datacite:identifier identifierType="URL">http://data.d4science.org/ctlg/ResourceCatalogue/city-to-city_migration</datacite:identifier>
<datacite:alternateIdentifiers>
<datacite:alternateIdentifier type="URL"/>
</datacite:alternateIdentifiers>
<datacite:creators>
<datacite:creator>
<datacite:creatorName>Pappalardo, Luca</datacite:creatorName>
<datacite:affiliation/>
<datacite:nameIdentifier nameIdentifierScheme="ORCID" schemeURI="http://orcid.org">0000-0002-1547-6007</datacite:nameIdentifier>
</datacite:creator>
</datacite:creators>
<datacite:titles>
<datacite:title>City-to-city migration</datacite:title>
</datacite:titles>
<datacite:publisher>SoBigData++</datacite:publisher>
<datacite:publicationYear/>
<datacite:dates>
<datacite:date dateType="Issued">2018-02-15</datacite:date>
</datacite:dates>
<datacite:resourceType resourceTypeGeneral="Dataset">Dataset</datacite:resourceType>
<datacite:descriptions>
<datacite:description descriptionType="Abstract">Census data recording the migration of people between metropolitan areas in
the US</datacite:description>
</datacite:descriptions>
<datacite:subjects>
<datacite:subject>Human Mobility data</datacite:subject>
</datacite:subjects>
<datacite:formats/>
</datacite:resource>
<oaf:accessrights>OPEN</oaf:accessrights>
<dr:CobjCategory type="dataset">0021</dr:CobjCategory>
<oaf:dateAccepted>2018-02-15</oaf:dateAccepted>
<oaf:hostedBy id="alessia_____::alessia" name="Alessia"/>
<oaf:collectedFrom id="alessia_____::alessia" name="Alessia"/>
<oaf:license>AFL-3.0</oaf:license>
<oaf:projectid>corda__h2020::871042</oaf:projectid>
</oai:metadata>
<about xmlns:dc="http://purl.org/dc/elements/1.1/"
xmlns:prov="http://www.openarchives.org/OAI/2.0/provenance" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
<provenance xmlns="http://www.openarchives.org/OAI/2.0/provenance" xsi:schemaLocation="http://www.openarchives.org/OAI/2.0/provenance http://www.openarchives.org/OAI/2.0/provenance.xsd">
<originDescription altered="true" harvestDate="2023-09-08T12:14:27.615+02:00">
<baseURL>https%3A%2F%2Fapi.d4science.org%2Fcatalogue%2Fitems</baseURL>
<identifier/>
<datestamp/>
<metadataNamespace/>
</originDescription>
</provenance>
<oaf:datainfo>
<oaf:inferred>false</oaf:inferred>
<oaf:deletedbyinference>false</oaf:deletedbyinference>
<oaf:trust>0.9</oaf:trust>
<oaf:inferenceprovenance/>
<oaf:provenanceaction classid="sysimport:crosswalk"
classname="Harvested" schemeid="dnet:provenanceActions" schemename="dnet:provenanceActions"/>
</oaf:datainfo>
</about>
</oai:record>

View File

@ -24,10 +24,7 @@ import eu.dnetlib.dhp.oa.provision.model.RelatedEntity;
import eu.dnetlib.dhp.oa.provision.model.RelatedEntityWrapper; import eu.dnetlib.dhp.oa.provision.model.RelatedEntityWrapper;
import eu.dnetlib.dhp.oa.provision.utils.ContextMapper; import eu.dnetlib.dhp.oa.provision.utils.ContextMapper;
import eu.dnetlib.dhp.oa.provision.utils.XmlRecordFactory; import eu.dnetlib.dhp.oa.provision.utils.XmlRecordFactory;
import eu.dnetlib.dhp.schema.oaf.Datasource; import eu.dnetlib.dhp.schema.oaf.*;
import eu.dnetlib.dhp.schema.oaf.Project;
import eu.dnetlib.dhp.schema.oaf.Publication;
import eu.dnetlib.dhp.schema.oaf.Relation;
public class XmlRecordFactoryTest { public class XmlRecordFactoryTest {
@ -196,4 +193,51 @@ public class XmlRecordFactoryTest {
assertEquals("dnet:pid_types", ((Element) pids.get(0)).attribute("schemeid").getValue()); assertEquals("dnet:pid_types", ((Element) pids.get(0)).attribute("schemeid").getValue());
assertEquals("dnet:pid_types", ((Element) pids.get(0)).attribute("schemename").getValue()); assertEquals("dnet:pid_types", ((Element) pids.get(0)).attribute("schemename").getValue());
} }
@Test
public void testD4ScienceTraining() throws DocumentException, IOException {
final ContextMapper contextMapper = new ContextMapper();
final XmlRecordFactory xmlRecordFactory = new XmlRecordFactory(contextMapper, false,
XmlConverterJob.schemaLocation);
final OtherResearchProduct p = OBJECT_MAPPER
.readValue(
IOUtils.toString(getClass().getResourceAsStream("d4science-1-training.json")),
OtherResearchProduct.class);
final String xml = xmlRecordFactory.build(new JoinedEntity<>(p));
assertNotNull(xml);
final Document doc = new SAXReader().read(new StringReader(xml));
assertNotNull(doc);
System.out.println(doc.asXML());
}
@Test
public void testD4ScienceDataset() throws DocumentException, IOException {
final ContextMapper contextMapper = new ContextMapper();
final XmlRecordFactory xmlRecordFactory = new XmlRecordFactory(contextMapper, false,
XmlConverterJob.schemaLocation);
final OtherResearchProduct p = OBJECT_MAPPER
.readValue(
IOUtils.toString(getClass().getResourceAsStream("d4science-2-dataset.json")),
OtherResearchProduct.class);
final String xml = xmlRecordFactory.build(new JoinedEntity<>(p));
assertNotNull(xml);
final Document doc = new SAXReader().read(new StringReader(xml));
assertNotNull(doc);
System.out.println(doc.asXML());
}
} }

File diff suppressed because one or more lines are too long

File diff suppressed because one or more lines are too long

View File

@ -39,7 +39,8 @@
<switch> <switch>
<!-- The default will be set as the normal start, a.k.a. get-doi-synonyms --> <!-- The default will be set as the normal start, a.k.a. get-doi-synonyms -->
<!-- If any different condition is set, go to the corresponding start --> <!-- If any different condition is set, go to the corresponding start -->
<case to="non-iterative-rankings">${wf:conf('resume') eq "rankings-start"}</case> <case to="spark-cc">${wf:conf('resume') eq "cc"}</case>
<case to="spark-ram">${wf:conf('resume') eq "ram"}</case>
<case to="spark-impulse">${wf:conf('resume') eq "impulse"}</case> <case to="spark-impulse">${wf:conf('resume') eq "impulse"}</case>
<case to="spark-pagerank">${wf:conf('resume') eq "pagerank"}</case> <case to="spark-pagerank">${wf:conf('resume') eq "pagerank"}</case>
<case to="spark-attrank">${wf:conf('resume') eq "attrank"}</case> <case to="spark-attrank">${wf:conf('resume') eq "attrank"}</case>
@ -89,18 +90,11 @@
<file>${nameNode}${wfAppPath}/create_openaire_ranking_graph.py#create_openaire_ranking_graph.py</file> <file>${nameNode}${wfAppPath}/create_openaire_ranking_graph.py#create_openaire_ranking_graph.py</file>
</spark> </spark>
<ok to="non-iterative-rankings" /> <ok to="spark-cc"/>
<error to="openaire-graph-error" /> <error to="openaire-graph-error" />
</action> </action>
<!-- Citation Count and RAM are calculated in parallel-->
<fork name="non-iterative-rankings">
<path start="spark-cc"/>
<!-- <path start="spark-impulse"/> -->
<path start="spark-ram"/>
</fork>
<!-- Run Citation Count calculation --> <!-- Run Citation Count calculation -->
<action name="spark-cc"> <action name="spark-cc">
<spark xmlns="uri:oozie:spark-action:0.2"> <spark xmlns="uri:oozie:spark-action:0.2">
@ -129,7 +123,7 @@
<file>${wfAppPath}/bip-ranker/CC.py#CC.py</file> <file>${wfAppPath}/bip-ranker/CC.py#CC.py</file>
</spark> </spark>
<ok to="join-non-iterative-rankings" /> <ok to="spark-ram" />
<error to="cc-fail" /> <error to="cc-fail" />
</action> </action>
@ -165,14 +159,11 @@
<file>${wfAppPath}/bip-ranker/TAR.py#TAR.py</file> <file>${wfAppPath}/bip-ranker/TAR.py#TAR.py</file>
</spark> </spark>
<ok to="join-non-iterative-rankings" /> <ok to="spark-impulse" />
<error to="ram-fail" /> <error to="ram-fail" />
</action> </action>
<!-- Join non-iterative methods -->
<join name="join-non-iterative-rankings" to="spark-impulse"/>
<action name="spark-impulse"> <action name="spark-impulse">
<spark xmlns="uri:oozie:spark-action:0.2"> <spark xmlns="uri:oozie:spark-action:0.2">

10
pom.xml
View File

@ -112,6 +112,16 @@
<url>https://maven.d4science.org/nexus/content/repositories/dnet-deps</url> <url>https://maven.d4science.org/nexus/content/repositories/dnet-deps</url>
<layout>default</layout> <layout>default</layout>
</repository> </repository>
<repository>
<id>maven-restlet</id>
<name>Restlet repository</name>
<url>https://maven.restlet.talend.com</url>
</repository>
<repository>
<id>conjars</id>
<name>conjars</name>
<url>https://conjars.wensel.net/repo/</url>
</repository>
</repositories> </repositories>
<dependencies> <dependencies>