Merge branch 'beta' into hierarchical_orgs_relations

This commit is contained in:
Claudio Atzori 2021-10-19 16:46:06 +02:00
commit 515e068a78
161 changed files with 13055 additions and 1056 deletions

View File

@ -85,6 +85,13 @@ public class MakeTarArchive implements Serializable {
String p_string = p.toString();
if (!p_string.endsWith("_SUCCESS")) {
String name = p_string.substring(p_string.lastIndexOf("/") + 1);
if (name.startsWith("part-") & name.length() > 10) {
String tmp = name.substring(0, 10);
if (name.contains(".")) {
tmp += name.substring(name.indexOf("."));
}
name = tmp;
}
TarArchiveEntry entry = new TarArchiveEntry(dir_name + "/" + name);
entry.setSize(fileStatus.getLen());
current_size += fileStatus.getLen();

View File

@ -1,4 +1,4 @@
package eu.dnetllib.dhp.sx.bio
package eu.dnetlib.dhp.sx.bio
import eu.dnetlib.dhp.schema.common.ModelConstants
import eu.dnetlib.dhp.schema.oaf.utils.{GraphCleaningFunctions, OafMapperUtils}

View File

@ -1,8 +1,8 @@
package eu.dnetllib.dhp.sx.bio
package eu.dnetlib.dhp.sx.bio
import eu.dnetlib.dhp.application.ArgumentApplicationParser
import eu.dnetlib.dhp.schema.oaf.Oaf
import eu.dnetllib.dhp.sx.bio.BioDBToOAF.ScholixResolved
import BioDBToOAF.ScholixResolved
import org.apache.commons.io.IOUtils
import org.apache.spark.SparkConf
import org.apache.spark.sql.{Encoder, Encoders, SaveMode, SparkSession}
@ -13,7 +13,7 @@ object SparkTransformBioDatabaseToOAF {
def main(args: Array[String]): Unit = {
val conf: SparkConf = new SparkConf()
val log: Logger = LoggerFactory.getLogger(getClass)
val parser = new ArgumentApplicationParser(IOUtils.toString(getClass.getResourceAsStream("/eu/dnetlib/dhp/sx/graph/bio/bio_to_oaf_params.json")))
val parser = new ArgumentApplicationParser(IOUtils.toString(getClass.getResourceAsStream("/eu/dnetlib/dhp/sx/bio/ebi/bio_to_oaf_params.json")))
parser.parseArgument(args)
val database: String = parser.get("database")
log.info("database: {}", database)

View File

@ -1,10 +1,10 @@
package eu.dnetllib.dhp.sx.bio.ebi
package eu.dnetlib.dhp.sx.bio.ebi
import eu.dnetlib.dhp.application.ArgumentApplicationParser
import eu.dnetlib.dhp.common.vocabulary.VocabularyGroup
import eu.dnetlib.dhp.schema.oaf.Result
import eu.dnetlib.dhp.sx.bio.pubmed.{PMArticle, PMAuthor, PMJournal, PMParser, PubMedToOaf}
import eu.dnetlib.dhp.utils.ISLookupClientFactory
import eu.dnetllib.dhp.sx.bio.pubmed.{PMArticle, PMAuthor, PMJournal, PMParser, PubMedToOaf}
import org.apache.commons.io.IOUtils
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.{FSDataOutputStream, FileSystem, Path}

View File

@ -1,8 +1,9 @@
package eu.dnetllib.dhp.sx.bio.ebi
package eu.dnetlib.dhp.sx.bio.ebi
import eu.dnetlib.dhp.application.ArgumentApplicationParser
import eu.dnetllib.dhp.sx.bio.BioDBToOAF.EBILinkItem
import eu.dnetllib.dhp.sx.bio.pubmed.{PMArticle, PMAuthor, PMJournal}
import eu.dnetlib.dhp.sx.bio.pubmed.{PMArticle, PMAuthor, PMJournal}
import eu.dnetlib.dhp.sx.bio.BioDBToOAF.EBILinkItem
import eu.dnetlib.dhp.sx.bio.pubmed.PMJournal
import org.apache.commons.io.IOUtils
import org.apache.http.client.config.RequestConfig
import org.apache.http.client.methods.HttpGet
@ -66,7 +67,7 @@ object SparkDownloadEBILinks {
val log: Logger = LoggerFactory.getLogger(getClass)
val MAX_ITEM_PER_PARTITION = 20000
val conf: SparkConf = new SparkConf()
val parser = new ArgumentApplicationParser(IOUtils.toString(getClass.getResourceAsStream("/eu/dnetlib/dhp/sx/graph/ebi/ebi_download_update.json")))
val parser = new ArgumentApplicationParser(IOUtils.toString(getClass.getResourceAsStream("/eu/dnetlib/dhp/sx/bio/ebi/ebi_download_update.json")))
parser.parseArgument(args)
val spark: SparkSession =
SparkSession
@ -86,7 +87,7 @@ object SparkDownloadEBILinks {
val workingPath = parser.get("workingPath")
log.info(s"workingPath -> $workingPath")
log.info("Getting max pubmedId where the links have been requested")
log.info("Getting max pubmedId where the links have already requested")
val links: Dataset[EBILinkItem] = spark.read.load(s"$sourcePath/ebi_links_dataset").as[EBILinkItem]
val lastPMIDRequested = links.map(l => l.id).select(max("value")).first.getLong(0)

View File

@ -1,9 +1,10 @@
package eu.dnetllib.dhp.sx.bio.ebi
package eu.dnetlib.dhp.sx.bio.ebi
import eu.dnetlib.dhp.application.ArgumentApplicationParser
import eu.dnetlib.dhp.schema.oaf.Oaf
import eu.dnetllib.dhp.sx.bio.BioDBToOAF
import eu.dnetllib.dhp.sx.bio.BioDBToOAF.EBILinkItem
import eu.dnetlib.dhp.sx.bio.BioDBToOAF
import eu.dnetlib.dhp.sx.bio.BioDBToOAF.EBILinkItem
import BioDBToOAF.EBILinkItem
import org.apache.commons.io.IOUtils
import org.apache.spark.SparkConf
import org.apache.spark.sql._
@ -14,7 +15,7 @@ object SparkEBILinksToOaf {
def main(args: Array[String]): Unit = {
val log: Logger = LoggerFactory.getLogger(getClass)
val conf: SparkConf = new SparkConf()
val parser = new ArgumentApplicationParser(IOUtils.toString(getClass.getResourceAsStream("/eu/dnetlib/dhp/sx/graph/ebi/ebi_to_df_params.json")))
val parser = new ArgumentApplicationParser(IOUtils.toString(getClass.getResourceAsStream("/eu/dnetlib/dhp/sx/bio/ebi/ebi_to_df_params.json")))
parser.parseArgument(args)
val spark: SparkSession =
SparkSession
@ -31,7 +32,7 @@ object SparkEBILinksToOaf {
log.info(s"targetPath -> $targetPath")
implicit val PMEncoder: Encoder[Oaf] = Encoders.kryo(classOf[Oaf])
val ebLinks: Dataset[EBILinkItem] = spark.read.load(s"${sourcePath}_dataset").as[EBILinkItem].filter(l => l.links != null)
val ebLinks: Dataset[EBILinkItem] = spark.read.load(sourcePath).as[EBILinkItem].filter(l => l.links != null && l.links.startsWith("{"))
ebLinks.flatMap(j => BioDBToOAF.parse_ebi_links(j.links))
.filter(p => BioDBToOAF.EBITargetLinksFilter(p))

View File

@ -1,5 +1,5 @@
package eu.dnetllib.dhp.sx.bio.pubmed;
package eu.dnetlib.dhp.sx.bio.pubmed;
import java.io.Serializable;
import java.util.ArrayList;

View File

@ -1,5 +1,5 @@
package eu.dnetllib.dhp.sx.bio.pubmed;
package eu.dnetlib.dhp.sx.bio.pubmed;
import java.io.Serializable;

View File

@ -1,5 +1,5 @@
package eu.dnetllib.dhp.sx.bio.pubmed;
package eu.dnetlib.dhp.sx.bio.pubmed;
public class PMGrant {

View File

@ -1,5 +1,5 @@
package eu.dnetllib.dhp.sx.bio.pubmed;
package eu.dnetlib.dhp.sx.bio.pubmed;
import java.io.Serializable;

View File

@ -1,4 +1,4 @@
package eu.dnetllib.dhp.sx.bio.pubmed
package eu.dnetlib.dhp.sx.bio.pubmed
import scala.xml.MetaData
import scala.xml.pull.{EvElemEnd, EvElemStart, EvText, XMLEventReader}

View File

@ -1,5 +1,5 @@
package eu.dnetllib.dhp.sx.bio.pubmed;
package eu.dnetlib.dhp.sx.bio.pubmed;
public class PMSubject {
private String value;

View File

@ -1,4 +1,4 @@
package eu.dnetllib.dhp.sx.bio.pubmed
package eu.dnetlib.dhp.sx.bio.pubmed
import eu.dnetlib.dhp.common.vocabulary.VocabularyGroup
import eu.dnetlib.dhp.schema.common.ModelConstants

View File

@ -0,0 +1,51 @@
<workflow-app name="Transform_BioEntity_Workflow" xmlns="uri:oozie:workflow:0.5">
<parameters>
<property>
<name>sourcePath</name>
<description>the PDB Database Working Path</description>
</property>
<property>
<name>database</name>
<description>the PDB Database Working Path</description>
</property>
<property>
<name>targetPath</name>
<description>the Target Working dir path</description>
</property>
</parameters>
<start to="ConvertDB"/>
<kill name="Kill">
<message>Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}]</message>
</kill>
<action name="ConvertDB">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn</master>
<mode>cluster</mode>
<name>Convert Bio DB to OAF Dataset</name>
<class>eu.dnetlib.dhp.sx.bio.SparkTransformBioDatabaseToOAF</class>
<jar>dhp-aggregation-${projectVersion}.jar</jar>
<spark-opts>
--executor-memory=${sparkExecutorMemory}
--executor-cores=${sparkExecutorCores}
--driver-memory=${sparkDriverMemory}
--conf spark.extraListeners=${spark2ExtraListeners}
--conf spark.sql.shuffle.partitions=2000
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
</spark-opts>
<arg>--master</arg><arg>yarn</arg>
<arg>--dbPath</arg><arg>${sourcePath}</arg>
<arg>--database</arg><arg>${database}</arg>
<arg>--targetPath</arg><arg>${targetPath}</arg>
</spark>
<ok to="End"/>
<error to="Kill"/>
</action>
<end name="End"/>
</workflow-app>

View File

@ -0,0 +1,105 @@
<workflow-app name="Create EBI Dataset" xmlns="uri:oozie:workflow:0.5">
<parameters>
<property>
<name>sourcePath</name>
<description>the Working Path</description>
</property>
<property>
<name>workingPath</name>
<description>the Working Path</description>
</property>
<property>
<name>targetPath</name>
<description>the OAF MDStore Path</description>
</property>
<property>
<name>sparkDriverMemory</name>
<description>memory for driver process</description>
</property>
<property>
<name>sparkExecutorMemory</name>
<description>memory for individual executor</description>
</property>
<property>
<name>sparkExecutorCores</name>
<description>number of cores used by single executor</description>
</property>
<property>
<name>resumeFrom</name>
<value>DownloadEBILinks</value>
<description>node to start</description>
</property>
</parameters>
<start to="resume_from"/>
<decision name="resume_from">
<switch>
<case to="DownloadEBILinks">${wf:conf('resumeFrom') eq 'DownloadEBILinks'}</case>
<case to="CreateEBIDataSet">${wf:conf('resumeFrom') eq 'CreateEBIDataSet'}</case>
<default to="DownloadEBILinks"/>
</switch>
</decision>
<kill name="Kill">
<message>Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}]</message>
</kill>
<action name="DownloadEBILinks">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn-cluster</master>
<mode>cluster</mode>
<name>Incremental Download EBI Links</name>
<class>eu.dnetllib.dhp.sx.bio.ebi.SparkDownloadEBILinks</class>
<jar>dhp-aggregation-${projectVersion}.jar</jar>
<spark-opts>
--executor-memory=${sparkExecutorMemory}
--executor-cores=${sparkExecutorCores}
--driver-memory=${sparkDriverMemory}
--conf spark.extraListeners=${spark2ExtraListeners}
--conf spark.sql.shuffle.partitions=2000
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
</spark-opts>
<arg>--sourcePath</arg><arg>${sourcePath}</arg>
<arg>--workingPath</arg><arg>${workingPath}</arg>
<arg>--master</arg><arg>yarn</arg>
</spark>
<ok to="OverrideFolders"/>
<error to="Kill"/>
</action>
<action name="OverrideFolders">
<fs>
<delete path="${sourcePath}/ebi_links_dataset_old"/>
<move source="${sourcePath}/ebi_links_dataset" target="${sourcePath}/ebi_links_dataset_old"/>
<move source="${workingPath}/links_final" target="${sourcePath}/ebi_links_dataset"/>
</fs>
<ok to="CreateEBIDataSet"/>
<error to="Kill"/>
</action>
<action name="CreateEBIDataSet">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn-cluster</master>
<mode>cluster</mode>
<name>Create OAF DataSet</name>
<class>eu.dnetllib.dhp.sx.bio.ebi.SparkEBILinksToOaf</class>
<jar>dhp-aggregation-${projectVersion}.jar</jar>
<spark-opts>
--executor-memory=${sparkExecutorMemory}
--executor-cores=${sparkExecutorCores}
--driver-memory=${sparkDriverMemory}
--conf spark.sql.shuffle.partitions=2000
${sparkExtraOPT}
</spark-opts>
<arg>--sourcePath</arg><arg>${sourcePath}/ebi_links_dataset</arg>
<arg>--targetPath</arg><arg>${targetPath}</arg>
<arg>--master</arg><arg>yarn</arg>
</spark>
<ok to="End"/>
<error to="Kill"/>
</action>
<end name="End"/>
</workflow-app>

View File

@ -1,10 +1,10 @@
package eu.dnetllib.dhp.sx.bio
package eu.dnetlib.dhp.sx.bio
import com.fasterxml.jackson.databind.{DeserializationFeature, ObjectMapper, SerializationFeature}
import eu.dnetlib.dhp.aggregation.AbstractVocabularyTest
import eu.dnetlib.dhp.schema.oaf.{Oaf, Relation, Result}
import eu.dnetllib.dhp.sx.bio.BioDBToOAF.ScholixResolved
import eu.dnetllib.dhp.sx.bio.pubmed.{PMArticle, PMParser, PubMedToOaf}
import eu.dnetlib.dhp.sx.bio.BioDBToOAF.ScholixResolved
import eu.dnetlib.dhp.sx.bio.pubmed.{PMArticle, PMParser, PubMedToOaf}
import org.json4s.DefaultFormats
import org.json4s.JsonAST.{JField, JObject, JString}
import org.json4s.jackson.JsonMethods.parse

View File

@ -91,8 +91,8 @@ public class ReadBlacklistFromDB implements Closeable {
String encoding = rs.getString("relationship");
RelationInverse ri = ModelSupport.relationInverseMap.get(encoding);
direct.setRelClass(ri.getRelation());
inverse.setRelClass(ri.getInverse());
direct.setRelClass(ri.getRelClass());
inverse.setRelClass(ri.getInverseRelClass());
direct.setRelType(ri.getRelType());
inverse.setRelType(ri.getRelType());
direct.setSubRelType(ri.getSubReltype());

View File

@ -107,7 +107,6 @@
<action name="ResetMagWorkingPath">
<fs>
<delete path="${inputPathMAG}/dataset"/>
<delete path="${inputPathMAG}/process"/>
</fs>
<ok to="ConvertMagToDataset"/>
<error to="Kill"/>

View File

@ -50,10 +50,6 @@
<name>inputPathMAG</name>
<description>the MAG input path</description>
</property>
<property>
<name>workingPathMAG</name>
<description>the MAG working path where to store the intermediate process results</description>
</property>
<!-- UnpayWall Parameters -->
@ -142,7 +138,7 @@
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
</spark-opts>
<arg>--sourcePath</arg><arg>${inputPathMAG}/dataset</arg>
<arg>--workingPath</arg><arg>${workingPathMAG}</arg>
<arg>--workingPath</arg><arg>${workingPath}/MAG</arg>
<arg>--targetPath</arg><arg>${workingPath}</arg>
<arg>--master</arg><arg>yarn-cluster</arg>
</spark>

View File

@ -5,37 +5,40 @@ import java.util.Map;
import com.google.common.collect.Maps;
import eu.dnetlib.dhp.schema.common.ModelConstants;
public class Constants {
public static final Map<String, String> accessRightsCoarMap = Maps.newHashMap();
public static final Map<String, String> coarCodeLabelMap = Maps.newHashMap();
protected static final Map<String, String> accessRightsCoarMap = Maps.newHashMap();
protected static final Map<String, String> coarCodeLabelMap = Maps.newHashMap();
public static final String INFERRED = "Inferred by OpenAIRE";
public static final String CABF2 = "c_abf2";
public static final String HARVESTED = "Harvested";
public static final String DEFAULT_TRUST = "0.9";
public static final String USER_CLAIM = "Linked by user";
public static String COAR_ACCESS_RIGHT_SCHEMA = "http://vocabularies.coar-repositories.org/documentation/access_rights/";
public static final String COAR_ACCESS_RIGHT_SCHEMA = "http://vocabularies.coar-repositories.org/documentation/access_rights/";
public static String ZENODO_COMMUNITY_PREFIX = "https://zenodo.org/communities/";
public static final String ZENODO_COMMUNITY_PREFIX = "https://zenodo.org/communities/";
public static String RESEARCH_COMMUNITY = "Research Community";
public static final String RESEARCH_COMMUNITY = "Research Community";
public static String RESEARCH_INFRASTRUCTURE = "Research Infrastructure/Initiative";
public static final String RESEARCH_INFRASTRUCTURE = "Research Infrastructure/Initiative";
static {
accessRightsCoarMap.put("OPEN", "c_abf2");
accessRightsCoarMap.put(ModelConstants.ACCESS_RIGHT_OPEN, CABF2);
accessRightsCoarMap.put("RESTRICTED", "c_16ec");
accessRightsCoarMap.put("OPEN SOURCE", "c_abf2");
accessRightsCoarMap.put("CLOSED", "c_14cb");
accessRightsCoarMap.put("EMBARGO", "c_f1cf");
accessRightsCoarMap.put("OPEN SOURCE", CABF2);
accessRightsCoarMap.put(ModelConstants.ACCESS_RIGHT_CLOSED, "c_14cb");
accessRightsCoarMap.put(ModelConstants.ACCESS_RIGHT_EMBARGO, "c_f1cf");
}
static {
coarCodeLabelMap.put("c_abf2", "OPEN");
coarCodeLabelMap.put(CABF2, ModelConstants.ACCESS_RIGHT_OPEN);
coarCodeLabelMap.put("c_16ec", "RESTRICTED");
coarCodeLabelMap.put("c_14cb", "CLOSED");
coarCodeLabelMap.put("c_14cb", ModelConstants.ACCESS_RIGHT_CLOSED);
coarCodeLabelMap.put("c_f1cf", "EMBARGO");
}

View File

@ -11,12 +11,14 @@ import java.util.Set;
import java.util.stream.Collectors;
import org.apache.spark.SparkConf;
import org.apache.spark.api.java.function.FilterFunction;
import org.apache.spark.api.java.function.MapFunction;
import org.apache.spark.sql.Encoders;
import org.apache.spark.sql.SaveMode;
import org.apache.spark.sql.SparkSession;
import eu.dnetlib.dhp.oa.graph.dump.community.CommunityMap;
import eu.dnetlib.dhp.oa.graph.dump.exceptions.NoAvailableEntityTypeException;
import eu.dnetlib.dhp.schema.oaf.*;
/**
@ -37,7 +39,8 @@ public class DumpProducts implements Serializable {
isSparkSessionManaged,
spark -> {
Utils.removeOutputDir(spark, outputPath);
execDump(spark, inputPath, outputPath, communityMapPath, inputClazz, outputClazz, dumpType);
execDump(
spark, inputPath, outputPath, communityMapPath, inputClazz, outputClazz, dumpType);
});
}
@ -55,7 +58,7 @@ public class DumpProducts implements Serializable {
Utils
.readPath(spark, inputPath, inputClazz)
.map((MapFunction<I, O>) value -> execMap(value, communityMap, dumpType), Encoders.bean(outputClazz))
.filter(Objects::nonNull)
.filter((FilterFunction<O>) value -> value != null)
.write()
.mode(SaveMode.Overwrite)
.option("compression", "gzip")
@ -65,7 +68,7 @@ public class DumpProducts implements Serializable {
private static <I extends OafEntity, O extends eu.dnetlib.dhp.schema.dump.oaf.Result> O execMap(I value,
CommunityMap communityMap,
String dumpType) {
String dumpType) throws NoAvailableEntityTypeException {
Optional<DataInfo> odInfo = Optional.ofNullable(value.getDataInfo());
if (odInfo.isPresent()) {
@ -89,11 +92,11 @@ public class DumpProducts implements Serializable {
return c.getId();
}
if (c.getId().contains("::") && communities.contains(c.getId().substring(0, c.getId().indexOf("::")))) {
return c.getId().substring(0, 3);
return c.getId().substring(0, c.getId().indexOf("::"));
}
return null;
}).filter(Objects::nonNull).collect(Collectors.toList());
if (toDumpFor.size() == 0) {
if (toDumpFor.isEmpty()) {
return null;
}
}

View File

@ -57,16 +57,16 @@ public class MakeTar implements Serializable {
public static void makeTArArchive(FileSystem fileSystem, String inputPath, String outputPath, int gBperSplit)
throws IOException {
RemoteIterator<LocatedFileStatus> dir_iterator = fileSystem.listLocatedStatus(new Path(inputPath));
RemoteIterator<LocatedFileStatus> dirIterator = fileSystem.listLocatedStatus(new Path(inputPath));
while (dir_iterator.hasNext()) {
LocatedFileStatus fileStatus = dir_iterator.next();
while (dirIterator.hasNext()) {
LocatedFileStatus fileStatus = dirIterator.next();
Path p = fileStatus.getPath();
String p_string = p.toString();
String entity = p_string.substring(p_string.lastIndexOf("/") + 1);
String pathString = p.toString();
String entity = pathString.substring(pathString.lastIndexOf("/") + 1);
MakeTarArchive.tarMaxSize(fileSystem, p_string, outputPath + "/" + entity, entity, gBperSplit);
MakeTarArchive.tarMaxSize(fileSystem, pathString, outputPath + "/" + entity, entity, gBperSplit);
}
}

View File

@ -18,10 +18,10 @@ public class QueryInformationSystem {
private ISLookUpService isLookUp;
private static final String XQUERY = "for $x in collection('/db/DRIVER/ContextDSResources/ContextDSResourceType') "
private static final String XQUERY_ALL = "for $x in collection('/db/DRIVER/ContextDSResources/ContextDSResourceType') "
+
" where $x//CONFIGURATION/context[./@type='community' or ./@type='ri'] " +
" and ($x//context/param[./@name = 'status']/text() = 'manager' or $x//context/param[./@name = 'status']/text() = 'all') "
" and ($x//context/param[./@name = 'status']/text() = 'all') "
+
" return " +
"<community> " +
@ -29,9 +29,22 @@ public class QueryInformationSystem {
"{$x//CONFIGURATION/context/@label}" +
"</community>";
public CommunityMap getCommunityMap()
private static final String XQUERY_CI = "for $x in collection('/db/DRIVER/ContextDSResources/ContextDSResourceType') "
+
" where $x//CONFIGURATION/context[./@type='community' or ./@type='ri'] " +
" and $x//CONFIGURATION/context[./@id=%s] "
+
" return " +
"<community> " +
"{$x//CONFIGURATION/context/@id}" +
"{$x//CONFIGURATION/context/@label}" +
"</community>";
public CommunityMap getCommunityMap(boolean singleCommunity, String communityId)
throws ISLookUpException, DocumentException, SAXException {
return getMap(isLookUp.quickSearchProfile(XQUERY));
if (singleCommunity)
return getMap(isLookUp.quickSearchProfile(XQUERY_CI.replace("%s", "'" + communityId + "'")));
return getMap(isLookUp.quickSearchProfile(XQUERY_ALL));
}

View File

@ -7,21 +7,28 @@ import java.util.stream.Collectors;
import org.apache.commons.lang3.StringUtils;
import eu.dnetlib.dhp.oa.graph.dump.exceptions.NoAvailableEntityTypeException;
import eu.dnetlib.dhp.schema.common.ModelConstants;
import eu.dnetlib.dhp.schema.dump.oaf.*;
import eu.dnetlib.dhp.schema.dump.oaf.AccessRight;
import eu.dnetlib.dhp.schema.dump.oaf.Author;
import eu.dnetlib.dhp.schema.dump.oaf.Country;
import eu.dnetlib.dhp.schema.dump.oaf.GeoLocation;
import eu.dnetlib.dhp.schema.dump.oaf.Instance;
import eu.dnetlib.dhp.schema.dump.oaf.KeyValue;
import eu.dnetlib.dhp.schema.dump.oaf.OpenAccessRoute;
import eu.dnetlib.dhp.schema.dump.oaf.Qualifier;
import eu.dnetlib.dhp.schema.dump.oaf.Result;
import eu.dnetlib.dhp.schema.dump.oaf.community.CommunityInstance;
import eu.dnetlib.dhp.schema.dump.oaf.community.CommunityResult;
import eu.dnetlib.dhp.schema.dump.oaf.community.Context;
import eu.dnetlib.dhp.schema.dump.oaf.graph.GraphResult;
import eu.dnetlib.dhp.schema.oaf.DataInfo;
import eu.dnetlib.dhp.schema.oaf.Field;
import eu.dnetlib.dhp.schema.oaf.Journal;
import eu.dnetlib.dhp.schema.oaf.StructuredProperty;
import eu.dnetlib.dhp.schema.oaf.*;
public class ResultMapper implements Serializable {
public static <E extends eu.dnetlib.dhp.schema.oaf.OafEntity> Result map(
E in, Map<String, String> communityMap, String dumpType) {
E in, Map<String, String> communityMap, String dumpType) throws NoAvailableEntityTypeException {
Result out;
if (Constants.DUMPTYPE.COMPLETE.getType().equals(dumpType)) {
@ -33,113 +40,28 @@ public class ResultMapper implements Serializable {
eu.dnetlib.dhp.schema.oaf.Result input = (eu.dnetlib.dhp.schema.oaf.Result) in;
Optional<eu.dnetlib.dhp.schema.oaf.Qualifier> ort = Optional.ofNullable(input.getResulttype());
if (ort.isPresent()) {
switch (ort.get().getClassid()) {
case "publication":
Optional<Journal> journal = Optional
.ofNullable(((eu.dnetlib.dhp.schema.oaf.Publication) input).getJournal());
if (journal.isPresent()) {
Journal j = journal.get();
Container c = new Container();
c.setConferencedate(j.getConferencedate());
c.setConferenceplace(j.getConferenceplace());
c.setEdition(j.getEdition());
c.setEp(j.getEp());
c.setIss(j.getIss());
c.setIssnLinking(j.getIssnLinking());
c.setIssnOnline(j.getIssnOnline());
c.setIssnPrinted(j.getIssnPrinted());
c.setName(j.getName());
c.setSp(j.getSp());
c.setVol(j.getVol());
out.setContainer(c);
out.setType(ModelConstants.PUBLICATION_DEFAULT_RESULTTYPE.getClassname());
}
break;
case "dataset":
eu.dnetlib.dhp.schema.oaf.Dataset id = (eu.dnetlib.dhp.schema.oaf.Dataset) input;
Optional.ofNullable(id.getSize()).ifPresent(v -> out.setSize(v.getValue()));
Optional.ofNullable(id.getVersion()).ifPresent(v -> out.setVersion(v.getValue()));
try {
out
.setGeolocation(
Optional
.ofNullable(id.getGeolocation())
.map(
igl -> igl
.stream()
.filter(Objects::nonNull)
.map(gli -> {
GeoLocation gl = new GeoLocation();
gl.setBox(gli.getBox());
gl.setPlace(gli.getPlace());
gl.setPoint(gli.getPoint());
return gl;
})
.collect(Collectors.toList()))
.orElse(null));
addTypeSpecificInformation(out, input, ort);
Optional<List<Measure>> mes = Optional.ofNullable(input.getMeasures());
if (mes.isPresent()) {
List<KeyValue> measure = new ArrayList<>();
mes
.get()
.forEach(
m -> m.getUnit().forEach(u -> measure.add(KeyValue.newInstance(m.getId(), u.getValue()))));
out.setMeasures(measure);
}
out.setType(ModelConstants.DATASET_DEFAULT_RESULTTYPE.getClassname());
break;
case "software":
Optional
.ofNullable(input.getAuthor())
.ifPresent(
ats -> out.setAuthor(ats.stream().map(ResultMapper::getAuthor).collect(Collectors.toList())));
eu.dnetlib.dhp.schema.oaf.Software is = (eu.dnetlib.dhp.schema.oaf.Software) input;
Optional
.ofNullable(is.getCodeRepositoryUrl())
.ifPresent(value -> out.setCodeRepositoryUrl(value.getValue()));
Optional
.ofNullable(is.getDocumentationUrl())
.ifPresent(
value -> out
.setDocumentationUrl(
value
.stream()
.map(Field::getValue)
.collect(Collectors.toList())));
// I do not map Access Right UNKNOWN or OTHER
Optional
.ofNullable(is.getProgrammingLanguage())
.ifPresent(value -> out.setProgrammingLanguage(value.getClassid()));
out.setType(ModelConstants.SOFTWARE_DEFAULT_RESULTTYPE.getClassname());
break;
case "other":
eu.dnetlib.dhp.schema.oaf.OtherResearchProduct ir = (eu.dnetlib.dhp.schema.oaf.OtherResearchProduct) input;
out
.setContactgroup(
Optional
.ofNullable(ir.getContactgroup())
.map(value -> value.stream().map(Field::getValue).collect(Collectors.toList()))
.orElse(null));
out
.setContactperson(
Optional
.ofNullable(ir.getContactperson())
.map(value -> value.stream().map(Field::getValue).collect(Collectors.toList()))
.orElse(null));
out
.setTool(
Optional
.ofNullable(ir.getTool())
.map(value -> value.stream().map(Field::getValue).collect(Collectors.toList()))
.orElse(null));
out.setType(ModelConstants.ORP_DEFAULT_RESULTTYPE.getClassname());
break;
}
Optional
.ofNullable(input.getAuthor())
.ifPresent(
ats -> out.setAuthor(ats.stream().map(ResultMapper::getAuthor).collect(Collectors.toList())));
// I do not map Access Right UNKNOWN or OTHER
Optional<eu.dnetlib.dhp.schema.oaf.Qualifier> oar = Optional.ofNullable(input.getBestaccessright());
if (oar.isPresent()) {
if (Constants.accessRightsCoarMap.containsKey(oar.get().getClassid())) {
Optional<eu.dnetlib.dhp.schema.oaf.Qualifier> oar = Optional.ofNullable(input.getBestaccessright());
if (oar.isPresent() && Constants.accessRightsCoarMap.containsKey(oar.get().getClassid())) {
String code = Constants.accessRightsCoarMap.get(oar.get().getClassid());
out
.setBestaccessright(
@ -149,226 +71,340 @@ public class ResultMapper implements Serializable {
Constants.coarCodeLabelMap.get(code),
Constants.COAR_ACCESS_RIGHT_SCHEMA));
}
}
final List<String> contributorList = new ArrayList<>();
Optional
.ofNullable(input.getContributor())
.ifPresent(value -> value.stream().forEach(c -> contributorList.add(c.getValue())));
out.setContributor(contributorList);
final List<String> contributorList = new ArrayList<>();
Optional
.ofNullable(input.getContributor())
.ifPresent(value -> value.stream().forEach(c -> contributorList.add(c.getValue())));
out.setContributor(contributorList);
Optional
.ofNullable(input.getCountry())
.ifPresent(
value -> out
.setCountry(
value
.stream()
.map(
c -> {
if (c.getClassid().equals((ModelConstants.UNKNOWN))) {
return null;
}
Country country = new Country();
country.setCode(c.getClassid());
country.setLabel(c.getClassname());
Optional
.ofNullable(c.getDataInfo())
.ifPresent(
provenance -> country
.setProvenance(
Provenance
.newInstance(
provenance
.getProvenanceaction()
.getClassname(),
c.getDataInfo().getTrust())));
return country;
})
.filter(Objects::nonNull)
.collect(Collectors.toList())));
final List<String> coverageList = new ArrayList<>();
Optional
.ofNullable(input.getCoverage())
.ifPresent(value -> value.stream().forEach(c -> coverageList.add(c.getValue())));
out.setCoverage(coverageList);
out.setDateofcollection(input.getDateofcollection());
final List<String> descriptionList = new ArrayList<>();
Optional
.ofNullable(input.getDescription())
.ifPresent(value -> value.forEach(d -> descriptionList.add(d.getValue())));
out.setDescription(descriptionList);
Optional<Field<String>> oStr = Optional.ofNullable(input.getEmbargoenddate());
if (oStr.isPresent()) {
out.setEmbargoenddate(oStr.get().getValue());
}
final List<String> formatList = new ArrayList<>();
Optional
.ofNullable(input.getFormat())
.ifPresent(value -> value.stream().forEach(f -> formatList.add(f.getValue())));
out.setFormat(formatList);
out.setId(input.getId());
out.setOriginalId(input.getOriginalId());
Optional<List<eu.dnetlib.dhp.schema.oaf.Instance>> oInst = Optional
.ofNullable(input.getInstance());
if (oInst.isPresent()) {
if (Constants.DUMPTYPE.COMPLETE.getType().equals(dumpType)) {
((GraphResult) out)
.setInstance(
oInst.get().stream().map(ResultMapper::getGraphInstance).collect(Collectors.toList()));
} else {
((CommunityResult) out)
.setInstance(
oInst.get().stream().map(ResultMapper::getCommunityInstance).collect(Collectors.toList()));
}
}
Optional<eu.dnetlib.dhp.schema.oaf.Qualifier> oL = Optional.ofNullable(input.getLanguage());
if (oL.isPresent()) {
eu.dnetlib.dhp.schema.oaf.Qualifier language = oL.get();
out.setLanguage(Qualifier.newInstance(language.getClassid(), language.getClassname()));
}
Optional<Long> oLong = Optional.ofNullable(input.getLastupdatetimestamp());
if (oLong.isPresent()) {
out.setLastupdatetimestamp(oLong.get());
}
Optional<List<StructuredProperty>> otitle = Optional.ofNullable(input.getTitle());
if (otitle.isPresent()) {
List<StructuredProperty> iTitle = otitle
.get()
.stream()
.filter(t -> t.getQualifier().getClassid().equalsIgnoreCase("main title"))
.collect(Collectors.toList());
if (iTitle.size() > 0) {
out.setMaintitle(iTitle.get(0).getValue());
}
iTitle = otitle
.get()
.stream()
.filter(t -> t.getQualifier().getClassid().equalsIgnoreCase("subtitle"))
.collect(Collectors.toList());
if (iTitle.size() > 0) {
out.setSubtitle(iTitle.get(0).getValue());
}
}
List<ControlledField> pids = new ArrayList<>();
Optional
.ofNullable(input.getPid())
.ifPresent(
value -> value
.stream()
.forEach(
p -> pids
.add(
ControlledField
.newInstance(p.getQualifier().getClassid(), p.getValue()))));
out.setPid(pids);
oStr = Optional.ofNullable(input.getDateofacceptance());
if (oStr.isPresent()) {
out.setPublicationdate(oStr.get().getValue());
}
oStr = Optional.ofNullable(input.getPublisher());
if (oStr.isPresent()) {
out.setPublisher(oStr.get().getValue());
}
List<String> sourceList = new ArrayList<>();
Optional
.ofNullable(input.getSource())
.ifPresent(value -> value.stream().forEach(s -> sourceList.add(s.getValue())));
// out.setSource(input.getSource().stream().map(s -> s.getValue()).collect(Collectors.toList()));
List<Subject> subjectList = new ArrayList<>();
Optional
.ofNullable(input.getSubject())
.ifPresent(
value -> value
.forEach(s -> subjectList.add(getSubject(s))));
out.setSubjects(subjectList);
out.setType(input.getResulttype().getClassid());
}
if (!Constants.DUMPTYPE.COMPLETE.getType().equals(dumpType)) {
((CommunityResult) out)
.setCollectedfrom(
input
.getCollectedfrom()
.stream()
.map(cf -> KeyValue.newInstance(cf.getKey(), cf.getValue()))
.collect(Collectors.toList()));
Set<String> communities = communityMap.keySet();
List<Context> contextList = Optional
.ofNullable(
input
.getContext())
.map(
value -> value
.stream()
.map(c -> {
String community_id = c.getId();
if (community_id.indexOf("::") > 0) {
community_id = community_id.substring(0, community_id.indexOf("::"));
}
if (communities.contains(community_id)) {
Context context = new Context();
context.setCode(community_id);
context.setLabel(communityMap.get(community_id));
Optional<List<DataInfo>> dataInfo = Optional.ofNullable(c.getDataInfo());
if (dataInfo.isPresent()) {
List<Provenance> provenance = new ArrayList<>();
provenance
.addAll(
dataInfo
.get()
.stream()
.map(
di -> Optional
.ofNullable(di.getProvenanceaction())
.map(
provenanceaction -> Provenance
Optional
.ofNullable(input.getCountry())
.ifPresent(
value -> out
.setCountry(
value
.stream()
.map(
c -> {
if (c.getClassid().equals((ModelConstants.UNKNOWN))) {
return null;
}
Country country = new Country();
country.setCode(c.getClassid());
country.setLabel(c.getClassname());
Optional
.ofNullable(c.getDataInfo())
.ifPresent(
provenance -> country
.setProvenance(
Provenance
.newInstance(
provenanceaction.getClassname(), di.getTrust()))
.orElse(null))
.filter(Objects::nonNull)
.collect(Collectors.toSet()));
provenance
.getProvenanceaction()
.getClassname(),
c.getDataInfo().getTrust())));
return country;
})
.filter(Objects::nonNull)
.collect(Collectors.toList())));
context.setProvenance(getUniqueProvenance(provenance));
}
return context;
}
return null;
})
.filter(Objects::nonNull)
.collect(Collectors.toList()))
.orElse(new ArrayList<>());
final List<String> coverageList = new ArrayList<>();
Optional
.ofNullable(input.getCoverage())
.ifPresent(value -> value.stream().forEach(c -> coverageList.add(c.getValue())));
out.setCoverage(coverageList);
if (contextList.size() > 0) {
Set<Integer> hashValue = new HashSet<>();
List<Context> remainigContext = new ArrayList<>();
contextList.forEach(c -> {
if (!hashValue.contains(c.hashCode())) {
remainigContext.add(c);
hashValue.add(c.hashCode());
out.setDateofcollection(input.getDateofcollection());
final List<String> descriptionList = new ArrayList<>();
Optional
.ofNullable(input.getDescription())
.ifPresent(value -> value.forEach(d -> descriptionList.add(d.getValue())));
out.setDescription(descriptionList);
Optional<Field<String>> oStr = Optional.ofNullable(input.getEmbargoenddate());
if (oStr.isPresent()) {
out.setEmbargoenddate(oStr.get().getValue());
}
final List<String> formatList = new ArrayList<>();
Optional
.ofNullable(input.getFormat())
.ifPresent(value -> value.stream().forEach(f -> formatList.add(f.getValue())));
out.setFormat(formatList);
out.setId(input.getId());
out.setOriginalId(input.getOriginalId());
Optional<List<eu.dnetlib.dhp.schema.oaf.Instance>> oInst = Optional
.ofNullable(input.getInstance());
if (oInst.isPresent()) {
if (Constants.DUMPTYPE.COMPLETE.getType().equals(dumpType)) {
((GraphResult) out)
.setInstance(
oInst.get().stream().map(ResultMapper::getGraphInstance).collect(Collectors.toList()));
} else {
((CommunityResult) out)
.setInstance(
oInst
.get()
.stream()
.map(ResultMapper::getCommunityInstance)
.collect(Collectors.toList()));
}
});
((CommunityResult) out).setContext(remainigContext);
}
Optional<eu.dnetlib.dhp.schema.oaf.Qualifier> oL = Optional.ofNullable(input.getLanguage());
if (oL.isPresent()) {
eu.dnetlib.dhp.schema.oaf.Qualifier language = oL.get();
out.setLanguage(Qualifier.newInstance(language.getClassid(), language.getClassname()));
}
Optional<Long> oLong = Optional.ofNullable(input.getLastupdatetimestamp());
if (oLong.isPresent()) {
out.setLastupdatetimestamp(oLong.get());
}
Optional<List<StructuredProperty>> otitle = Optional.ofNullable(input.getTitle());
if (otitle.isPresent()) {
List<StructuredProperty> iTitle = otitle
.get()
.stream()
.filter(t -> t.getQualifier().getClassid().equalsIgnoreCase("main title"))
.collect(Collectors.toList());
if (!iTitle.isEmpty()) {
out.setMaintitle(iTitle.get(0).getValue());
}
iTitle = otitle
.get()
.stream()
.filter(t -> t.getQualifier().getClassid().equalsIgnoreCase("subtitle"))
.collect(Collectors.toList());
if (!iTitle.isEmpty()) {
out.setSubtitle(iTitle.get(0).getValue());
}
}
Optional
.ofNullable(input.getPid())
.ifPresent(
value -> out
.setPid(
value
.stream()
.map(
p -> ControlledField
.newInstance(p.getQualifier().getClassid(), p.getValue()))
.collect(Collectors.toList())));
oStr = Optional.ofNullable(input.getDateofacceptance());
if (oStr.isPresent()) {
out.setPublicationdate(oStr.get().getValue());
}
oStr = Optional.ofNullable(input.getPublisher());
if (oStr.isPresent()) {
out.setPublisher(oStr.get().getValue());
}
Optional
.ofNullable(input.getSource())
.ifPresent(
value -> out.setSource(value.stream().map(Field::getValue).collect(Collectors.toList())));
List<Subject> subjectList = new ArrayList<>();
Optional
.ofNullable(input.getSubject())
.ifPresent(
value -> value
.forEach(s -> subjectList.add(getSubject(s))));
out.setSubjects(subjectList);
out.setType(input.getResulttype().getClassid());
if (!Constants.DUMPTYPE.COMPLETE.getType().equals(dumpType)) {
((CommunityResult) out)
.setCollectedfrom(
input
.getCollectedfrom()
.stream()
.map(cf -> KeyValue.newInstance(cf.getKey(), cf.getValue()))
.collect(Collectors.toList()));
Set<String> communities = communityMap.keySet();
List<Context> contextList = Optional
.ofNullable(
input
.getContext())
.map(
value -> value
.stream()
.map(c -> {
String communityId = c.getId();
if (communityId.contains("::")) {
communityId = communityId.substring(0, communityId.indexOf("::"));
}
if (communities.contains(communityId)) {
Context context = new Context();
context.setCode(communityId);
context.setLabel(communityMap.get(communityId));
Optional<List<DataInfo>> dataInfo = Optional.ofNullable(c.getDataInfo());
if (dataInfo.isPresent()) {
List<Provenance> provenance = new ArrayList<>();
provenance
.addAll(
dataInfo
.get()
.stream()
.map(
di -> Optional
.ofNullable(di.getProvenanceaction())
.map(
provenanceaction -> Provenance
.newInstance(
provenanceaction.getClassname(),
di.getTrust()))
.orElse(null))
.filter(Objects::nonNull)
.collect(Collectors.toSet()));
try {
context.setProvenance(getUniqueProvenance(provenance));
} catch (NoAvailableEntityTypeException e) {
e.printStackTrace();
}
}
return context;
}
return null;
})
.filter(Objects::nonNull)
.collect(Collectors.toList()))
.orElse(new ArrayList<>());
if (!contextList.isEmpty()) {
Set<Integer> hashValue = new HashSet<>();
List<Context> remainigContext = new ArrayList<>();
contextList.forEach(c -> {
if (!hashValue.contains(c.hashCode())) {
remainigContext.add(c);
hashValue.add(c.hashCode());
}
});
((CommunityResult) out).setContext(remainigContext);
}
}
} catch (ClassCastException cce) {
return out;
}
}
return out;
}
private static void addTypeSpecificInformation(Result out, eu.dnetlib.dhp.schema.oaf.Result input,
Optional<eu.dnetlib.dhp.schema.oaf.Qualifier> ort) throws NoAvailableEntityTypeException {
switch (ort.get().getClassid()) {
case "publication":
Optional<Journal> journal = Optional
.ofNullable(((Publication) input).getJournal());
if (journal.isPresent()) {
Journal j = journal.get();
Container c = new Container();
c.setConferencedate(j.getConferencedate());
c.setConferenceplace(j.getConferenceplace());
c.setEdition(j.getEdition());
c.setEp(j.getEp());
c.setIss(j.getIss());
c.setIssnLinking(j.getIssnLinking());
c.setIssnOnline(j.getIssnOnline());
c.setIssnPrinted(j.getIssnPrinted());
c.setName(j.getName());
c.setSp(j.getSp());
c.setVol(j.getVol());
out.setContainer(c);
out.setType(ModelConstants.PUBLICATION_DEFAULT_RESULTTYPE.getClassname());
}
break;
case "dataset":
Dataset id = (Dataset) input;
Optional.ofNullable(id.getSize()).ifPresent(v -> out.setSize(v.getValue()));
Optional.ofNullable(id.getVersion()).ifPresent(v -> out.setVersion(v.getValue()));
out
.setGeolocation(
Optional
.ofNullable(id.getGeolocation())
.map(
igl -> igl
.stream()
.filter(Objects::nonNull)
.map(gli -> {
GeoLocation gl = new GeoLocation();
gl.setBox(gli.getBox());
gl.setPlace(gli.getPlace());
gl.setPoint(gli.getPoint());
return gl;
})
.collect(Collectors.toList()))
.orElse(null));
out.setType(ModelConstants.DATASET_DEFAULT_RESULTTYPE.getClassname());
break;
case "software":
Software is = (Software) input;
Optional
.ofNullable(is.getCodeRepositoryUrl())
.ifPresent(value -> out.setCodeRepositoryUrl(value.getValue()));
Optional
.ofNullable(is.getDocumentationUrl())
.ifPresent(
value -> out
.setDocumentationUrl(
value
.stream()
.map(Field::getValue)
.collect(Collectors.toList())));
Optional
.ofNullable(is.getProgrammingLanguage())
.ifPresent(value -> out.setProgrammingLanguage(value.getClassid()));
out.setType(ModelConstants.SOFTWARE_DEFAULT_RESULTTYPE.getClassname());
break;
case "other":
OtherResearchProduct ir = (OtherResearchProduct) input;
out
.setContactgroup(
Optional
.ofNullable(ir.getContactgroup())
.map(value -> value.stream().map(Field::getValue).collect(Collectors.toList()))
.orElse(null));
out
.setContactperson(
Optional
.ofNullable(ir.getContactperson())
.map(value -> value.stream().map(Field::getValue).collect(Collectors.toList()))
.orElse(null));
out
.setTool(
Optional
.ofNullable(ir.getTool())
.map(value -> value.stream().map(Field::getValue).collect(Collectors.toList()))
.orElse(null));
out.setType(ModelConstants.ORP_DEFAULT_RESULTTYPE.getClassname());
break;
default:
throw new NoAvailableEntityTypeException();
}
}
private static Instance getGraphInstance(eu.dnetlib.dhp.schema.oaf.Instance i) {
Instance instance = new Instance();
@ -397,21 +433,58 @@ public class ResultMapper implements Serializable {
}
private static <I extends Instance> void setCommonValue(eu.dnetlib.dhp.schema.oaf.Instance i, I instance) {
Optional<eu.dnetlib.dhp.schema.oaf.Qualifier> opAr = Optional
.ofNullable(i.getAccessright());
if (opAr.isPresent()) {
if (Constants.accessRightsCoarMap.containsKey(opAr.get().getClassid())) {
String code = Constants.accessRightsCoarMap.get(opAr.get().getClassid());
instance
.setAccessright(
AccessRight
.newInstance(
code,
Constants.coarCodeLabelMap.get(code),
Constants.COAR_ACCESS_RIGHT_SCHEMA));
Optional<eu.dnetlib.dhp.schema.oaf.AccessRight> opAr = Optional.ofNullable(i.getAccessright());
if (opAr.isPresent() && Constants.accessRightsCoarMap.containsKey(opAr.get().getClassid())) {
String code = Constants.accessRightsCoarMap.get(opAr.get().getClassid());
instance
.setAccessright(
AccessRight
.newInstance(
code,
Constants.coarCodeLabelMap.get(code),
Constants.COAR_ACCESS_RIGHT_SCHEMA));
if (opAr.get().getOpenAccessRoute() != null) {
switch (opAr.get().getOpenAccessRoute()) {
case hybrid:
instance.getAccessright().setOpenAccessRoute(OpenAccessRoute.hybrid);
break;
case gold:
instance.getAccessright().setOpenAccessRoute(OpenAccessRoute.gold);
break;
case green:
instance.getAccessright().setOpenAccessRoute(OpenAccessRoute.green);
break;
case bronze:
instance.getAccessright().setOpenAccessRoute(OpenAccessRoute.bronze);
break;
}
}
}
Optional
.ofNullable(i.getPid())
.ifPresent(
pid -> instance
.setPid(
pid
.stream()
.map(p -> ControlledField.newInstance(p.getQualifier().getClassid(), p.getValue()))
.collect(Collectors.toList())));
Optional
.ofNullable(i.getAlternateIdentifier())
.ifPresent(
ai -> instance
.setAlternateIdentifier(
ai
.stream()
.map(p -> ControlledField.newInstance(p.getQualifier().getClassid(), p.getValue()))
.collect(Collectors.toList())));
Optional
.ofNullable(i.getLicense())
.ifPresent(value -> instance.setLicense(value.getValue()));
@ -424,11 +497,26 @@ public class ResultMapper implements Serializable {
Optional
.ofNullable(i.getInstancetype())
.ifPresent(value -> instance.setType(value.getClassname()));
Optional.ofNullable(i.getUrl()).ifPresent(value -> instance.setUrl(value));
Optional<Field<String>> oPca = Optional.ofNullable(i.getProcessingchargeamount());
Optional<Field<String>> oPcc = Optional.ofNullable(i.getProcessingchargecurrency());
if (oPca.isPresent() && oPcc.isPresent()) {
Field<String> pca = oPca.get();
Field<String> pcc = oPcc.get();
if (!pca.getValue().trim().equals("") && !pcc.getValue().trim().equals("")) {
APC apc = new APC();
apc.setCurrency(oPcc.get().getValue());
apc.setAmount(oPca.get().getValue());
instance.setArticleprocessingcharge(apc);
}
}
Optional.ofNullable(i.getUrl()).ifPresent(instance::setUrl);
}
private static List<Provenance> getUniqueProvenance(List<Provenance> provenance) {
private static List<Provenance> getUniqueProvenance(List<Provenance> provenance)
throws NoAvailableEntityTypeException {
Provenance iProv = new Provenance();
Provenance hProv = new Provenance();
@ -450,6 +538,8 @@ public class ResultMapper implements Serializable {
case Constants.USER_CLAIM:
lProv = getHighestTrust(lProv, p);
break;
default:
throw new NoAvailableEntityTypeException();
}
}
@ -503,34 +593,67 @@ public class ResultMapper implements Serializable {
return a;
}
private static Pid getOrcid(List<StructuredProperty> p) {
for (StructuredProperty pid : p) {
if (pid.getQualifier().getClassid().equals(ModelConstants.ORCID)) {
Optional<DataInfo> di = Optional.ofNullable(pid.getDataInfo());
if (di.isPresent()) {
return Pid
private static Pid getAuthorPid(StructuredProperty pid) {
Optional<DataInfo> di = Optional.ofNullable(pid.getDataInfo());
if (di.isPresent()) {
return Pid
.newInstance(
ControlledField
.newInstance(
ControlledField
.newInstance(
pid.getQualifier().getClassid(),
pid.getValue()),
Provenance
.newInstance(
di.get().getProvenanceaction().getClassname(),
di.get().getTrust()));
} else {
return Pid
pid.getQualifier().getClassid(),
pid.getValue()),
Provenance
.newInstance(
ControlledField
.newInstance(
pid.getQualifier().getClassid(),
pid.getValue())
di.get().getProvenanceaction().getClassname(),
di.get().getTrust()));
} else {
return Pid
.newInstance(
ControlledField
.newInstance(
pid.getQualifier().getClassid(),
pid.getValue())
);
}
}
);
}
}
private static Pid getOrcid(List<StructuredProperty> p) {
List<StructuredProperty> pidList = p.stream().map(pid -> {
if (pid.getQualifier().getClassid().equals(ModelConstants.ORCID) ||
(pid.getQualifier().getClassid().equals(ModelConstants.ORCID_PENDING))) {
return pid;
}
return null;
}).filter(Objects::nonNull).collect(Collectors.toList());
if (pidList.size() == 1) {
return getAuthorPid(pidList.get(0));
}
List<StructuredProperty> orcid = pidList
.stream()
.filter(
ap -> ap
.getQualifier()
.getClassid()
.equals(ModelConstants.ORCID))
.collect(Collectors.toList());
if (orcid.size() == 1) {
return getAuthorPid(orcid.get(0));
}
orcid = pidList
.stream()
.filter(
ap -> ap
.getQualifier()
.getClassid()
.equals(ModelConstants.ORCID_PENDING))
.collect(Collectors.toList());
if (orcid.size() == 1) {
return getAuthorPid(orcid.get(0));
}
return null;
}

View File

@ -6,6 +6,7 @@ import java.io.IOException;
import java.io.OutputStreamWriter;
import java.io.Serializable;
import java.nio.charset.StandardCharsets;
import java.util.Optional;
import org.apache.commons.io.IOUtils;
import org.apache.hadoop.conf.Configuration;
@ -18,6 +19,7 @@ import org.slf4j.LoggerFactory;
import org.xml.sax.SAXException;
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
import eu.dnetlib.dhp.oa.graph.dump.community.CommunityMap;
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException;
/**
@ -30,9 +32,9 @@ import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException;
public class SaveCommunityMap implements Serializable {
private static final Logger log = LoggerFactory.getLogger(SaveCommunityMap.class);
private final QueryInformationSystem queryInformationSystem;
private final transient QueryInformationSystem queryInformationSystem;
private final BufferedWriter writer;
private final transient BufferedWriter writer;
public SaveCommunityMap(String hdfsPath, String hdfsNameNode, String isLookUpUrl) throws IOException {
final Configuration conf = new Configuration();
@ -70,13 +72,28 @@ public class SaveCommunityMap implements Serializable {
final String isLookUpUrl = parser.get("isLookUpUrl");
log.info("isLookUpUrl: {}", isLookUpUrl);
final Boolean singleCommunity = Optional
.ofNullable(parser.get("singleDeposition"))
.map(Boolean::valueOf)
.orElse(false);
final String community_id = Optional.ofNullable(parser.get("communityId")).orElse(null);
final SaveCommunityMap scm = new SaveCommunityMap(outputPath, nameNode, isLookUpUrl);
scm.saveCommunityMap();
scm.saveCommunityMap(singleCommunity, community_id);
}
private void saveCommunityMap() throws ISLookUpException, IOException, DocumentException, SAXException {
writer.write(Utils.OBJECT_MAPPER.writeValueAsString(queryInformationSystem.getCommunityMap()));
private void saveCommunityMap(boolean singleCommunity, String communityId)
throws ISLookUpException, IOException, DocumentException, SAXException {
final String communityMapString = Utils.OBJECT_MAPPER
.writeValueAsString(queryInformationSystem.getCommunityMap(singleCommunity, communityId));
log.info("communityMap {} ", communityMapString);
writer
.write(
communityMapString);
writer.close();
}
}

View File

@ -5,15 +5,13 @@ import java.io.Serializable;
import java.util.Optional;
import org.apache.commons.io.IOUtils;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.*;
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
import eu.dnetlib.dhp.common.api.MissingConceptDoiException;
import eu.dnetlib.dhp.common.api.ZenodoAPIClient;
import eu.dnetlib.dhp.oa.graph.dump.community.CommunityMap;
import eu.dnetlib.dhp.oa.graph.dump.exceptions.NoAvailableEntityTypeException;
public class SendToZenodoHDFS implements Serializable {
@ -21,8 +19,6 @@ public class SendToZenodoHDFS implements Serializable {
private static final String VERSION = "version"; // to be used to upload a new version of a published deposition
private static final String UPDATE = "update"; // to upload content to an open deposition not published
private static final Log log = LogFactory.getLog(SendToZenodoHDFS.class);
public static void main(final String[] args) throws Exception, MissingConceptDoiException {
final ArgumentApplicationParser parser = new ArgumentApplicationParser(
IOUtils
@ -48,15 +44,12 @@ public class SendToZenodoHDFS implements Serializable {
.orElse(false);
final String depositionId = Optional.ofNullable(parser.get("depositionId")).orElse(null);
final String communityMapPath = parser.get("communityMapPath");
Configuration conf = new Configuration();
conf.set("fs.defaultFS", hdfsNameNode);
FileSystem fileSystem = FileSystem.get(conf);
CommunityMap communityMap = Utils.readCommunityMap(fileSystem, communityMapPath);
RemoteIterator<LocatedFileStatus> fileStatusListIterator = fileSystem
.listFiles(
new Path(hdfsPath), true);
@ -77,19 +70,17 @@ public class SendToZenodoHDFS implements Serializable {
}
zenodoApiClient.uploadOpenDeposition(depositionId);
break;
default:
throw new NoAvailableEntityTypeException();
}
while (fileStatusListIterator.hasNext()) {
LocatedFileStatus fileStatus = fileStatusListIterator.next();
Path p = fileStatus.getPath();
String p_string = p.toString();
if (!p_string.endsWith("_SUCCESS")) {
String name = p_string.substring(p_string.lastIndexOf("/") + 1);
log.info("Sending information for community: " + name);
if (communityMap.containsKey(name.substring(0, name.lastIndexOf(".")))) {
name = communityMap.get(name.substring(0, name.lastIndexOf("."))).replace(" ", "_") + ".tar";
}
String pString = p.toString();
if (!pString.endsWith("_SUCCESS")) {
String name = pString.substring(pString.lastIndexOf("/") + 1);
FSDataInputStream inputStream = fileSystem.open(p);
zenodoApiClient.uploadIS(inputStream, name, fileStatus.getLen());
@ -101,7 +92,7 @@ public class SendToZenodoHDFS implements Serializable {
zenodoApiClient.sendMretadata(metadata);
}
if (publish) {
if (Boolean.TRUE.equals(publish)) {
zenodoApiClient.publish();
}
}

View File

@ -4,9 +4,7 @@ package eu.dnetlib.dhp.oa.graph.dump.community;
import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession;
import java.io.Serializable;
import java.util.NoSuchElementException;
import java.util.Optional;
import java.util.Set;
import java.util.stream.Collectors;
import org.apache.spark.SparkConf;
@ -17,6 +15,7 @@ import org.apache.spark.sql.SparkSession;
import eu.dnetlib.dhp.oa.graph.dump.Utils;
import eu.dnetlib.dhp.schema.dump.oaf.community.CommunityResult;
import eu.dnetlib.dhp.schema.dump.oaf.community.Context;
/**
* This class splits the dumped results according to the research community - research initiative/infrastructure they
@ -35,12 +34,13 @@ public class CommunitySplit implements Serializable {
isSparkSessionManaged,
spark -> {
Utils.removeOutputDir(spark, outputPath);
execSplit(spark, inputPath, outputPath, Utils.getCommunityMap(spark, communityMapPath).keySet());
CommunityMap communityMap = Utils.getCommunityMap(spark, communityMapPath);
execSplit(spark, inputPath, outputPath, communityMap);
});
}
private static void execSplit(SparkSession spark, String inputPath, String outputPath,
Set<String> communities) {
CommunityMap communities) {
Dataset<CommunityResult> result = Utils
.readPath(spark, inputPath + "/publication", CommunityResult.class)
@ -49,34 +49,32 @@ public class CommunitySplit implements Serializable {
.union(Utils.readPath(spark, inputPath + "/software", CommunityResult.class));
communities
.forEach(c -> printResult(c, result, outputPath));
.keySet()
.stream()
.forEach(c -> printResult(c, result, outputPath + "/" + communities.get(c).replace(" ", "_")));
}
private static void printResult(String community, Dataset<CommunityResult> result, String outputPath) {
private static void printResult(String c, Dataset<CommunityResult> result, String outputPath) {
Dataset<CommunityResult> communityProducts = result
.filter((FilterFunction<CommunityResult>) r -> containsCommunity(r, community));
.filter((FilterFunction<CommunityResult>) r -> containsCommunity(r, c));
communityProducts
.write()
.option("compression", "gzip")
.mode(SaveMode.Overwrite)
.json(outputPath);
try {
communityProducts.first();
communityProducts
.write()
.option("compression", "gzip")
.mode(SaveMode.Overwrite)
.json(outputPath + "/" + community);
} catch (NoSuchElementException e) {
// ignoring it on purpose
}
}
private static boolean containsCommunity(CommunityResult r, String community) {
private static boolean containsCommunity(CommunityResult r, String c) {
if (Optional.ofNullable(r.getContext()).isPresent()) {
return !r
return r
.getContext()
.stream()
.filter(con -> con.getCode().equals(community))
.map(Context::getCode)
.collect(Collectors.toList())
.isEmpty();
.contains(c);
}
return false;
}

View File

@ -26,9 +26,11 @@ import org.xml.sax.SAXException;
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
import eu.dnetlib.dhp.oa.graph.dump.Utils;
import eu.dnetlib.dhp.schema.common.ModelConstants;
import eu.dnetlib.dhp.schema.dump.oaf.Provenance;
import eu.dnetlib.dhp.schema.dump.oaf.community.Funder;
import eu.dnetlib.dhp.schema.dump.oaf.community.Project;
import eu.dnetlib.dhp.schema.dump.oaf.community.Validated;
import eu.dnetlib.dhp.schema.oaf.DataInfo;
import eu.dnetlib.dhp.schema.oaf.Field;
import eu.dnetlib.dhp.schema.oaf.Relation;
@ -78,7 +80,9 @@ public class SparkPrepareResultProject implements Serializable {
private static void prepareResultProjectList(SparkSession spark, String inputPath, String outputPath) {
Dataset<Relation> relation = Utils
.readPath(spark, inputPath + "/relation", Relation.class)
.filter("dataInfo.deletedbyinference = false and lower(relClass) = 'isproducedby'");
.filter(
"dataInfo.deletedbyinference = false and lower(relClass) = '"
+ ModelConstants.IS_PRODUCED_BY.toLowerCase() + "'");
Dataset<eu.dnetlib.dhp.schema.oaf.Project> projects = Utils
.readPath(spark, inputPath + "/project", eu.dnetlib.dhp.schema.oaf.Project.class);
@ -98,7 +102,7 @@ public class SparkPrepareResultProject implements Serializable {
rp.setResultId(s);
eu.dnetlib.dhp.schema.oaf.Project p = first._1();
projectSet.add(p.getId());
Project ps = getProject(p);
Project ps = getProject(p, first._2);
List<Project> projList = new ArrayList<>();
projList.add(ps);
@ -107,7 +111,7 @@ public class SparkPrepareResultProject implements Serializable {
eu.dnetlib.dhp.schema.oaf.Project op = c._1();
if (!projectSet.contains(op.getId())) {
projList
.add(getProject(op));
.add(getProject(op, c._2));
projectSet.add(op.getId());
@ -122,7 +126,7 @@ public class SparkPrepareResultProject implements Serializable {
.json(outputPath);
}
private static Project getProject(eu.dnetlib.dhp.schema.oaf.Project op) {
private static Project getProject(eu.dnetlib.dhp.schema.oaf.Project op, Relation relation) {
Project p = Project
.newInstance(
op.getId(),
@ -157,7 +161,9 @@ public class SparkPrepareResultProject implements Serializable {
provenance.setTrust(di.get().getTrust());
p.setProvenance(provenance);
}
if (Boolean.TRUE.equals(relation.getValidated())) {
p.setValidated(Validated.newInstance(relation.getValidated(), relation.getValidationDate()));
}
return p;
}
@ -173,8 +179,8 @@ public class SparkPrepareResultProject implements Serializable {
f.setName(((Node) (doc.selectNodes("//funder/name").get(0))).getText());
f.setJurisdiction(((Node) (doc.selectNodes("//funder/jurisdiction").get(0))).getText());
for (Object o : doc.selectNodes("//funding_level_0")) {
List node = ((Node) o).selectNodes("./name");
f.setFundingStream(((Node) node.get(0)).getText());
List<Node> node = ((Node) o).selectNodes("./name");
f.setFundingStream((node.get(0)).getText());
}
return f;

View File

@ -22,6 +22,7 @@ import org.slf4j.LoggerFactory;
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
import eu.dnetlib.dhp.oa.graph.dump.Utils;
import eu.dnetlib.dhp.oa.graph.dump.exceptions.MyRuntimeException;
import eu.dnetlib.dhp.schema.common.ModelSupport;
import eu.dnetlib.dhp.schema.dump.oaf.graph.*;
import eu.dnetlib.dhp.schema.oaf.Datasource;
@ -38,6 +39,7 @@ public class CreateContextRelation implements Serializable {
private final transient QueryInformationSystem queryInformationSystem;
private static final String CONTEX_RELATION_DATASOURCE = "contentproviders";
private static final String CONTEX_RELATION_PROJECT = "projects";
public static void main(String[] args) throws Exception {
String jsonConfiguration = IOUtils
@ -72,6 +74,10 @@ public class CreateContextRelation implements Serializable {
cce.execute(Process::getRelation, CONTEX_RELATION_DATASOURCE, ModelSupport.getIdPrefix(Datasource.class));
log.info("Creating relations for projects... ");
cce
.execute(
Process::getRelation, CONTEX_RELATION_PROJECT,
ModelSupport.getIdPrefix(eu.dnetlib.dhp.schema.oaf.Project.class));
cce.close();
@ -115,7 +121,7 @@ public class CreateContextRelation implements Serializable {
writer.write(Utils.OBJECT_MAPPER.writeValueAsString(r));
writer.newLine();
} catch (final Exception e) {
throw new RuntimeException(e);
throw new MyRuntimeException(e);
}
}

View File

@ -1,6 +1,7 @@
package eu.dnetlib.dhp.oa.graph.dump.complete;
import static com.jayway.jsonpath.Filter.filter;
import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession;
import java.io.Serializable;
@ -9,7 +10,10 @@ import java.util.*;
import java.util.stream.Collectors;
import org.apache.spark.SparkConf;
import org.apache.spark.api.java.function.FilterFunction;
import org.apache.spark.api.java.function.ForeachFunction;
import org.apache.spark.api.java.function.MapFunction;
import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Encoders;
import org.apache.spark.sql.SaveMode;
import org.apache.spark.sql.SparkSession;
@ -18,6 +22,8 @@ import org.dom4j.DocumentException;
import org.dom4j.Node;
import org.dom4j.io.SAXReader;
import com.fasterxml.jackson.databind.ObjectMapper;
import eu.dnetlib.dhp.oa.graph.dump.DumpProducts;
import eu.dnetlib.dhp.oa.graph.dump.Utils;
import eu.dnetlib.dhp.schema.common.ModelSupport;
@ -453,13 +459,17 @@ public class DumpGraphEntities implements Serializable {
.map(
(MapFunction<E, Organization>) o -> mapOrganization((eu.dnetlib.dhp.schema.oaf.Organization) o),
Encoders.bean(Organization.class))
.filter((FilterFunction<Organization>) o -> o != null)
.write()
.mode(SaveMode.Overwrite)
.option("compression", "gzip")
.json(outputPath);
}
private static Organization mapOrganization(eu.dnetlib.dhp.schema.oaf.Organization org) {
private static eu.dnetlib.dhp.schema.dump.oaf.graph.Organization mapOrganization(
eu.dnetlib.dhp.schema.oaf.Organization org) {
if (org.getDataInfo().getDeletedbyinference())
return null;
Organization organization = new Organization();
Optional

View File

@ -62,22 +62,7 @@ public class Extractor implements Serializable {
.readPath(spark, inputPath, inputClazz)
.flatMap((FlatMapFunction<R, Relation>) value -> {
List<Relation> relationList = new ArrayList<>();
Optional
.ofNullable(value.getInstance())
.ifPresent(inst -> inst.forEach(instance -> {
Optional
.ofNullable(instance.getCollectedfrom())
.ifPresent(
cf -> getRelatioPair(
value, relationList, cf,
ModelConstants.IS_PROVIDED_BY, ModelConstants.PROVIDES, hashCodes));
Optional
.ofNullable(instance.getHostedby())
.ifPresent(
hb -> getRelatioPair(
value, relationList, hb,
Constants.IS_HOSTED_BY, Constants.HOSTS, hashCodes));
}));
extractRelationsFromInstance(hashCodes, value, relationList);
Set<String> communities = communityMap.keySet();
Optional
.ofNullable(value.getContext())
@ -136,8 +121,28 @@ public class Extractor implements Serializable {
}
private <R extends Result> void extractRelationsFromInstance(Set<Integer> hashCodes, R value,
List<Relation> relationList) {
Optional
.ofNullable(value.getInstance())
.ifPresent(inst -> inst.forEach(instance -> {
Optional
.ofNullable(instance.getCollectedfrom())
.ifPresent(
cf -> getRelatioPair(
value, relationList, cf,
ModelConstants.IS_PROVIDED_BY, ModelConstants.PROVIDES, hashCodes));
Optional
.ofNullable(instance.getHostedby())
.ifPresent(
hb -> getRelatioPair(
value, relationList, hb,
Constants.IS_HOSTED_BY, Constants.HOSTS, hashCodes));
}));
}
private static <R extends Result> void getRelatioPair(R value, List<Relation> relationList, KeyValue cf,
String result_dtasource, String datasource_result,
String resultDatasource, String datasourceResult,
Set<Integer> hashCodes) {
Provenance provenance = Optional
.ofNullable(cf.getDataInfo())
@ -147,7 +152,7 @@ public class Extractor implements Serializable {
.map(
paction -> Provenance
.newInstance(
paction.getClassid(),
paction.getClassname(),
dinfo.getTrust()))
.orElse(
Provenance
@ -162,7 +167,7 @@ public class Extractor implements Serializable {
Relation r = getRelation(
value.getId(),
cf.getKey(), Constants.RESULT_ENTITY, Constants.DATASOURCE_ENTITY,
result_dtasource, ModelConstants.PROVISION,
resultDatasource, ModelConstants.PROVISION,
provenance);
if (!hashCodes.contains(r.hashCode())) {
relationList
@ -173,7 +178,7 @@ public class Extractor implements Serializable {
r = getRelation(
cf.getKey(), value.getId(),
Constants.DATASOURCE_ENTITY, Constants.RESULT_ENTITY,
datasource_result, ModelConstants.PROVISION,
datasourceResult, ModelConstants.PROVISION,
provenance);
if (!hashCodes.contains(r.hashCode())) {

View File

@ -9,6 +9,7 @@ import org.apache.commons.lang3.StringUtils;
import eu.dnetlib.dhp.oa.graph.dump.Constants;
import eu.dnetlib.dhp.oa.graph.dump.Utils;
import eu.dnetlib.dhp.oa.graph.dump.exceptions.MyRuntimeException;
import eu.dnetlib.dhp.schema.common.ModelConstants;
import eu.dnetlib.dhp.schema.common.ModelSupport;
import eu.dnetlib.dhp.schema.dump.oaf.Provenance;
@ -43,7 +44,7 @@ public class Process implements Serializable {
return (R) ri;
} catch (final Exception e) {
throw new RuntimeException(e);
throw new MyRuntimeException(e);
}
}
@ -91,7 +92,7 @@ public class Process implements Serializable {
return relationList;
} catch (final Exception e) {
throw new RuntimeException(e);
throw new MyRuntimeException(e);
}
}

View File

@ -13,6 +13,8 @@ import org.dom4j.io.SAXReader;
import org.jetbrains.annotations.NotNull;
import org.xml.sax.SAXException;
import eu.dnetlib.dhp.schema.common.ModelSupport;
import eu.dnetlib.dhp.utils.DHPUtils;
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException;
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService;
@ -91,9 +93,9 @@ public class QueryInformationSystem {
Element root = doc.getRootElement();
cinfo.setId(root.attributeValue("id"));
Iterator it = root.elementIterator();
Iterator<Element> it = root.elementIterator();
while (it.hasNext()) {
Element el = (Element) it.next();
Element el = it.next();
if (el.getName().equals("category")) {
String categoryId = el.attributeValue("id");
categoryId = categoryId.substring(categoryId.lastIndexOf("::") + 2);
@ -115,14 +117,79 @@ public class QueryInformationSystem {
@NotNull
private List<String> getCategoryList(Element el, String prefix) {
List<String> datasourceList = new ArrayList<>();
for (Object node : el.selectNodes(".//param")) {
Node n = (Node) node;
if (n.valueOf("./@name").equals("openaireId")) {
datasourceList.add(prefix + "|" + n.getText());
}
for (Object node : el.selectNodes(".//concept")) {
String oid = getOpenaireId((Node) node, prefix);
if (oid != null)
datasourceList.add(oid);
}
return datasourceList;
}
private String getOpenaireId(Node el, String prefix) {
for (Object node : el.selectNodes(".//param")) {
Node n = (Node) node;
if (n.valueOf("./@name").equals("openaireId")) {
return prefix + "|" + n.getText();
}
}
return makeOpenaireId(el, prefix);
}
private String makeOpenaireId(Node el, String prefix) {
if (!prefix.equals(ModelSupport.entityIdPrefix.get("project"))) {
return null;
}
String funder = "";
String grantId = null;
String funding = null;
for (Object node : el.selectNodes(".//param")) {
Node n = (Node) node;
switch (n.valueOf("./@name")) {
case "funding":
funding = n.getText();
break;
case "funder":
funder = n.getText();
break;
case "CD_PROJECT_NUMBER":
grantId = n.getText();
break;
default:
break;
}
}
String nsp = null;
switch (funder.toLowerCase()) {
case "ec":
if (funding == null) {
return null;
}
if (funding.toLowerCase().contains("h2020")) {
nsp = "corda__h2020::";
} else {
nsp = "corda_______::";
}
break;
case "tubitak":
nsp = "tubitakf____::";
break;
case "dfg":
nsp = "dfgf________::";
break;
default:
StringBuilder bld = new StringBuilder();
bld.append(funder.toLowerCase());
for (int i = funder.length(); i < 12; i++)
bld.append("_");
bld.append("::");
nsp = bld.toString();
}
return prefix + "|" + nsp + DHPUtils.md5(grantId);
}
}

View File

@ -71,22 +71,22 @@ public class SparkDumpRelationJob implements Serializable {
Dataset<Relation> relations = Utils.readPath(spark, inputPath, Relation.class);
relations
.map((MapFunction<Relation, eu.dnetlib.dhp.schema.dump.oaf.graph.Relation>) relation -> {
eu.dnetlib.dhp.schema.dump.oaf.graph.Relation rel_new = new eu.dnetlib.dhp.schema.dump.oaf.graph.Relation();
rel_new
eu.dnetlib.dhp.schema.dump.oaf.graph.Relation relNew = new eu.dnetlib.dhp.schema.dump.oaf.graph.Relation();
relNew
.setSource(
Node
.newInstance(
relation.getSource(),
ModelSupport.idPrefixEntity.get(relation.getSource().substring(0, 2))));
rel_new
relNew
.setTarget(
Node
.newInstance(
relation.getTarget(),
ModelSupport.idPrefixEntity.get(relation.getTarget().substring(0, 2))));
rel_new
relNew
.setReltype(
RelType
.newInstance(
@ -96,26 +96,22 @@ public class SparkDumpRelationJob implements Serializable {
Optional<DataInfo> odInfo = Optional.ofNullable(relation.getDataInfo());
if (odInfo.isPresent()) {
DataInfo dInfo = odInfo.get();
if (Optional.ofNullable(dInfo.getProvenanceaction()).isPresent()) {
if (Optional.ofNullable(dInfo.getProvenanceaction().getClassname()).isPresent()) {
rel_new
.setProvenance(
Provenance
.newInstance(
dInfo.getProvenanceaction().getClassname(),
dInfo.getTrust()));
}
if (Optional.ofNullable(dInfo.getProvenanceaction()).isPresent() &&
Optional.ofNullable(dInfo.getProvenanceaction().getClassname()).isPresent()) {
relNew
.setProvenance(
Provenance
.newInstance(
dInfo.getProvenanceaction().getClassname(),
dInfo.getTrust()));
}
}
// Optional
// .ofNullable(relation.getDataInfo())
// .ifPresent(
// datainfo -> rel_new
// .setProvenance(
// Provenance
// .newInstance(datainfo.getProvenanceaction().getClassname(), datainfo.getTrust())));
if (Boolean.TRUE.equals(relation.getValidated())) {
relNew.setValidated(relation.getValidated());
relNew.setValidationDate(relation.getValidationDate());
}
return rel_new;
return relNew;
}, Encoders.bean(eu.dnetlib.dhp.schema.dump.oaf.graph.Relation.class))
.write()

View File

@ -58,7 +58,8 @@ public class SparkOrganizationRelation implements Serializable {
final OrganizationMap organizationMap = new Gson()
.fromJson(parser.get("organizationCommunityMap"), OrganizationMap.class);
log.info("organization map : {}", new Gson().toJson(organizationMap));
final String serializedOrganizationMap = new Gson().toJson(organizationMap);
log.info("organization map : {}", serializedOrganizationMap);
final String communityMapPath = parser.get("communityMapPath");
log.info("communityMapPath: {}", communityMapPath);

View File

@ -0,0 +1,136 @@
package eu.dnetlib.dhp.oa.graph.dump.complete;
import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession;
import java.io.Serializable;
import java.util.Optional;
import org.apache.commons.io.IOUtils;
import org.apache.spark.SparkConf;
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.slf4j.Logger;
import org.slf4j.LoggerFactory;
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
import eu.dnetlib.dhp.oa.graph.dump.Utils;
import eu.dnetlib.dhp.schema.oaf.*;
/**
* It selects the valid relations among those present in the graph. One relation is valid if it is not deletedbyinference
* and if both the source and the target node are present in the graph and are not deleted by inference nor invisible.
* To check this I made a view of the ids of all the entities in the graph, and select the relations for which a join exists
* with this view for both the source and the target
*/
public class SparkSelectValidRelationsJob implements Serializable {
private static final Logger log = LoggerFactory.getLogger(SparkSelectValidRelationsJob.class);
public static void main(String[] args) throws Exception {
String jsonConfiguration = IOUtils
.toString(
SparkSelectValidRelationsJob.class
.getResourceAsStream(
"/eu/dnetlib/dhp/oa/graph/dump/complete/input_relationdump_parameters.json"));
final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration);
parser.parseArgument(args);
Boolean isSparkSessionManaged = Optional
.ofNullable(parser.get("isSparkSessionManaged"))
.map(Boolean::valueOf)
.orElse(Boolean.TRUE);
log.info("isSparkSessionManaged: {}", isSparkSessionManaged);
final String inputPath = parser.get("sourcePath");
log.info("inputPath: {}", inputPath);
final String outputPath = parser.get("outputPath");
log.info("outputPath: {}", outputPath);
SparkConf conf = new SparkConf();
runWithSparkSession(
conf,
isSparkSessionManaged,
spark -> {
Utils.removeOutputDir(spark, outputPath);
selectValidRelation(spark, inputPath, outputPath);
});
}
private static void selectValidRelation(SparkSession spark, String inputPath, String outputPath) {
Dataset<Relation> relation = Utils.readPath(spark, inputPath + "/relation", Relation.class);
Dataset<Publication> publication = Utils.readPath(spark, inputPath + "/publication", Publication.class);
Dataset<eu.dnetlib.dhp.schema.oaf.Dataset> dataset = Utils
.readPath(spark, inputPath + "/dataset", eu.dnetlib.dhp.schema.oaf.Dataset.class);
Dataset<Software> software = Utils.readPath(spark, inputPath + "/software", Software.class);
Dataset<OtherResearchProduct> other = Utils
.readPath(spark, inputPath + "/otherresearchproduct", OtherResearchProduct.class);
Dataset<Organization> organization = Utils.readPath(spark, inputPath + "/organization", Organization.class);
Dataset<Project> project = Utils.readPath(spark, inputPath + "/project", Project.class);
Dataset<Datasource> datasource = Utils.readPath(spark, inputPath + "/datasource", Datasource.class);
relation.createOrReplaceTempView("relation");
publication.createOrReplaceTempView("publication");
dataset.createOrReplaceTempView("dataset");
other.createOrReplaceTempView("other");
software.createOrReplaceTempView("software");
organization.createOrReplaceTempView("organization");
project.createOrReplaceTempView("project");
datasource.createOrReplaceTempView("datasource");
spark
.sql(
"SELECT id " +
"FROM publication " +
"WHERE datainfo.deletedbyinference = false AND datainfo.invisible = false " +
"UNION ALL " +
"SELECT id " +
"FROM dataset " +
"WHERE datainfo.deletedbyinference = false AND datainfo.invisible = false " +
"UNION ALL " +
"SELECT id " +
"FROM other " +
"WHERE datainfo.deletedbyinference = false AND datainfo.invisible = false " +
"UNION ALL " +
"SELECT id " +
"FROM software " +
"WHERE datainfo.deletedbyinference = false AND datainfo.invisible = false " +
"UNION ALL " +
"SELECT id " +
"FROM organization " +
"WHERE datainfo.deletedbyinference = false AND datainfo.invisible = false " +
"UNION ALL " +
"SELECT id " +
"FROM project " +
"WHERE datainfo.deletedbyinference = false AND datainfo.invisible = false " +
"UNION ALL " +
"SELECT id " +
"FROM datasource " +
"WHERE datainfo.deletedbyinference = false AND datainfo.invisible = false ")
.createOrReplaceTempView("identifiers");
spark
.sql(
"SELECT relation.* " +
"FROM relation " +
"JOIN identifiers i1 " +
"ON source = i1.id " +
"JOIN identifiers i2 " +
"ON target = i2.id " +
"WHERE datainfo.deletedbyinference = false")
.as(Encoders.bean(Relation.class))
.write()
.option("compression", "gzip")
.mode(SaveMode.Overwrite)
.json(outputPath);
}
}

View File

@ -0,0 +1,30 @@
package eu.dnetlib.dhp.oa.graph.dump.exceptions;
public class MyRuntimeException extends RuntimeException {
public MyRuntimeException() {
super();
}
public MyRuntimeException(
final String message,
final Throwable cause,
final boolean enableSuppression,
final boolean writableStackTrace) {
super(message, cause, enableSuppression, writableStackTrace);
}
public MyRuntimeException(final String message, final Throwable cause) {
super(message, cause);
}
public MyRuntimeException(final String message) {
super(message);
}
public MyRuntimeException(final Throwable cause) {
super(cause);
}
}

View File

@ -0,0 +1,29 @@
package eu.dnetlib.dhp.oa.graph.dump.exceptions;
public class NoAvailableEntityTypeException extends Exception {
public NoAvailableEntityTypeException() {
super();
}
public NoAvailableEntityTypeException(
final String message,
final Throwable cause,
final boolean enableSuppression,
final boolean writableStackTrace) {
super(message, cause, enableSuppression, writableStackTrace);
}
public NoAvailableEntityTypeException(final String message, final Throwable cause) {
super(message, cause);
}
public NoAvailableEntityTypeException(final String message) {
super(message);
}
public NoAvailableEntityTypeException(final Throwable cause) {
super(cause);
}
}

View File

@ -17,10 +17,8 @@ import org.slf4j.LoggerFactory;
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
import eu.dnetlib.dhp.oa.graph.dump.Utils;
import eu.dnetlib.dhp.schema.common.ModelConstants;
import eu.dnetlib.dhp.schema.dump.oaf.community.CommunityResult;
import eu.dnetlib.dhp.schema.dump.oaf.community.Project;
import eu.dnetlib.dhp.schema.oaf.Relation;
/**
* Splits the dumped results by funder and stores them in a folder named as the funder nsp (for all the funders, but the EC
@ -51,8 +49,8 @@ public class SparkDumpFunderResults implements Serializable {
final String outputPath = parser.get("outputPath");
log.info("outputPath: {}", outputPath);
final String relationPath = parser.get("relationPath");
log.info("relationPath: {}", relationPath);
final String graphPath = parser.get("graphPath");
log.info("relationPath: {}", graphPath);
SparkConf conf = new SparkConf();
@ -61,18 +59,15 @@ public class SparkDumpFunderResults implements Serializable {
isSparkSessionManaged,
spark -> {
Utils.removeOutputDir(spark, outputPath);
writeResultProjectList(spark, inputPath, outputPath, relationPath);
writeResultProjectList(spark, inputPath, outputPath, graphPath);
});
}
private static void writeResultProjectList(SparkSession spark, String inputPath, String outputPath,
String relationPath) {
String graphPath) {
Dataset<Relation> relation = Utils
.readPath(spark, relationPath + "/relation", Relation.class)
.filter(
"dataInfo.deletedbyinference = false and lower(relClass) = '"
+ ModelConstants.IS_PRODUCED_BY.toLowerCase() + "'");
Dataset<eu.dnetlib.dhp.schema.oaf.Project> project = Utils
.readPath(spark, graphPath + "/project", eu.dnetlib.dhp.schema.oaf.Project.class);
Dataset<CommunityResult> result = Utils
.readPath(spark, inputPath + "/publication", CommunityResult.class)
@ -80,8 +75,8 @@ public class SparkDumpFunderResults implements Serializable {
.union(Utils.readPath(spark, inputPath + "/orp", CommunityResult.class))
.union(Utils.readPath(spark, inputPath + "/software", CommunityResult.class));
List<String> funderList = relation
.select("target")
List<String> funderList = project
.select("id")
.map((MapFunction<Row, String>) value -> value.getString(0).substring(0, 15), Encoders.STRING())
.distinct()
.collectAsList();
@ -99,19 +94,26 @@ public class SparkDumpFunderResults implements Serializable {
} else {
funderdump = fundernsp.substring(0, fundernsp.indexOf("_")).toUpperCase();
}
writeFunderResult(funder, result, outputPath + "/" + funderdump);
writeFunderResult(funder, result, outputPath, funderdump);
});
}
private static void writeFunderResult(String funder, Dataset<CommunityResult> results, String outputPath) {
private static void dumpResults(String nsp, Dataset<CommunityResult> results, String outputPath,
String funderName) {
results.map((MapFunction<CommunityResult, CommunityResult>) r -> {
if (!Optional.ofNullable(r.getProjects()).isPresent()) {
return null;
}
for (Project p : r.getProjects()) {
if (p.getId().startsWith(funder)) {
if (p.getId().startsWith(nsp)) {
if (nsp.startsWith("40|irb")) {
if (p.getFunder().getShortName().equals(funderName))
return r;
else
return null;
}
return r;
}
}
@ -121,7 +123,18 @@ public class SparkDumpFunderResults implements Serializable {
.write()
.mode(SaveMode.Overwrite)
.option("compression", "gzip")
.json(outputPath);
.json(outputPath + "/" + funderName);
}
private static void writeFunderResult(String funder, Dataset<CommunityResult> results, String outputPath,
String funderDump) {
if (funder.startsWith("40|irb")) {
dumpResults(funder, results, outputPath, "HRZZ");
dumpResults(funder, results, outputPath, "MZOS");
} else
dumpResults(funder, results, outputPath, funderDump);
}
}

View File

@ -20,13 +20,13 @@ import org.slf4j.LoggerFactory;
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
import eu.dnetlib.dhp.oa.graph.dump.Utils;
import eu.dnetlib.dhp.schema.common.ModelConstants;
import eu.dnetlib.dhp.schema.oaf.Project;
import eu.dnetlib.dhp.schema.oaf.Relation;
import eu.dnetlib.dhp.schema.oaf.Result;
import scala.Tuple2;
/**
* Selects the results linked to projects. Only for these results the dump will be performed.
* The code to perform the dump and to expend the dumped results with the informaiton related to projects
* The code to perform the dump and to expend the dumped results with the information related to projects
* is the one used for the dump of the community products
*/
public class SparkResultLinkedToProject implements Serializable {
@ -58,8 +58,8 @@ public class SparkResultLinkedToProject implements Serializable {
final String resultClassName = parser.get("resultTableName");
log.info("resultTableName: {}", resultClassName);
final String relationPath = parser.get("relationPath");
log.info("relationPath: {}", relationPath);
final String graphPath = parser.get("graphPath");
log.info("graphPath: {}", graphPath);
@SuppressWarnings("unchecked")
Class<? extends Result> inputClazz = (Class<? extends Result>) Class.forName(resultClassName);
@ -70,37 +70,47 @@ public class SparkResultLinkedToProject implements Serializable {
isSparkSessionManaged,
spark -> {
Utils.removeOutputDir(spark, outputPath);
writeResultsLinkedToProjects(spark, inputClazz, inputPath, outputPath, relationPath);
writeResultsLinkedToProjects(spark, inputClazz, inputPath, outputPath, graphPath);
});
}
private static <R extends Result> void writeResultsLinkedToProjects(SparkSession spark, Class<R> inputClazz,
String inputPath, String outputPath, String relationPath) {
String inputPath, String outputPath, String graphPath) {
Dataset<R> results = Utils
.readPath(spark, inputPath, inputClazz)
.filter("dataInfo.deletedbyinference = false and datainfo.invisible = false");
Dataset<Relation> relations = Utils
.readPath(spark, relationPath, Relation.class)
.readPath(spark, graphPath + "/relation", Relation.class)
.filter(
"dataInfo.deletedbyinference = false and lower(relClass) = '"
+ ModelConstants.IS_PRODUCED_BY.toLowerCase() + "'");
Dataset<Project> project = Utils.readPath(spark, graphPath + "/project", Project.class);
relations
.joinWith(
results, relations.col("source").equalTo(results.col("id")),
"inner")
results.createOrReplaceTempView("result");
relations.createOrReplaceTempView("relation");
project.createOrReplaceTempView("project");
Dataset<R> tmp = spark
.sql(
"Select res.* " +
"from relation rel " +
"join result res " +
"on rel.source = res.id " +
"join project p " +
"on rel.target = p.id " +
"")
.as(Encoders.bean(inputClazz));
tmp
.groupByKey(
(MapFunction<Tuple2<Relation, R>, String>) value -> value
._2()
(MapFunction<R, String>) value -> value
.getId(),
Encoders.STRING())
.mapGroups(
(MapGroupsFunction<String, Tuple2<Relation, R>, R>) (k, it) -> it.next()._2(),
Encoders.bean(inputClazz))
.mapGroups((MapGroupsFunction<String, R, R>) (k, it) -> it.next(), Encoders.bean(inputClazz))
.write()
.mode(SaveMode.Overwrite)
.option("compression", "gzip")
.json(outputPath);
}
}

View File

@ -0,0 +1,136 @@
package eu.dnetlib.dhp.oa.graph.raw;
import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession;
import java.util.*;
import org.apache.commons.io.IOUtils;
import org.apache.spark.SparkConf;
import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.api.java.function.FilterFunction;
import org.apache.spark.api.java.function.FlatMapFunction;
import org.apache.spark.api.java.function.MapFunction;
import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Encoders;
import org.apache.spark.sql.SaveMode;
import org.apache.spark.sql.SparkSession;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import com.clearspring.analytics.util.Lists;
import com.fasterxml.jackson.databind.ObjectMapper;
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
import eu.dnetlib.dhp.common.HdfsSupport;
import eu.dnetlib.dhp.common.vocabulary.VocabularyGroup;
import eu.dnetlib.dhp.oa.graph.raw.common.AbstractMigrationApplication;
import eu.dnetlib.dhp.schema.common.EntityType;
import eu.dnetlib.dhp.schema.common.ModelSupport;
import eu.dnetlib.dhp.schema.oaf.Oaf;
import eu.dnetlib.dhp.schema.oaf.Relation;
import eu.dnetlib.dhp.utils.ISLookupClientFactory;
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService;
public class CopyHdfsOafApplication extends AbstractMigrationApplication {
private static final Logger log = LoggerFactory.getLogger(CopyHdfsOafApplication.class);
private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
public static void main(final String[] args) throws Exception {
final ArgumentApplicationParser parser = new ArgumentApplicationParser(
IOUtils
.toString(
CopyHdfsOafApplication.class
.getResourceAsStream("/eu/dnetlib/dhp/oa/graph/copy_hdfs_oaf_parameters.json")));
parser.parseArgument(args);
final Boolean isSparkSessionManaged = Optional
.ofNullable(parser.get("isSparkSessionManaged"))
.map(Boolean::valueOf)
.orElse(Boolean.TRUE);
log.info("isSparkSessionManaged: {}", isSparkSessionManaged);
final String mdstoreManagerUrl = parser.get("mdstoreManagerUrl");
log.info("mdstoreManagerUrl: {}", mdstoreManagerUrl);
final String mdFormat = parser.get("mdFormat");
log.info("mdFormat: {}", mdFormat);
final String mdLayout = parser.get("mdLayout");
log.info("mdLayout: {}", mdLayout);
final String mdInterpretation = parser.get("mdInterpretation");
log.info("mdInterpretation: {}", mdInterpretation);
final String hdfsPath = parser.get("hdfsPath");
log.info("hdfsPath: {}", hdfsPath);
final String isLookupUrl = parser.get("isLookupUrl");
log.info("isLookupUrl: {}", isLookupUrl);
final ISLookUpService isLookupService = ISLookupClientFactory.getLookUpService(isLookupUrl);
final VocabularyGroup vocs = VocabularyGroup.loadVocsFromIS(isLookupService);
final Set<String> paths = mdstorePaths(mdstoreManagerUrl, mdFormat, mdLayout, mdInterpretation);
final SparkConf conf = new SparkConf();
runWithSparkSession(conf, isSparkSessionManaged, spark -> processPaths(spark, vocs, hdfsPath, paths));
}
public static void processPaths(final SparkSession spark,
final VocabularyGroup vocs,
final String outputPath,
final Set<String> paths) {
final JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext());
log.info("Found {} mdstores", paths.size());
paths.forEach(log::info);
final String[] validPaths = paths
.stream()
.filter(p -> HdfsSupport.exists(p, sc.hadoopConfiguration()))
.toArray(String[]::new);
log.info("Non empty mdstores {}", validPaths.length);
if (validPaths.length > 0) {
// load the dataset
Dataset<Oaf> oaf = spark
.read()
.load(validPaths)
.as(Encoders.kryo(Oaf.class));
// dispatch each entity type individually in the respective graph subdirectory in append mode
for (Map.Entry<String, Class> e : ModelSupport.oafTypes.entrySet()) {
oaf
.filter((FilterFunction<Oaf>) o -> o.getClass().getSimpleName().toLowerCase().equals(e.getKey()))
.map((MapFunction<Oaf, String>) OBJECT_MAPPER::writeValueAsString, Encoders.bean(e.getValue()))
.write()
.option("compression", "gzip")
.mode(SaveMode.Append)
.text(outputPath + "/" + e.getKey());
}
}
}
private static Relation getInverse(Relation rel, VocabularyGroup vocs) {
final Relation inverse = new Relation();
inverse.setProperties(rel.getProperties());
inverse.setValidated(rel.getValidated());
inverse.setValidationDate(rel.getValidationDate());
inverse.setCollectedfrom(rel.getCollectedfrom());
inverse.setDataInfo(rel.getDataInfo());
inverse.setLastupdatetimestamp(rel.getLastupdatetimestamp());
inverse.setSource(rel.getTarget());
inverse.setTarget(rel.getSource());
inverse.setRelType(rel.getRelType());
inverse.setSubRelType(rel.getSubRelType());
return inverse;
}
}

View File

@ -135,30 +135,4 @@ public class MigrateHdfsMdstoresApplication extends AbstractMigrationApplication
}
}
private static Set<String> mdstorePaths(final String mdstoreManagerUrl,
final String format,
final String layout,
final String interpretation) throws IOException {
final String url = mdstoreManagerUrl + "/mdstores/";
final ObjectMapper objectMapper = new ObjectMapper();
final HttpGet req = new HttpGet(url);
try (final CloseableHttpClient client = HttpClients.createDefault()) {
try (final CloseableHttpResponse response = client.execute(req)) {
final String json = IOUtils.toString(response.getEntity().getContent());
final MDStoreWithInfo[] mdstores = objectMapper.readValue(json, MDStoreWithInfo[].class);
return Arrays
.stream(mdstores)
.filter(md -> md.getFormat().equalsIgnoreCase(format))
.filter(md -> md.getLayout().equalsIgnoreCase(layout))
.filter(md -> md.getInterpretation().equalsIgnoreCase(interpretation))
.filter(md -> StringUtils.isNotBlank(md.getHdfsPath()))
.filter(md -> StringUtils.isNotBlank(md.getCurrentVersion()))
.filter(md -> md.getSize() > 0)
.map(md -> md.getHdfsPath() + "/" + md.getCurrentVersion() + "/store")
.collect(Collectors.toSet());
}
}
}
}

View File

@ -3,18 +3,28 @@ package eu.dnetlib.dhp.oa.graph.raw.common;
import java.io.Closeable;
import java.io.IOException;
import java.util.Arrays;
import java.util.Set;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.stream.Collectors;
import org.apache.commons.io.IOUtils;
import org.apache.commons.lang3.StringUtils;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.SequenceFile;
import org.apache.hadoop.io.Text;
import org.apache.http.client.methods.CloseableHttpResponse;
import org.apache.http.client.methods.HttpGet;
import org.apache.http.impl.client.CloseableHttpClient;
import org.apache.http.impl.client.HttpClients;
import com.fasterxml.jackson.core.JsonProcessingException;
import com.fasterxml.jackson.databind.ObjectMapper;
import eu.dnetlib.dhp.schema.mdstore.MDStoreWithInfo;
import eu.dnetlib.dhp.schema.oaf.Oaf;
public class AbstractMigrationApplication implements Closeable {
@ -47,6 +57,43 @@ public class AbstractMigrationApplication implements Closeable {
SequenceFile.Writer.valueClass(Text.class));
}
/**
* Retrieves from the metadata store manager application the list of paths associated with mdstores characterized
* by he given format, layout, interpretation
* @param mdstoreManagerUrl the URL of the mdstore manager service
* @param format the mdstore format
* @param layout the mdstore layout
* @param interpretation the mdstore interpretation
* @return the set of hdfs paths
* @throws IOException in case of HTTP communication issues
*/
protected static Set<String> mdstorePaths(final String mdstoreManagerUrl,
final String format,
final String layout,
final String interpretation) throws IOException {
final String url = mdstoreManagerUrl + "/mdstores/";
final ObjectMapper objectMapper = new ObjectMapper();
final HttpGet req = new HttpGet(url);
try (final CloseableHttpClient client = HttpClients.createDefault()) {
try (final CloseableHttpResponse response = client.execute(req)) {
final String json = IOUtils.toString(response.getEntity().getContent());
final MDStoreWithInfo[] mdstores = objectMapper.readValue(json, MDStoreWithInfo[].class);
return Arrays
.stream(mdstores)
.filter(md -> md.getFormat().equalsIgnoreCase(format))
.filter(md -> md.getLayout().equalsIgnoreCase(layout))
.filter(md -> md.getInterpretation().equalsIgnoreCase(interpretation))
.filter(md -> StringUtils.isNotBlank(md.getHdfsPath()))
.filter(md -> StringUtils.isNotBlank(md.getCurrentVersion()))
.filter(md -> md.getSize() > 0)
.map(md -> md.getHdfsPath() + "/" + md.getCurrentVersion() + "/store")
.collect(Collectors.toSet());
}
}
}
private Configuration getConf() {
return new Configuration();
/*

View File

@ -0,0 +1,38 @@
[
{
"paramName": "p",
"paramLongName": "hdfsPath",
"paramDescription": "the path where storing the sequential file",
"paramRequired": true
},
{
"paramName": "u",
"paramLongName": "mdstoreManagerUrl",
"paramDescription": "the MdstoreManager url",
"paramRequired": true
},
{
"paramName": "f",
"paramLongName": "mdFormat",
"paramDescription": "metadata format",
"paramRequired": true
},
{
"paramName": "l",
"paramLongName": "mdLayout",
"paramDescription": "metadata layout",
"paramRequired": true
},
{
"paramName": "i",
"paramLongName": "mdInterpretation",
"paramDescription": "metadata interpretation",
"paramRequired": true
},
{
"paramName": "isu",
"paramLongName": "isLookupUrl",
"paramDescription": "the url of the ISLookupService",
"paramRequired": true
}
]

View File

@ -1,6 +1,23 @@
{
"$schema": "http://json-schema.org/draft-07/schema#",
"definitions": {
"AccessRight":{
"type":"object",
"properties":{
"code": {
"type": "string",
"description": "COAR access mode code: http://vocabularies.coar-repositories.org/documentation/access_rights/"
},
"label": {
"type": "string",
"description": "Label for the access mode"
},
"scheme": {
"type": "string",
"description": "Scheme of reference for access right code. Always set to COAR access rights vocabulary: http://vocabularies.coar-repositories.org/documentation/access_rights/"
}
}
},
"ControlledField": {
"type": "object",
"properties": {
@ -266,6 +283,57 @@
]
}
},
"instance":{
"type":"array",
"items":{
"type":"object",
"properties":{
"accessright":{
"allOf":[
{
"$ref":"#/definitions/AccessRight"
},
{
"description":"The accessright of this materialization of the result"
}
]
},
"articleprocessingcharge":{
"type":"object",
"properties":{
"amount":{
"type":"string"
},
"currency":{
"type":"string"
}
}
},
"license":{
"type":"string"
},
"publicationdate":{
"type":"string"
},
"refereed":{
"type":"string"
},
"type":{
"type":"string",
"description":"The specific sub-type of this materialization of the result (see https://api.openaire.eu/vocabularies/dnet:result_typologies following the links)"
},
"url":{
"description":"Description of url",
"type":"array",
"items":{
"type":"string",
"description":"urls where it is possible to access the materialization of the result"
}
}
},
"description":"One of the materialization for this result"
}
},
"programmingLanguage": {
"type": "string",
"description": "Only for results with type 'software': the programming language"
@ -302,7 +370,7 @@
"subject": {
"allOf": [
{"$ref": "#/definitions/ControlledField"},
{"description": "OpenAIRE subject classification scheme (https://api.openaire.eu/vocabularies/dnet:subject_classification_typologies) and value. When the scheme is 'keyword', it means that the subject is free-text (i.e. not a term from a controlled vocabulary)."},
{"description": "OpenAIRE subject classification scheme (https://api.openaire.eu/vocabularies/dnet:subject_classification_typologies) and value. When the scheme is 'keyword', it means that the subject is free-text (i.e. not a term from a controlled vocabulary)."}
]
}
}

View File

@ -18,8 +18,8 @@
"paramRequired": false
},
{
"paramName": "rp",
"paramLongName": "relationPath",
"paramName": "gp",
"paramLongName": "graphPath",
"paramDescription": "the relationPath",
"paramRequired": true
}

View File

@ -18,6 +18,18 @@
"paramLongName": "outputPath",
"paramDescription": "the path used to store temporary output files",
"paramRequired": true
},
{
"paramName": "sd",
"paramLongName": "singleDeposition",
"paramDescription": "true if the dump should be created for a single community",
"paramRequired": true
},
{
"paramName": "ci",
"paramLongName": "communityId",
"paramDescription": "the id of the community for which to create the dump",
"paramRequired": true
}
]

View File

@ -35,7 +35,12 @@
"paramLongName":"dumpType",
"paramDescription": "the type of the dump (complete for the whole graph, community for the products related to communities, funder for the results with at least a link to project",
"paramRequired": false
}
}, {
"paramName":"cid",
"paramLongName":"communityId",
"paramDescription": "the id of the community to be dumped",
"paramRequired": false
}
]

View File

@ -24,8 +24,8 @@
"paramRequired": true
},
{
"paramName":"rp",
"paramLongName":"relationPath",
"paramName":"gp",
"paramLongName":"graphPath",
"paramDescription": "the path to the relations",
"paramRequired": true
}

View File

@ -25,7 +25,12 @@
"paramLongName": "isSparkSessionManaged",
"paramDescription": "true if the spark session is managed, false otherwise",
"paramRequired": false
}
}, {
"paramName":"cid",
"paramLongName":"communityId",
"paramDescription": "the id of the community to be dumped",
"paramRequired": false
}
]

View File

@ -12,12 +12,7 @@
"paramDescription": "The id of the concept record for a new version",
"paramRequired": false
},
{
"paramName":"cmp",
"paramLongName":"communityMapPath",
"paramDescription": "the path to the serialization of the community map",
"paramRequired": false
},
{
"paramName":"di",
"paramLongName":"depositionId",

View File

@ -0,0 +1,30 @@
<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>hiveMetastoreUris</name>
<value>thrift://iis-cdh5-test-m3.ocean.icm.edu.pl:9083</value>
</property>
<property>
<name>hiveJdbcUrl</name>
<value>jdbc:hive2://iis-cdh5-test-m3.ocean.icm.edu.pl:10000</value>
</property>
<property>
<name>hiveDbName</name>
<value>openaire</value>
</property>
<property>
<name>oozie.launcher.mapreduce.user.classpath.first</name>
<value>true</value>
</property>
</configuration>

View File

@ -0,0 +1,4 @@
## This is a classpath-based import file (this header is required)
dump_complete classpath eu/dnetlib/dhp/oa/graph/dump/wf/subworkflows/complete/oozie_app
dump_funder classpath eu/dnetlib/dhp/oa/graph/dump/wf/subworkflows/funder/oozie_app
dump_community classpath eu/dnetlib/dhp/oa/graph/dump/wf/subworkflows/community/oozie_app

View File

@ -0,0 +1,306 @@
<workflow-app name="dump_graph" xmlns="uri:oozie:workflow:0.5">
<parameters>
<property>
<name>singleDeposition</name>
<description>Indicates if it is a single community deposition</description>
</property>
<property>
<name>communityId</name>
<description>the id of the community to be dumped if a dump for a single community should be done</description>
</property>
<property>
<name>dumpType</name>
<description>the type of the dump one of {complete, community, funder}</description>
</property>
<property>
<name>onlyUpload</name>
<description>true if the dump is already done and should only be upload in zenodo</description>
</property>
<property>
<name>upload</name>
<description>true if the dump should be upload in zenodo</description>
</property>
<property>
<name>sourcePath</name>
<description>the source path</description>
</property>
<property>
<name>isLookUpUrl</name>
<description>the isLookup service endpoint</description>
</property>
<property>
<name>outputPath</name>
<description>the output path</description>
</property>
<property>
<name>resultAggregation</name>
<description>true if all the result type have to be dumped under result. false otherwise</description>
</property>
<property>
<name>accessToken</name>
<description>the access token used for the deposition in Zenodo</description>
</property>
<property>
<name>connectionUrl</name>
<description>the connection url for Zenodo</description>
</property>
<property>
<name>metadata</name>
<description> the metadata associated to the deposition</description>
</property>
<property>
<name>depositionType</name>
<description>the type of deposition we want to perform. "new" for brand new deposition, "version" for a new version of a published deposition (in this case the concept record id must be provided), "upload" to upload content to an open deposition for which we already have the deposition id (in this case the deposition id should be provided)</description>
</property>
<property>
<name>conceptRecordId</name>
<description>for new version, the id of the record for the old deposition</description>
</property>
<property>
<name>depositionId</name>
<description>the depositionId of a deposition open that has to be added content</description>
</property>
<property>
<name>organizationCommunityMap</name>
<description>the organization community map</description>
</property>
<property>
<name>hiveDbName</name>
<description>the target hive database name</description>
</property>
<property>
<name>hiveJdbcUrl</name>
<description>hive server jdbc url</description>
</property>
<property>
<name>hiveMetastoreUris</name>
<description>hive server metastore URIs</description>
</property>
<property>
<name>sparkDriverMemory</name>
<description>memory for driver process</description>
</property>
<property>
<name>sparkExecutorMemory</name>
<description>memory for individual executor</description>
</property>
<property>
<name>sparkExecutorCores</name>
<description>number of cores used by single executor</description>
</property>
<property>
<name>oozieActionShareLibForSpark2</name>
<description>oozie action sharelib for spark 2.*</description>
</property>
<property>
<name>spark2ExtraListeners</name>
<value>com.cloudera.spark.lineage.NavigatorAppListener</value>
<description>spark 2.* extra listeners classname</description>
</property>
<property>
<name>spark2SqlQueryExecutionListeners</name>
<value>com.cloudera.spark.lineage.NavigatorQueryListener</value>
<description>spark 2.* sql query execution listeners classname</description>
</property>
<property>
<name>spark2YarnHistoryServerAddress</name>
<description>spark 2.* yarn history server address</description>
</property>
<property>
<name>spark2EventLogDir</name>
<description>spark 2.* event log dir location</description>
</property>
</parameters>
<global>
<job-tracker>${jobTracker}</job-tracker>
<name-node>${nameNode}</name-node>
<configuration>
<property>
<name>mapreduce.job.queuename</name>
<value>${queueName}</value>
</property>
<property>
<name>oozie.launcher.mapred.job.queue.name</name>
<value>${oozieLauncherQueueName}</value>
</property>
<property>
<name>oozie.action.sharelib.for.spark</name>
<value>${oozieActionShareLibForSpark2}</value>
</property>
</configuration>
</global>
<start to="only_upload"/>
<kill name="Kill">
<message>Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}]</message>
</kill>
<decision name="only_upload">
<switch>
<case to="send_zenodo">${wf:conf('onlyUpload') eq true}</case>
<default to="reset_outputpath"/>
</switch>
</decision>
<action name="reset_outputpath">
<fs>
<delete path="${outputPath}"/>
<mkdir path="${outputPath}"/>
</fs>
<ok to="save_community_map"/>
<error to="Kill"/>
</action>
<action name="save_community_map">
<java>
<main-class>eu.dnetlib.dhp.oa.graph.dump.SaveCommunityMap</main-class>
<arg>--outputPath</arg><arg>${workingDir}/communityMap</arg>
<arg>--nameNode</arg><arg>${nameNode}</arg>
<arg>--isLookUpUrl</arg><arg>${isLookUpUrl}</arg>
<arg>--singleDeposition</arg><arg>${singleDeposition}</arg>
<arg>--communityId</arg><arg>${communityId}</arg>
</java>
<ok to="choose_dump"/>
<error to="Kill"/>
</action>
<decision name="choose_dump">
<switch>
<case to="dump_funder">${wf:conf('dumpType') eq "funder"}</case>
<case to="dump_community">${wf:conf('dumpType') eq "community"}</case>
<default to="dump_complete"/>
</switch>
</decision>
<!-- Sub-workflow which runs the dump for the complete graph -->
<action name="dump_complete">
<sub-workflow>
<app-path>${wf:appPath()}/dump_complete
</app-path>
<propagate-configuration/>
<configuration>
<property>
<name>communityMapPath</name>
<value>${workingDir}/communityMap</value>
</property>
<property>
<name>outputPath</name>
<value>${workingDir}/tar</value>
</property>
<property>
<name>sourcePath</name>
<value>${sourcePath}</value>
</property>
<property>
<name>organizationCommunityMap</name>
<value>${organizationCommunityMap}</value>
</property>
<property>
<name>isLookUpUrl</name>
<value>${isLookUpUrl}</value>
</property>
<property>
<name>resultAggregation</name>
<value>${resultAggregation}</value>
</property>
</configuration>
</sub-workflow>
<ok to="make_archive" />
<error to="Kill" />
</action>
<!-- Sub-workflow which runs the dump for the complete graph -->
<action name="dump_community">
<sub-workflow>
<app-path>${wf:appPath()}/dump_community
</app-path>
<propagate-configuration/>
<configuration>
<property>
<name>sourcePath</name>
<value>${sourcePath}</value>
</property>
<property>
<name>communityMapPath</name>
<value>${workingDir}/communityMap</value>
</property>
<property>
<name>outputPath</name>
<value>${workingDir}/tar</value>
</property>
</configuration>
</sub-workflow>
<ok to="make_archive" />
<error to="Kill" />
</action>
<action name="dump_funder">
<sub-workflow>
<app-path>${wf:appPath()}/dump_funder
</app-path>
<propagate-configuration/>
<configuration>
<property>
<name>communityMapPath</name>
<value>${workingDir}/communityMap</value>
</property>
<property>
<name>outputPath</name>
<value>${workingDir}/tar</value>
</property>
<property>
<name>sourcePath</name>
<value>${sourcePath}</value>
</property>
<property>
<name>dumpType</name>
<value>${dumpType}</value>
</property>
</configuration>
</sub-workflow>
<ok to="make_archive" />
<error to="Kill" />
</action>
<action name="make_archive">
<java>
<main-class>eu.dnetlib.dhp.oa.graph.dump.MakeTar</main-class>
<arg>--hdfsPath</arg><arg>${outputPath}</arg>
<arg>--nameNode</arg><arg>${nameNode}</arg>
<arg>--sourcePath</arg><arg>${workingDir}/tar</arg>
</java>
<ok to="should_upload"/>
<error to="Kill"/>
</action>
<decision name="should_upload">
<switch>
<case to="send_zenodo">${wf:conf('upload') eq true}</case>
<default to="End"/>
</switch>
</decision>
<action name="send_zenodo">
<java>
<main-class>eu.dnetlib.dhp.oa.graph.dump.SendToZenodoHDFS</main-class>
<arg>--hdfsPath</arg><arg>${outputPath}</arg>
<arg>--nameNode</arg><arg>${nameNode}</arg>
<arg>--accessToken</arg><arg>${accessToken}</arg>
<arg>--connectionUrl</arg><arg>${connectionUrl}</arg>
<arg>--metadata</arg><arg>${metadata}</arg>
<arg>--conceptRecordId</arg><arg>${conceptRecordId}</arg>
<arg>--depositionType</arg><arg>${depositionType}</arg>
<arg>--depositionId</arg><arg>${depositionId}</arg>
</java>
<ok to="End"/>
<error to="Kill"/>
</action>
<end name="End"/>
</workflow-app>

View File

@ -0,0 +1,30 @@
<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>hiveMetastoreUris</name>
<value>thrift://iis-cdh5-test-m3.ocean.icm.edu.pl:9083</value>
</property>
<property>
<name>hiveJdbcUrl</name>
<value>jdbc:hive2://iis-cdh5-test-m3.ocean.icm.edu.pl:10000</value>
</property>
<property>
<name>hiveDbName</name>
<value>openaire</value>
</property>
<property>
<name>oozie.launcher.mapreduce.user.classpath.first</name>
<value>true</value>
</property>
</configuration>

View File

@ -0,0 +1,347 @@
<workflow-app name="sub_dump_community_funder_results" xmlns="uri:oozie:workflow:0.5">
<parameters>
<property>
<name>sourcePath</name>
<description>the source path</description>
</property>
<property>
<name>outputPath</name>
<description>the output path</description>
</property>
<property>
<name>communityMapPath</name>
<description>the path to the community map</description>
</property>
<property>
<name>selectedResults</name>
<description>the path the the possible subset ot results to be dumped</description>
</property>
<property>
<name>hiveDbName</name>
<description>the target hive database name</description>
</property>
<property>
<name>hiveJdbcUrl</name>
<description>hive server jdbc url</description>
</property>
<property>
<name>hiveMetastoreUris</name>
<description>hive server metastore URIs</description>
</property>
<property>
<name>sparkDriverMemory</name>
<description>memory for driver process</description>
</property>
<property>
<name>sparkExecutorMemory</name>
<description>memory for individual executor</description>
</property>
<property>
<name>sparkExecutorCores</name>
<description>number of cores used by single executor</description>
</property>
<property>
<name>oozieActionShareLibForSpark2</name>
<description>oozie action sharelib for spark 2.*</description>
</property>
<property>
<name>spark2ExtraListeners</name>
<value>com.cloudera.spark.lineage.NavigatorAppListener</value>
<description>spark 2.* extra listeners classname</description>
</property>
<property>
<name>spark2SqlQueryExecutionListeners</name>
<value>com.cloudera.spark.lineage.NavigatorQueryListener</value>
<description>spark 2.* sql query execution listeners classname</description>
</property>
<property>
<name>spark2YarnHistoryServerAddress</name>
<description>spark 2.* yarn history server address</description>
</property>
<property>
<name>spark2EventLogDir</name>
<description>spark 2.* event log dir location</description>
</property>
</parameters>
<global>
<job-tracker>${jobTracker}</job-tracker>
<name-node>${nameNode}</name-node>
<configuration>
<property>
<name>mapreduce.job.queuename</name>
<value>${queueName}</value>
</property>
<property>
<name>oozie.launcher.mapred.job.queue.name</name>
<value>${oozieLauncherQueueName}</value>
</property>
<property>
<name>oozie.action.sharelib.for.spark</name>
<value>${oozieActionShareLibForSpark2}</value>
</property>
</configuration>
</global>
<start to="fork_dump"/>
<kill name="Kill">
<message>Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}]</message>
</kill>
<fork name="fork_dump">
<path start="dump_publication"/>
<path start="dump_dataset"/>
<path start="dump_orp"/>
<path start="dump_software"/>
</fork>
<action name="dump_publication">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn</master>
<mode>cluster</mode>
<name>Dump table publication for community/funder related products</name>
<class>eu.dnetlib.dhp.oa.graph.dump.community.SparkDumpCommunityProducts</class>
<jar>dhp-graph-mapper-${projectVersion}.jar</jar>
<spark-opts>
--executor-memory=${sparkExecutorMemory}
--executor-cores=${sparkExecutorCores}
--driver-memory=${sparkDriverMemory}
--conf spark.extraListeners=${spark2ExtraListeners}
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
--conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir}
</spark-opts>
<arg>--sourcePath</arg><arg>${selectedResults}/publication</arg>
<arg>--resultTableName</arg><arg>eu.dnetlib.dhp.schema.oaf.Publication</arg>
<arg>--outputPath</arg><arg>${workingDir}/dump/publication</arg>
<arg>--communityMapPath</arg><arg>${communityMapPath}</arg>
<arg>--dumpType</arg><arg>${dumpType}</arg>
</spark>
<ok to="join_dump"/>
<error to="Kill"/>
</action>
<action name="dump_dataset">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn</master>
<mode>cluster</mode>
<name>Dump table dataset for community/funder related products</name>
<class>eu.dnetlib.dhp.oa.graph.dump.community.SparkDumpCommunityProducts</class>
<jar>dhp-graph-mapper-${projectVersion}.jar</jar>
<spark-opts>
--executor-memory=${sparkExecutorMemory}
--executor-cores=${sparkExecutorCores}
--driver-memory=${sparkDriverMemory}
--conf spark.extraListeners=${spark2ExtraListeners}
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
--conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir}
</spark-opts>
<arg>--sourcePath</arg><arg>${selectedResults}/dataset</arg>
<arg>--resultTableName</arg><arg>eu.dnetlib.dhp.schema.oaf.Dataset</arg>
<arg>--outputPath</arg><arg>${workingDir}/dump/dataset</arg>
<arg>--communityMapPath</arg><arg>${communityMapPath}</arg>
</spark>
<ok to="join_dump"/>
<error to="Kill"/>
</action>
<action name="dump_orp">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn</master>
<mode>cluster</mode>
<name>Dump table ORP for community related products</name>
<class>eu.dnetlib.dhp.oa.graph.dump.community.SparkDumpCommunityProducts</class>
<jar>dhp-graph-mapper-${projectVersion}.jar</jar>
<spark-opts>
--executor-memory=${sparkExecutorMemory}
--executor-cores=${sparkExecutorCores}
--driver-memory=${sparkDriverMemory}
--conf spark.extraListeners=${spark2ExtraListeners}
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
--conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir}
</spark-opts>
<arg>--sourcePath</arg><arg>${selectedResults}/otherresearchproduct</arg>
<arg>--resultTableName</arg><arg>eu.dnetlib.dhp.schema.oaf.OtherResearchProduct</arg>
<arg>--outputPath</arg><arg>${workingDir}/dump/otherresearchproduct</arg>
<arg>--communityMapPath</arg><arg>${communityMapPath}</arg>
</spark>
<ok to="join_dump"/>
<error to="Kill"/>
</action>
<action name="dump_software">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn</master>
<mode>cluster</mode>
<name>Dump table software for community related products</name>
<class>eu.dnetlib.dhp.oa.graph.dump.community.SparkDumpCommunityProducts</class>
<jar>dhp-graph-mapper-${projectVersion}.jar</jar>
<spark-opts>
--executor-memory=${sparkExecutorMemory}
--executor-cores=${sparkExecutorCores}
--driver-memory=${sparkDriverMemory}
--conf spark.extraListeners=${spark2ExtraListeners}
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
--conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir}
</spark-opts>
<arg>--sourcePath</arg><arg>${selectedResults}/software</arg>
<arg>--resultTableName</arg><arg>eu.dnetlib.dhp.schema.oaf.Software</arg>
<arg>--outputPath</arg><arg>${workingDir}/dump/software</arg>
<arg>--communityMapPath</arg><arg>${communityMapPath}</arg>
</spark>
<ok to="join_dump"/>
<error to="Kill"/>
</action>
<join name="join_dump" to="prepareResultProject"/>
<action name="prepareResultProject">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn</master>
<mode>cluster</mode>
<name>Prepare association result subset of project info</name>
<class>eu.dnetlib.dhp.oa.graph.dump.community.SparkPrepareResultProject</class>
<jar>dhp-graph-mapper-${projectVersion}.jar</jar>
<spark-opts>
--executor-memory=${sparkExecutorMemory}
--executor-cores=${sparkExecutorCores}
--driver-memory=${sparkDriverMemory}
--conf spark.extraListeners=${spark2ExtraListeners}
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
--conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir}
</spark-opts>
<arg>--sourcePath</arg><arg>${sourcePath}</arg>
<arg>--outputPath</arg><arg>${workingDir}/preparedInfo</arg>
</spark>
<ok to="fork_extendWithProject"/>
<error to="Kill"/>
</action>
<fork name="fork_extendWithProject">
<path start="extend_publication"/>
<path start="extend_dataset"/>
<path start="extend_orp"/>
<path start="extend_software"/>
</fork>
<action name="extend_publication">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn</master>
<mode>cluster</mode>
<name>Extend dumped publications with information about project</name>
<class>eu.dnetlib.dhp.oa.graph.dump.community.SparkUpdateProjectInfo</class>
<jar>dhp-graph-mapper-${projectVersion}.jar</jar>
<spark-opts>
--executor-memory=${sparkExecutorMemory}
--executor-cores=${sparkExecutorCores}
--driver-memory=${sparkDriverMemory}
--conf spark.extraListeners=${spark2ExtraListeners}
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
--conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir}
</spark-opts>
<arg>--sourcePath</arg><arg>${workingDir}/dump/publication</arg>
<arg>--outputPath</arg><arg>${outputPath}/ext/publication</arg>
<arg>--preparedInfoPath</arg><arg>${workingDir}/preparedInfo</arg>
</spark>
<ok to="join_extend"/>
<error to="Kill"/>
</action>
<action name="extend_dataset">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn</master>
<mode>cluster</mode>
<name>Extend dumped dataset with information about project</name>
<class>eu.dnetlib.dhp.oa.graph.dump.community.SparkUpdateProjectInfo</class>
<jar>dhp-graph-mapper-${projectVersion}.jar</jar>
<spark-opts>
--executor-memory=${sparkExecutorMemory}
--executor-cores=${sparkExecutorCores}
--driver-memory=${sparkDriverMemory}
--conf spark.extraListeners=${spark2ExtraListeners}
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
--conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir}
</spark-opts>
<arg>--sourcePath</arg><arg>${workingDir}/dump/dataset</arg>
<arg>--outputPath</arg><arg>${outputPath}/ext/dataset</arg>
<arg>--preparedInfoPath</arg><arg>${workingDir}/preparedInfo</arg>
</spark>
<ok to="join_extend"/>
<error to="Kill"/>
</action>
<action name="extend_orp">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn</master>
<mode>cluster</mode>
<name>Extend dumped ORP with information about project</name>
<class>eu.dnetlib.dhp.oa.graph.dump.community.SparkUpdateProjectInfo</class>
<jar>dhp-graph-mapper-${projectVersion}.jar</jar>
<spark-opts>
--executor-memory=${sparkExecutorMemory}
--executor-cores=${sparkExecutorCores}
--driver-memory=${sparkDriverMemory}
--conf spark.extraListeners=${spark2ExtraListeners}
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
--conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir}
</spark-opts>
<arg>--sourcePath</arg><arg>${workingDir}/dump/otherresearchproduct</arg>
<arg>--outputPath</arg><arg>${outputPath}/ext/orp</arg>
<arg>--preparedInfoPath</arg><arg>${workingDir}/preparedInfo</arg>
</spark>
<ok to="join_extend"/>
<error to="Kill"/>
</action>
<action name="extend_software">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn</master>
<mode>cluster</mode>
<name>Extend dumped software with information about project</name>
<class>eu.dnetlib.dhp.oa.graph.dump.community.SparkUpdateProjectInfo</class>
<jar>dhp-graph-mapper-${projectVersion}.jar</jar>
<spark-opts>
--executor-memory=${sparkExecutorMemory}
--executor-cores=${sparkExecutorCores}
--driver-memory=${sparkDriverMemory}
--conf spark.extraListeners=${spark2ExtraListeners}
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
--conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir}
</spark-opts>
<arg>--sourcePath</arg><arg>${workingDir}/dump/software</arg>
<arg>--outputPath</arg><arg>${outputPath}/ext/software</arg>
<arg>--preparedInfoPath</arg><arg>${workingDir}/preparedInfo</arg>
</spark>
<ok to="join_extend"/>
<error to="Kill"/>
</action>
<join name="join_extend" to="End"/>
<end name="End"/>
</workflow-app>

View File

@ -0,0 +1,30 @@
<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>hiveMetastoreUris</name>
<value>thrift://iis-cdh5-test-m3.ocean.icm.edu.pl:9083</value>
</property>
<property>
<name>hiveJdbcUrl</name>
<value>jdbc:hive2://iis-cdh5-test-m3.ocean.icm.edu.pl:10000</value>
</property>
<property>
<name>hiveDbName</name>
<value>openaire</value>
</property>
<property>
<name>oozie.launcher.mapreduce.user.classpath.first</name>
<value>true</value>
</property>
</configuration>

View File

@ -0,0 +1,2 @@
## This is a classpath-based import file (this header is required)
dump_common classpath eu/dnetlib/dhp/oa/graph/dump/wf/subworkflows/commoncommunityfunder/oozie_app

View File

@ -0,0 +1,145 @@
<workflow-app name="sub_dump_community_products" xmlns="uri:oozie:workflow:0.5">
<parameters>
<property>
<name>sourcePath</name>
<description>the source path</description>
</property>
<property>
<name>outputPath</name>
<description>the output path</description>
</property>
<property>
<name>hiveDbName</name>
<description>the target hive database name</description>
</property>
<property>
<name>hiveJdbcUrl</name>
<description>hive server jdbc url</description>
</property>
<property>
<name>hiveMetastoreUris</name>
<description>hive server metastore URIs</description>
</property>
<property>
<name>sparkDriverMemory</name>
<description>memory for driver process</description>
</property>
<property>
<name>sparkExecutorMemory</name>
<description>memory for individual executor</description>
</property>
<property>
<name>sparkExecutorCores</name>
<description>number of cores used by single executor</description>
</property>
<property>
<name>oozieActionShareLibForSpark2</name>
<description>oozie action sharelib for spark 2.*</description>
</property>
<property>
<name>spark2ExtraListeners</name>
<value>com.cloudera.spark.lineage.NavigatorAppListener</value>
<description>spark 2.* extra listeners classname</description>
</property>
<property>
<name>spark2SqlQueryExecutionListeners</name>
<value>com.cloudera.spark.lineage.NavigatorQueryListener</value>
<description>spark 2.* sql query execution listeners classname</description>
</property>
<property>
<name>spark2YarnHistoryServerAddress</name>
<description>spark 2.* yarn history server address</description>
</property>
<property>
<name>spark2EventLogDir</name>
<description>spark 2.* event log dir location</description>
</property>
</parameters>
<global>
<job-tracker>${jobTracker}</job-tracker>
<name-node>${nameNode}</name-node>
<configuration>
<property>
<name>mapreduce.job.queuename</name>
<value>${queueName}</value>
</property>
<property>
<name>oozie.launcher.mapred.job.queue.name</name>
<value>${oozieLauncherQueueName}</value>
</property>
<property>
<name>oozie.action.sharelib.for.spark</name>
<value>${oozieActionShareLibForSpark2}</value>
</property>
</configuration>
</global>
<start to="common_action_community_funder"/>
<kill name="Kill">
<message>Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}]</message>
</kill>
<action name="common_action_community_funder">
<sub-workflow>
<app-path>${wf:appPath()}/dump_common
</app-path>
<propagate-configuration/>
<configuration>
<property>
<name>sourcePath</name>
<value>${sourcePath}</value>
</property>
<property>
<name>selectedResults</name>
<value>${sourcePath}</value>
</property>
<property>
<name>communityMapPath</name>
<value>${workingDir}/communityMap</value>
</property>
<property>
<name>outputPath</name>
<value>${workingDir}</value>
</property>
</configuration>
</sub-workflow>
<ok to="splitForCommunities" />
<error to="Kill" />
</action>
<action name="splitForCommunities">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn</master>
<mode>cluster</mode>
<name>Split dumped result for community</name>
<class>eu.dnetlib.dhp.oa.graph.dump.community.SparkSplitForCommunity</class>
<jar>dhp-graph-mapper-${projectVersion}.jar</jar>
<spark-opts>
--executor-memory=${sparkExecutorMemory}
--executor-cores=${sparkExecutorCores}
--driver-memory=${sparkDriverMemory}
--conf spark.extraListeners=${spark2ExtraListeners}
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
--conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir}
</spark-opts>
<arg>--sourcePath</arg><arg>${workingDir}/ext</arg>
<arg>--outputPath</arg><arg>${outputPath}</arg>
<arg>--communityMapPath</arg><arg>${communityMapPath}</arg>
</spark>
<ok to="End"/>
<error to="Kill"/>
</action>
<end name="End"/>
</workflow-app>

View File

@ -0,0 +1,30 @@
<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>hiveMetastoreUris</name>
<value>thrift://iis-cdh5-test-m3.ocean.icm.edu.pl:9083</value>
</property>
<property>
<name>hiveJdbcUrl</name>
<value>jdbc:hive2://iis-cdh5-test-m3.ocean.icm.edu.pl:10000</value>
</property>
<property>
<name>hiveDbName</name>
<value>openaire</value>
</property>
<property>
<name>oozie.launcher.mapreduce.user.classpath.first</name>
<value>true</value>
</property>
</configuration>

View File

@ -0,0 +1,537 @@
<workflow-app name="sub-dump_complete" xmlns="uri:oozie:workflow:0.5">
<parameters>
<property>
<name>sourcePath</name>
<description>the source path</description>
</property>
<property>
<name>outputPath</name>
<description>the output path</description>
</property>
<property>
<name>resultAggregation</name>
<description>true if all the result type have to be dumped under result. false otherwise</description>
</property>
<property>
<name>organizationCommunityMap</name>
<description>the organization community map</description>
</property>
<property>
<name>hiveDbName</name>
<description>the target hive database name</description>
</property>
<property>
<name>hiveJdbcUrl</name>
<description>hive server jdbc url</description>
</property>
<property>
<name>hiveMetastoreUris</name>
<description>hive server metastore URIs</description>
</property>
<property>
<name>sparkDriverMemory</name>
<description>memory for driver process</description>
</property>
<property>
<name>sparkExecutorMemory</name>
<description>memory for individual executor</description>
</property>
<property>
<name>sparkExecutorCores</name>
<description>number of cores used by single executor</description>
</property>
<property>
<name>oozieActionShareLibForSpark2</name>
<description>oozie action sharelib for spark 2.*</description>
</property>
<property>
<name>spark2ExtraListeners</name>
<value>com.cloudera.spark.lineage.NavigatorAppListener</value>
<description>spark 2.* extra listeners classname</description>
</property>
<property>
<name>spark2SqlQueryExecutionListeners</name>
<value>com.cloudera.spark.lineage.NavigatorQueryListener</value>
<description>spark 2.* sql query execution listeners classname</description>
</property>
<property>
<name>spark2YarnHistoryServerAddress</name>
<description>spark 2.* yarn history server address</description>
</property>
<property>
<name>spark2EventLogDir</name>
<description>spark 2.* event log dir location</description>
</property>
</parameters>
<global>
<job-tracker>${jobTracker}</job-tracker>
<name-node>${nameNode}</name-node>
<configuration>
<property>
<name>mapreduce.job.queuename</name>
<value>${queueName}</value>
</property>
<property>
<name>oozie.launcher.mapred.job.queue.name</name>
<value>${oozieLauncherQueueName}</value>
</property>
<property>
<name>oozie.action.sharelib.for.spark</name>
<value>${oozieActionShareLibForSpark2}</value>
</property>
</configuration>
</global>
<start to="fork_dump" />
<fork name="fork_dump">
<path start="dump_publication"/>
<path start="dump_dataset"/>
<path start="dump_orp"/>
<path start="dump_software"/>
<path start="dump_organization"/>
<path start="dump_project"/>
<path start="dump_datasource"/>
<path start="select_relation"/>
</fork>
<action name="dump_publication">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn</master>
<mode>cluster</mode>
<name>Dump table publication </name>
<class>eu.dnetlib.dhp.oa.graph.dump.complete.SparkDumpEntitiesJob</class>
<jar>dhp-graph-mapper-${projectVersion}.jar</jar>
<spark-opts>
--executor-memory=${sparkExecutorMemory}
--executor-cores=${sparkExecutorCores}
--driver-memory=${sparkDriverMemory}
--conf spark.extraListeners=${spark2ExtraListeners}
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
--conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir}
</spark-opts>
<arg>--sourcePath</arg><arg>${sourcePath}/publication</arg>
<arg>--resultTableName</arg><arg>eu.dnetlib.dhp.schema.oaf.Publication</arg>
<arg>--outputPath</arg><arg>${workingDir}/result/publication</arg>
<arg>--communityMapPath</arg><arg>${communityMapPath}</arg>
</spark>
<ok to="join_dump"/>
<error to="Kill"/>
</action>
<action name="dump_dataset">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn</master>
<mode>cluster</mode>
<name>Dump table dataset </name>
<class>eu.dnetlib.dhp.oa.graph.dump.complete.SparkDumpEntitiesJob</class>
<jar>dhp-graph-mapper-${projectVersion}.jar</jar>
<spark-opts>
--executor-memory=${sparkExecutorMemory}
--executor-cores=${sparkExecutorCores}
--driver-memory=${sparkDriverMemory}
--conf spark.extraListeners=${spark2ExtraListeners}
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
--conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir}
</spark-opts>
<arg>--sourcePath</arg><arg>${sourcePath}/dataset</arg>
<arg>--resultTableName</arg><arg>eu.dnetlib.dhp.schema.oaf.Dataset</arg>
<arg>--outputPath</arg><arg>${workingDir}/result/dataset</arg>
<arg>--communityMapPath</arg><arg>${communityMapPath}</arg>
</spark>
<ok to="join_dump"/>
<error to="Kill"/>
</action>
<action name="dump_orp">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn</master>
<mode>cluster</mode>
<name>Dump table ORP </name>
<class>eu.dnetlib.dhp.oa.graph.dump.complete.SparkDumpEntitiesJob</class>
<jar>dhp-graph-mapper-${projectVersion}.jar</jar>
<spark-opts>
--executor-memory=${sparkExecutorMemory}
--executor-cores=${sparkExecutorCores}
--driver-memory=${sparkDriverMemory}
--conf spark.extraListeners=${spark2ExtraListeners}
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
--conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir}
</spark-opts>
<arg>--sourcePath</arg><arg>${sourcePath}/otherresearchproduct</arg>
<arg>--resultTableName</arg><arg>eu.dnetlib.dhp.schema.oaf.OtherResearchProduct</arg>
<arg>--outputPath</arg><arg>${workingDir}/result/otherresearchproduct</arg>
<arg>--communityMapPath</arg><arg>${communityMapPath}</arg>
</spark>
<ok to="join_dump"/>
<error to="Kill"/>
</action>
<action name="dump_software">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn</master>
<mode>cluster</mode>
<name>Dump table software </name>
<class>eu.dnetlib.dhp.oa.graph.dump.complete.SparkDumpEntitiesJob</class>
<jar>dhp-graph-mapper-${projectVersion}.jar</jar>
<spark-opts>
--executor-memory=${sparkExecutorMemory}
--executor-cores=${sparkExecutorCores}
--driver-memory=${sparkDriverMemory}
--conf spark.extraListeners=${spark2ExtraListeners}
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
--conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir}
</spark-opts>
<arg>--sourcePath</arg><arg>${sourcePath}/software</arg>
<arg>--resultTableName</arg><arg>eu.dnetlib.dhp.schema.oaf.Software</arg>
<arg>--outputPath</arg><arg>${workingDir}/result/software</arg>
<arg>--communityMapPath</arg><arg>${communityMapPath}</arg>
</spark>
<ok to="join_dump"/>
<error to="Kill"/>
</action>
<action name="dump_organization">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn</master>
<mode>cluster</mode>
<name>Dump table organization </name>
<class>eu.dnetlib.dhp.oa.graph.dump.complete.SparkDumpEntitiesJob</class>
<jar>dhp-graph-mapper-${projectVersion}.jar</jar>
<spark-opts>
--executor-memory=${sparkExecutorMemory}
--executor-cores=${sparkExecutorCores}
--driver-memory=${sparkDriverMemory}
--conf spark.extraListeners=${spark2ExtraListeners}
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
--conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir}
</spark-opts>
<arg>--sourcePath</arg><arg>${sourcePath}/organization</arg>
<arg>--resultTableName</arg><arg>eu.dnetlib.dhp.schema.oaf.Organization</arg>
<arg>--outputPath</arg><arg>${outputPath}/organization</arg>
<arg>--communityMapPath</arg><arg>${communityMapPath}</arg>
</spark>
<ok to="join_dump"/>
<error to="Kill"/>
</action>
<action name="dump_project">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn</master>
<mode>cluster</mode>
<name>Dump table project </name>
<class>eu.dnetlib.dhp.oa.graph.dump.complete.SparkDumpEntitiesJob</class>
<jar>dhp-graph-mapper-${projectVersion}.jar</jar>
<spark-opts>
--executor-memory=${sparkExecutorMemory}
--executor-cores=${sparkExecutorCores}
--driver-memory=${sparkDriverMemory}
--conf spark.extraListeners=${spark2ExtraListeners}
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
--conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir}
</spark-opts>
<arg>--sourcePath</arg><arg>${sourcePath}/project</arg>
<arg>--resultTableName</arg><arg>eu.dnetlib.dhp.schema.oaf.Project</arg>
<arg>--outputPath</arg><arg>${outputPath}/project</arg>
<arg>--communityMapPath</arg><arg>${communityMapPath}</arg>
</spark>
<ok to="join_dump"/>
<error to="Kill"/>
</action>
<action name="dump_datasource">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn</master>
<mode>cluster</mode>
<name>Dump table datasource </name>
<class>eu.dnetlib.dhp.oa.graph.dump.complete.SparkDumpEntitiesJob</class>
<jar>dhp-graph-mapper-${projectVersion}.jar</jar>
<spark-opts>
--executor-memory=${sparkExecutorMemory}
--executor-cores=${sparkExecutorCores}
--driver-memory=${sparkDriverMemory}
--conf spark.extraListeners=${spark2ExtraListeners}
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
--conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir}
</spark-opts>
<arg>--sourcePath</arg><arg>${sourcePath}/datasource</arg>
<arg>--resultTableName</arg><arg>eu.dnetlib.dhp.schema.oaf.Datasource</arg>
<arg>--outputPath</arg><arg>${outputPath}/datasource</arg>
<arg>--communityMapPath</arg><arg>${workingDir}/communityMap</arg>
</spark>
<ok to="join_dump"/>
<error to="Kill"/>
</action>
<action name="select_relation">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn</master>
<mode>cluster</mode>
<name>Select valid table relation </name>
<class>eu.dnetlib.dhp.oa.graph.dump.complete.SparkSelectValidRelationsJob</class>
<jar>dhp-graph-mapper-${projectVersion}.jar</jar>
<spark-opts>
--executor-memory=${sparkExecutorMemory}
--executor-cores=${sparkExecutorCores}
--driver-memory=${sparkDriverMemory}
--conf spark.extraListeners=${spark2ExtraListeners}
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
--conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir}
</spark-opts>
<arg>--sourcePath</arg><arg>${sourcePath}</arg>
<arg>--outputPath</arg><arg>${workingDir}/validrelation</arg>
</spark>
<ok to="dump_relation"/>
<error to="Kill"/>
</action>
<action name="dump_relation">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn</master>
<mode>cluster</mode>
<name>Dump table relation </name>
<class>eu.dnetlib.dhp.oa.graph.dump.complete.SparkDumpRelationJob</class>
<jar>dhp-graph-mapper-${projectVersion}.jar</jar>
<spark-opts>
--executor-memory=${sparkExecutorMemory}
--executor-cores=${sparkExecutorCores}
--driver-memory=${sparkDriverMemory}
--conf spark.extraListeners=${spark2ExtraListeners}
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
--conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir}
</spark-opts>
<arg>--sourcePath</arg><arg>${workingDir}/validrelation</arg>
<arg>--outputPath</arg><arg>${workingDir}/relation/relation</arg>
</spark>
<ok to="join_dump"/>
<error to="Kill"/>
</action>
<join name="join_dump" to="fork_context"/>
<fork name="fork_context">
<path start="create_entities_fromcontext"/>
<path start="create_relation_fromcontext"/>
<path start="create_relation_fromorgs"/>
</fork>
<action name="create_entities_fromcontext">
<java>
<main-class>eu.dnetlib.dhp.oa.graph.dump.complete.CreateContextEntities</main-class>
<arg>--hdfsPath</arg><arg>${outputPath}/communities_infrastructures/communities_infrastructure.json.gz</arg>
<arg>--nameNode</arg><arg>${nameNode}</arg>
<arg>--isLookUpUrl</arg><arg>${isLookUpUrl}</arg>
</java>
<ok to="join_context"/>
<error to="Kill"/>
</action>
<action name="create_relation_fromcontext">
<java>
<main-class>eu.dnetlib.dhp.oa.graph.dump.complete.CreateContextRelation</main-class>
<arg>--hdfsPath</arg><arg>${workingDir}/relation/context</arg>
<arg>--nameNode</arg><arg>${nameNode}</arg>
<arg>--isLookUpUrl</arg><arg>${isLookUpUrl}</arg>
</java>
<ok to="join_context"/>
<error to="Kill"/>
</action>
<action name="create_relation_fromorgs">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn</master>
<mode>cluster</mode>
<name>Dump table relation </name>
<class>eu.dnetlib.dhp.oa.graph.dump.complete.SparkOrganizationRelation</class>
<jar>dhp-graph-mapper-${projectVersion}.jar</jar>
<spark-opts>
--executor-memory=${sparkExecutorMemory}
--executor-cores=${sparkExecutorCores}
--driver-memory=${sparkDriverMemory}
--conf spark.extraListeners=${spark2ExtraListeners}
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
--conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir}
</spark-opts>
<arg>--sourcePath</arg><arg>${sourcePath}/relation</arg>
<arg>--outputPath</arg><arg>${workingDir}/relation/contextOrg</arg>
<arg>--organizationCommunityMap</arg><arg>${organizationCommunityMap}</arg>
<arg>--communityMapPath</arg><arg>${communityMapPath}</arg>
</spark>
<ok to="join_context"/>
<error to="Kill"/>
</action>
<join name="join_context" to="fork_extract_relations"/>
<fork name="fork_extract_relations">
<path start="rels_from_pubs"/>
<path start="rels_from_dats"/>
<path start="rels_from_orp"/>
<path start="rels_from_sw"/>
</fork>
<action name="rels_from_pubs">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn</master>
<mode>cluster</mode>
<name>Extract Relations from publication </name>
<class>eu.dnetlib.dhp.oa.graph.dump.complete.SparkExtractRelationFromEntities</class>
<jar>dhp-graph-mapper-${projectVersion}.jar</jar>
<spark-opts>
--executor-memory=${sparkExecutorMemory}
--executor-cores=${sparkExecutorCores}
--driver-memory=${sparkDriverMemory}
--conf spark.extraListeners=${spark2ExtraListeners}
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
--conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir}
</spark-opts>
<arg>--sourcePath</arg><arg>${sourcePath}/publication</arg>
<arg>--resultTableName</arg><arg>eu.dnetlib.dhp.schema.oaf.Publication</arg>
<arg>--outputPath</arg><arg>${workingDir}/relation/publication</arg>
<arg>--communityMapPath</arg><arg>${communityMapPath}</arg>
</spark>
<ok to="join_extract_relations"/>
<error to="Kill"/>
</action>
<action name="rels_from_dats">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn</master>
<mode>cluster</mode>
<name>Dump table dataset </name>
<class>eu.dnetlib.dhp.oa.graph.dump.complete.SparkExtractRelationFromEntities</class>
<jar>dhp-graph-mapper-${projectVersion}.jar</jar>
<spark-opts>
--executor-memory=${sparkExecutorMemory}
--executor-cores=${sparkExecutorCores}
--driver-memory=${sparkDriverMemory}
--conf spark.extraListeners=${spark2ExtraListeners}
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
--conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir}
</spark-opts>
<arg>--sourcePath</arg><arg>${sourcePath}/dataset</arg>
<arg>--resultTableName</arg><arg>eu.dnetlib.dhp.schema.oaf.Dataset</arg>
<arg>--outputPath</arg><arg>${workingDir}/relation/dataset</arg>
<arg>--communityMapPath</arg><arg>${communityMapPath}</arg>
</spark>
<ok to="join_extract_relations"/>
<error to="Kill"/>
</action>
<action name="rels_from_orp">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn</master>
<mode>cluster</mode>
<name>Dump table ORP </name>
<class>eu.dnetlib.dhp.oa.graph.dump.complete.SparkExtractRelationFromEntities</class>
<jar>dhp-graph-mapper-${projectVersion}.jar</jar>
<spark-opts>
--executor-memory=${sparkExecutorMemory}
--executor-cores=${sparkExecutorCores}
--driver-memory=${sparkDriverMemory}
--conf spark.extraListeners=${spark2ExtraListeners}
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
--conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir}
</spark-opts>
<arg>--sourcePath</arg><arg>${sourcePath}/otherresearchproduct</arg>
<arg>--resultTableName</arg><arg>eu.dnetlib.dhp.schema.oaf.OtherResearchProduct</arg>
<arg>--outputPath</arg><arg>${workingDir}/relation/orp</arg>
<arg>--communityMapPath</arg><arg>${communityMapPath}</arg>
</spark>
<ok to="join_extract_relations"/>
<error to="Kill"/>
</action>
<action name="rels_from_sw">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn</master>
<mode>cluster</mode>
<name>Dump table software </name>
<class>eu.dnetlib.dhp.oa.graph.dump.complete.SparkExtractRelationFromEntities</class>
<jar>dhp-graph-mapper-${projectVersion}.jar</jar>
<spark-opts>
--executor-memory=${sparkExecutorMemory}
--executor-cores=${sparkExecutorCores}
--driver-memory=${sparkDriverMemory}
--conf spark.extraListeners=${spark2ExtraListeners}
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
--conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir}
</spark-opts>
<arg>--sourcePath</arg><arg>${sourcePath}/software</arg>
<arg>--resultTableName</arg><arg>eu.dnetlib.dhp.schema.oaf.Software</arg>
<arg>--outputPath</arg><arg>${workingDir}/relation/software</arg>
<arg>--communityMapPath</arg><arg>${communityMapPath}</arg>
</spark>
<ok to="join_extract_relations"/>
<error to="Kill"/>
</action>
<join name="join_extract_relations" to="collect_and_save"/>
<action name="collect_and_save">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn</master>
<mode>cluster</mode>
<name>Collect Results and Relations and put them in the right path </name>
<class>eu.dnetlib.dhp.oa.graph.dump.complete.SparkCollectAndSave</class>
<jar>dhp-graph-mapper-${projectVersion}.jar</jar>
<spark-opts>
--executor-memory=${sparkExecutorMemory}
--executor-cores=${sparkExecutorCores}
--driver-memory=${sparkDriverMemory}
--conf spark.extraListeners=${spark2ExtraListeners}
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
--conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir}
</spark-opts>
<arg>--sourcePath</arg><arg>${workingDir}</arg>
<arg>--outputPath</arg><arg>${outputPath}</arg>
<arg>--resultAggregation</arg><arg>${resultAggregation}</arg>
</spark>
<ok to="End"/>
<error to="Kill"/>
</action>
<kill name="Kill">
<message>Sub-workflow dump complete failed with error message ${wf:errorMessage()}
</message>
</kill>
<end name="End" />
</workflow-app>

View File

@ -0,0 +1,30 @@
<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>hiveMetastoreUris</name>
<value>thrift://iis-cdh5-test-m3.ocean.icm.edu.pl:9083</value>
</property>
<property>
<name>hiveJdbcUrl</name>
<value>jdbc:hive2://iis-cdh5-test-m3.ocean.icm.edu.pl:10000</value>
</property>
<property>
<name>hiveDbName</name>
<value>openaire</value>
</property>
<property>
<name>oozie.launcher.mapreduce.user.classpath.first</name>
<value>true</value>
</property>
</configuration>

View File

@ -0,0 +1,2 @@
## This is a classpath-based import file (this header is required)
dump_common classpath eu/dnetlib/dhp/oa/graph/dump/wf/subworkflows/commoncommunityfunder/oozie_app

View File

@ -0,0 +1,256 @@
<workflow-app name="sub_dump_funder_results" xmlns="uri:oozie:workflow:0.5">
<parameters>
<property>
<name>sourcePath</name>
<description>the source path</description>
</property>
<property>
<name>outputPath</name>
<description>the output path</description>
</property>
<property>
<name>hiveDbName</name>
<description>the target hive database name</description>
</property>
<property>
<name>hiveJdbcUrl</name>
<description>hive server jdbc url</description>
</property>
<property>
<name>hiveMetastoreUris</name>
<description>hive server metastore URIs</description>
</property>
<property>
<name>sparkDriverMemory</name>
<description>memory for driver process</description>
</property>
<property>
<name>sparkExecutorMemory</name>
<description>memory for individual executor</description>
</property>
<property>
<name>sparkExecutorCores</name>
<description>number of cores used by single executor</description>
</property>
<property>
<name>oozieActionShareLibForSpark2</name>
<description>oozie action sharelib for spark 2.*</description>
</property>
<property>
<name>spark2ExtraListeners</name>
<value>com.cloudera.spark.lineage.NavigatorAppListener</value>
<description>spark 2.* extra listeners classname</description>
</property>
<property>
<name>spark2SqlQueryExecutionListeners</name>
<value>com.cloudera.spark.lineage.NavigatorQueryListener</value>
<description>spark 2.* sql query execution listeners classname</description>
</property>
<property>
<name>spark2YarnHistoryServerAddress</name>
<description>spark 2.* yarn history server address</description>
</property>
<property>
<name>spark2EventLogDir</name>
<description>spark 2.* event log dir location</description>
</property>
</parameters>
<global>
<job-tracker>${jobTracker}</job-tracker>
<name-node>${nameNode}</name-node>
<configuration>
<property>
<name>mapreduce.job.queuename</name>
<value>${queueName}</value>
</property>
<property>
<name>oozie.launcher.mapred.job.queue.name</name>
<value>${oozieLauncherQueueName}</value>
</property>
<property>
<name>oozie.action.sharelib.for.spark</name>
<value>${oozieActionShareLibForSpark2}</value>
</property>
</configuration>
</global>
<start to="fork_result_linked_to_projects"/>
<kill name="Kill">
<message>Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}]</message>
</kill>
<fork name="fork_result_linked_to_projects">
<path start="select_publication_linked_to_projects"/>
<path start="select_dataset_linked_to_projects"/>
<path start="select_orp_linked_to_project"/>
<path start="select_software_linked_to_projects"/>
</fork>
<action name="select_publication_linked_to_projects">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn</master>
<mode>cluster</mode>
<name>Dump funder results </name>
<class>eu.dnetlib.dhp.oa.graph.dump.funderresults.SparkResultLinkedToProject</class>
<jar>dhp-graph-mapper-${projectVersion}.jar</jar>
<spark-opts>
--executor-memory=${sparkExecutorMemory}
--executor-cores=${sparkExecutorCores}
--driver-memory=${sparkDriverMemory}
--conf spark.extraListeners=${spark2ExtraListeners}
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
--conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir}
</spark-opts>
<arg>--sourcePath</arg><arg>${sourcePath}/publication</arg>
<arg>--resultTableName</arg><arg>eu.dnetlib.dhp.schema.oaf.Publication</arg>
<arg>--outputPath</arg><arg>${workingDir}/result/publication</arg>
<arg>--graphPath</arg><arg>${sourcePath}</arg>
</spark>
<ok to="join_link"/>
<error to="Kill"/>
</action>
<action name="select_dataset_linked_to_projects">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn</master>
<mode>cluster</mode>
<name>Dump funder results </name>
<class>eu.dnetlib.dhp.oa.graph.dump.funderresults.SparkResultLinkedToProject</class>
<jar>dhp-graph-mapper-${projectVersion}.jar</jar>
<spark-opts>
--executor-memory=${sparkExecutorMemory}
--executor-cores=${sparkExecutorCores}
--driver-memory=${sparkDriverMemory}
--conf spark.extraListeners=${spark2ExtraListeners}
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
--conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir}
</spark-opts>
<arg>--sourcePath</arg><arg>${sourcePath}/dataset</arg>
<arg>--resultTableName</arg><arg>eu.dnetlib.dhp.schema.oaf.Dataset</arg>
<arg>--outputPath</arg><arg>${workingDir}/result/dataset</arg>
<arg>--graphPath</arg><arg>${sourcePath}</arg>
</spark>
<ok to="join_link"/>
<error to="Kill"/>
</action>
<action name="select_orp_linked_to_project">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn</master>
<mode>cluster</mode>
<name>Dump funder results </name>
<class>eu.dnetlib.dhp.oa.graph.dump.funderresults.SparkResultLinkedToProject</class>
<jar>dhp-graph-mapper-${projectVersion}.jar</jar>
<spark-opts>
--executor-memory=${sparkExecutorMemory}
--executor-cores=${sparkExecutorCores}
--driver-memory=${sparkDriverMemory}
--conf spark.extraListeners=${spark2ExtraListeners}
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
--conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir}
</spark-opts>
<arg>--sourcePath</arg><arg>${sourcePath}/otherresearchproduct</arg>
<arg>--resultTableName</arg><arg>eu.dnetlib.dhp.schema.oaf.OtherResearchProduct</arg>
<arg>--outputPath</arg><arg>${workingDir}/result/otherresearchproduct</arg>
<arg>--graphPath</arg><arg>${sourcePath}</arg>
</spark>
<ok to="join_link"/>
<error to="Kill"/>
</action>
<action name="select_software_linked_to_projects">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn</master>
<mode>cluster</mode>
<name>Dump funder results </name>
<class>eu.dnetlib.dhp.oa.graph.dump.funderresults.SparkResultLinkedToProject</class>
<jar>dhp-graph-mapper-${projectVersion}.jar</jar>
<spark-opts>
--executor-memory=${sparkExecutorMemory}
--executor-cores=${sparkExecutorCores}
--driver-memory=${sparkDriverMemory}
--conf spark.extraListeners=${spark2ExtraListeners}
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
--conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir}
</spark-opts>
<arg>--sourcePath</arg><arg>${sourcePath}/software</arg>
<arg>--resultTableName</arg><arg>eu.dnetlib.dhp.schema.oaf.Software</arg>
<arg>--outputPath</arg><arg>${workingDir}/result/software</arg>
<arg>--graphPath</arg><arg>${sourcePath}</arg>
</spark>
<ok to="join_link"/>
<error to="Kill"/>
</action>
<join name="join_link" to="common_action_community_funder"/>
<action name="common_action_community_funder">
<sub-workflow>
<app-path>${wf:appPath()}/dump_common
</app-path>
<propagate-configuration/>
<configuration>
<property>
<name>sourcePath</name>
<value>${sourcePath}</value>
</property>
<property>
<name>selectedResults</name>
<value>${workingDir}/result</value>
</property>
<property>
<name>communityMapPath</name>
<value>${workingDir}/communityMap</value>
</property>
<property>
<name>outputPath</name>
<value>${workingDir}</value>
</property>
</configuration>
</sub-workflow>
<ok to="dump_funder_results" />
<error to="Kill" />
</action>
<action name="dump_funder_results">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn</master>
<mode>cluster</mode>
<name>Dump funder results </name>
<class>eu.dnetlib.dhp.oa.graph.dump.funderresults.SparkDumpFunderResults</class>
<jar>dhp-graph-mapper-${projectVersion}.jar</jar>
<spark-opts>
--executor-memory=${sparkExecutorMemory}
--executor-cores=${sparkExecutorCores}
--driver-memory=${sparkDriverMemory}
--conf spark.extraListeners=${spark2ExtraListeners}
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
--conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir}
</spark-opts>
<arg>--sourcePath</arg><arg>${workingDir}/ext</arg>
<arg>--outputPath</arg><arg>${outputPath}</arg>
<arg>--graphPath</arg><arg>${sourcePath}</arg>
</spark>
<ok to="End"/>
<error to="Kill"/>
</action>
<end name="End"/>
</workflow-app>

View File

@ -544,6 +544,33 @@
<arg>--sourcePath</arg><arg>${workingDir}/entities</arg>
<arg>--graphRawPath</arg><arg>${workingDir}/graph_raw</arg>
</spark>
<ok to="Import_oaf_store_graph"/>
<error to="Kill"/>
</action>
<action name="Import_oaf_store_graph">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn</master>
<mode>cluster</mode>
<name>ImportOAF_hdfs_graph</name>
<class>eu.dnetlib.dhp.oa.graph.raw.CopyHdfsOafApplication</class>
<jar>dhp-graph-mapper-${projectVersion}.jar</jar>
<spark-opts>
--executor-memory ${sparkExecutorMemory}
--executor-cores ${sparkExecutorCores}
--driver-memory=${sparkDriverMemory}
--conf spark.extraListeners=${spark2ExtraListeners}
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
</spark-opts>
<arg>--hdfsPath</arg><arg>${workingDir}/graph_raw</arg>
<arg>--mdstoreManagerUrl</arg><arg>${mdstoreManagerUrl}</arg>
<arg>--mdFormat</arg><arg>OAF</arg>
<arg>--mdLayout</arg><arg>store</arg>
<arg>--mdInterpretation</arg><arg>graph</arg>
<arg>--isLookupUrl</arg><arg>${isLookupUrl}</arg>
</spark>
<ok to="wait_graphs"/>
<error to="Kill"/>
</action>

View File

@ -1,177 +0,0 @@
<workflow-app name="Transform_BioEntity_Workflow" xmlns="uri:oozie:workflow:0.5">
<parameters>
<property>
<name>PDBPath</name>
<description>the PDB Database Working Path</description>
</property>
<property>
<name>UNIPROTDBPath</name>
<description>the UNIPROT Database Working Path</description>
</property>
<property>
<name>EBIDataset</name>
<description>the EBI Links Dataset Path</description>
</property>
<property>
<name>ScholixResolvedDBPath</name>
<description>the Scholix Resolved Dataset Path</description>
</property>
<property>
<name>CrossrefLinksPath</name>
<description>the CrossrefLinks Path</description>
</property>
<property>
<name>targetPath</name>
<description>the Target Working dir path</description>
</property>
</parameters>
<start to="ConvertPDB"/>
<kill name="Kill">
<message>Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}]</message>
</kill>
<action name="ConvertPDB">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn</master>
<mode>cluster</mode>
<name>Convert PDB to OAF Dataset</name>
<class>eu.dnetlib.dhp.sx.graph.bio.SparkTransformBioDatabaseToOAF</class>
<jar>dhp-graph-mapper-${projectVersion}.jar</jar>
<spark-opts>
--executor-memory=${sparkExecutorMemory}
--executor-cores=${sparkExecutorCores}
--driver-memory=${sparkDriverMemory}
--conf spark.extraListeners=${spark2ExtraListeners}
--conf spark.sql.shuffle.partitions=2000
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
</spark-opts>
<arg>--master</arg><arg>yarn</arg>
<arg>--dbPath</arg><arg>${PDBPath}</arg>
<arg>--database</arg><arg>PDB</arg>
<arg>--targetPath</arg><arg>${targetPath}/pdb_OAF</arg>
</spark>
<ok to="ConvertUNIPROT"/>
<error to="Kill"/>
</action>
<action name="ConvertUNIPROT">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn</master>
<mode>cluster</mode>
<name>Convert UNIPROT to OAF Dataset</name>
<class>eu.dnetlib.dhp.sx.graph.bio.SparkTransformBioDatabaseToOAF</class>
<jar>dhp-graph-mapper-${projectVersion}.jar</jar>
<spark-opts>
--executor-memory=${sparkExecutorMemory}
--executor-cores=${sparkExecutorCores}
--driver-memory=${sparkDriverMemory}
--conf spark.extraListeners=${spark2ExtraListeners}
--conf spark.sql.shuffle.partitions=2000
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
</spark-opts>
<arg>--master</arg><arg>yarn</arg>
<arg>--dbPath</arg><arg>${UNIPROTDBPath}</arg>
<arg>--database</arg><arg>UNIPROT</arg>
<arg>--targetPath</arg><arg>${targetPath}/uniprot_OAF</arg>
</spark>
<ok to="ConvertEBILinks"/>
<error to="Kill"/>
</action>
<action name="ConvertEBILinks">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn</master>
<mode>cluster</mode>
<name>Convert EBI Links to OAF Dataset</name>
<class>eu.dnetlib.dhp.sx.graph.ebi.SparkEBILinksToOaf</class>
<jar>dhp-graph-mapper-${projectVersion}.jar</jar>
<spark-opts>
--executor-memory=${sparkExecutorMemory}
--executor-cores=${sparkExecutorCores}
--driver-memory=${sparkDriverMemory}
--conf spark.extraListeners=${spark2ExtraListeners}
--conf spark.sql.shuffle.partitions=2000
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
</spark-opts>
<arg>--master</arg><arg>yarn</arg>
<arg>--sourcePath</arg><arg>${EBIDataset}</arg>
<arg>--targetPath</arg><arg>${targetPath}/ebi_OAF</arg>
</spark>
<ok to="ConvertScholixResolved"/>
<error to="Kill"/>
</action>
<action name="ConvertScholixResolved">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn</master>
<mode>cluster</mode>
<name>Convert Scholix to OAF Dataset</name>
<class>eu.dnetlib.dhp.sx.graph.bio.SparkTransformBioDatabaseToOAF</class>
<jar>dhp-graph-mapper-${projectVersion}.jar</jar>
<spark-opts>
--executor-memory=${sparkExecutorMemory}
--executor-cores=${sparkExecutorCores}
--driver-memory=${sparkDriverMemory}
--conf spark.extraListeners=${spark2ExtraListeners}
--conf spark.sql.shuffle.partitions=2000
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
</spark-opts>
<arg>--master</arg><arg>yarn</arg>
<arg>--dbPath</arg><arg>${ScholixResolvedDBPath}</arg>
<arg>--database</arg><arg>SCHOLIX</arg>
<arg>--targetPath</arg><arg>${targetPath}/scholix_resolved_OAF</arg>
</spark>
<ok to="ConvertCrossrefLinks"/>
<error to="Kill"/>
</action>
<action name="ConvertCrossrefLinks">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn</master>
<mode>cluster</mode>
<name>Convert Crossref Links to OAF Dataset</name>
<class>eu.dnetlib.dhp.sx.graph.bio.SparkTransformBioDatabaseToOAF</class>
<jar>dhp-graph-mapper-${projectVersion}.jar</jar>
<spark-opts>
--executor-memory=${sparkExecutorMemory}
--executor-cores=${sparkExecutorCores}
--driver-memory=${sparkDriverMemory}
--conf spark.extraListeners=${spark2ExtraListeners}
--conf spark.sql.shuffle.partitions=2000
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
</spark-opts>
<arg>--master</arg><arg>yarn</arg>
<arg>--dbPath</arg><arg>${CrossrefLinksPath}</arg>
<arg>--database</arg><arg>CROSSREF_LINKS</arg>
<arg>--targetPath</arg><arg>${targetPath}/crossref_unresolved_relation_OAF</arg>
</spark>
<ok to="End"/>
<error to="Kill"/>
</action>
<end name="End"/>
</workflow-app>

View File

@ -1,99 +0,0 @@
<workflow-app name="Create EBI Dataset" xmlns="uri:oozie:workflow:0.5">
<parameters>
<property>
<name>workingPath</name>
<description>the Working Path</description>
</property>
<property>
<name>sparkDriverMemory</name>
<description>memory for driver process</description>
</property>
<property>
<name>sparkExecutorMemory</name>
<description>memory for individual executor</description>
</property>
<property>
<name>sparkExecutorCores</name>
<description>number of cores used by single executor</description>
</property>
</parameters>
<start to="CreateEBIDataSet"/>
<kill name="Kill">
<message>Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}]</message>
</kill>
<action name="GenerateBaselineDataset">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn-cluster</master>
<mode>cluster</mode>
<name>Create Baselnie DataSet</name>
<class>eu.dnetlib.dhp.sx.ebi.SparkCreateBaselineDataFrame</class>
<jar>dhp-graph-mapper-${projectVersion}.jar</jar>
<spark-opts>
--executor-memory=${sparkExecutorMemory}
--executor-cores=1
--driver-memory=${sparkDriverMemory}
--executor-cores=${sparkExecutorCores}
${sparkExtraOPT}
</spark-opts>
<arg>--workingPath</arg><arg>${workingPath}</arg>
<arg>--master</arg><arg>yarn</arg>
<arg>--hdfsServerUri</arg><arg>${nameNode}</arg>
</spark>
<ok to="End"/>
<error to="Kill"/>
</action>
<action name="CreateEBIDataSet">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn-cluster</master>
<mode>cluster</mode>
<name>Create EBI DataSet</name>
<class>eu.dnetlib.dhp.sx.ebi.SparkEBILinksToOaf</class>
<jar>dhp-graph-mapper-${projectVersion}.jar</jar>
<spark-opts>
--executor-memory=${sparkExecutorMemory}
--executor-cores=${sparkExecutorCores}
--driver-memory=${sparkDriverMemory}
--conf spark.sql.shuffle.partitions=1000
${sparkExtraOPT}
</spark-opts>
<arg>--workingPath</arg><arg>${workingPath}</arg>
<arg>--master</arg><arg>yarn</arg>
</spark>
<ok to="GenerateUpdates"/>
<error to="Kill"/>
</action>
<action name="GenerateUpdates">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn-cluster</master>
<mode>cluster</mode>
<name>Create Baseline DataSet</name>
<class>eu.dnetlib.dhp.sx.ebi.SparkAddLinkUpdates</class>
<jar>dhp-graph-mapper-${projectVersion}.jar</jar>
<spark-opts>
--executor-memory=${sparkExecutorMemory}
--executor-cores=1
--driver-memory=${sparkDriverMemory}
--executor-cores=${sparkExecutorCores}
${sparkExtraOPT}
</spark-opts>
<arg>--workingPath</arg><arg>${workingPath}</arg>
<arg>--master</arg><arg>yarn</arg>
</spark>
<ok to="End"/>
<error to="Kill"/>
</action>
<end name="End"/>
</workflow-app>

View File

@ -1,68 +0,0 @@
<configuration>
<!-- OCEAN -->
<property>
<name>jobTracker</name>
<value>yarnRM</value>
</property>
<property>
<name>nameNode</name>
<value>hdfs://nameservice1</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>
<!-- GARR -->
<!-- <property>-->
<!-- <name>jobTracker</name>-->
<!-- <value>yarn</value>-->
<!-- </property>-->
<!-- <property>-->
<!-- <name>nameNode</name>-->
<!-- <value>hdfs://hadoop-rm1.garr-pa1.d4science.org:8020</value>-->
<!-- </property>-->
<!-- <property>-->
<!-- <name>hive_metastore_uris</name>-->
<!-- <value>thrift://hadoop-edge3.garr-pa1.d4science.org:9083</value>-->
<!-- </property>-->
<!-- <property>-->
<!-- <name>spark2YarnHistoryServerAddress</name>-->
<!-- <value>http://hadoop-rm2.garr-pa1.d4science.org:19888</value>-->
<!-- </property>-->
<property>
<name>oozie.launcher.mapreduce.user.classpath.first</name>
<value>true</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>spark2EventLogDir</name>
<value>/user/spark/spark2ApplicationHistory</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>
</configuration>

View File

@ -1,67 +0,0 @@
<workflow-app name="Create EBI Dataset" xmlns="uri:oozie:workflow:0.5">
<parameters>
<property>
<name>sourcePath</name>
<description>the Working Path</description>
</property>
<property>
<name>workingPath</name>
<description>the Working Path</description>
</property>
<property>
<name>sparkDriverMemory</name>
<description>memory for driver process</description>
</property>
<property>
<name>sparkExecutorMemory</name>
<description>memory for individual executor</description>
</property>
<property>
<name>sparkExecutorCores</name>
<description>number of cores used by single executor</description>
</property>
</parameters>
<start to="DownloadEBILinks"/>
<kill name="Kill">
<message>Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}]</message>
</kill>
<action name="DownloadEBILinks">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn-cluster</master>
<mode>cluster</mode>
<name>Incremental Download EBI Links</name>
<class>eu.dnetlib.dhp.sx.graph.ebi.SparkDownloadEBILinks</class>
<jar>dhp-graph-mapper-${projectVersion}.jar</jar>
<spark-opts>
--executor-memory=${sparkExecutorMemory}
--executor-cores=${sparkExecutorCores}
--driver-memory=${sparkDriverMemory}
--conf spark.extraListeners=${spark2ExtraListeners}
--conf spark.sql.shuffle.partitions=2000
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
</spark-opts>
<arg>--sourcePath</arg><arg>${sourcePath}</arg>
<arg>--workingPath</arg><arg>${workingPath}</arg>
<arg>--master</arg><arg>yarn</arg>
</spark>
<ok to="OverrideFolders"/>
<error to="Kill"/>
</action>
<action name="OverrideFolders">
<fs>
<delete path="${sourcePath}/ebi_links_dataset_old"/>
<move source="${sourcePath}/ebi_links_dataset" target="${sourcePath}/ebi_links_dataset_old"/>
<move source="${workingPath}/links_final" target="${sourcePath}/ebi_links_dataset"/>
</fs>
<ok to="End"/>
<error to="Kill"/>
</action>
<end name="End"/>
</workflow-app>

View File

@ -11,17 +11,21 @@ import org.apache.commons.io.FileUtils;
import org.apache.spark.SparkConf;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.api.java.function.ForeachFunction;
import org.apache.spark.sql.Encoders;
import org.apache.spark.sql.Row;
import org.apache.spark.sql.SparkSession;
import org.junit.jupiter.api.*;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import com.fasterxml.jackson.core.JsonProcessingException;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.gson.Gson;
import eu.dnetlib.dhp.oa.graph.dump.community.CommunityMap;
import eu.dnetlib.dhp.schema.common.ModelConstants;
import eu.dnetlib.dhp.schema.dump.oaf.Instance;
import eu.dnetlib.dhp.schema.dump.oaf.OpenAccessRoute;
import eu.dnetlib.dhp.schema.dump.oaf.community.CommunityResult;
import eu.dnetlib.dhp.schema.dump.oaf.graph.GraphResult;
import eu.dnetlib.dhp.schema.oaf.Dataset;
@ -137,6 +141,512 @@ public class DumpJobTest {
System.out.println(new Gson().toJson(map));
}
@Test
public void testPublicationDump() {
final String sourcePath = getClass()
.getResource("/eu/dnetlib/dhp/oa/graph/dump/resultDump/publication_extendedinstance")
.getPath();
final String communityMapPath = getClass()
.getResource("/eu/dnetlib/dhp/oa/graph/dump/communityMapPath/communitymap.json")
.getPath();
DumpProducts dump = new DumpProducts();
dump
.run(
// false, sourcePath, workingDir.toString() + "/result", communityMapPath, Publication.class,
false, sourcePath, workingDir.toString() + "/result", communityMapPath, Publication.class,
GraphResult.class, Constants.DUMPTYPE.COMPLETE.getType());
final JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext());
JavaRDD<GraphResult> tmp = sc
.textFile(workingDir.toString() + "/result")
.map(item -> OBJECT_MAPPER.readValue(item, GraphResult.class));
org.apache.spark.sql.Dataset<GraphResult> verificationDataset = spark
.createDataset(tmp.rdd(), Encoders.bean(GraphResult.class));
Assertions.assertEquals(1, verificationDataset.count());
GraphResult gr = verificationDataset.first();
Assertions.assertEquals(2, gr.getMeasures().size());
Assertions
.assertTrue(
gr
.getMeasures()
.stream()
.anyMatch(
m -> m.getKey().equals("influence")
&& m.getValue().equals("1.62759106106e-08")));
Assertions
.assertTrue(
gr
.getMeasures()
.stream()
.anyMatch(
m -> m.getKey().equals("popularity")
&& m.getValue().equals("0.22519296")));
Assertions.assertEquals(6, gr.getAuthor().size());
Assertions
.assertTrue(
gr
.getAuthor()
.stream()
.anyMatch(
a -> a.getFullname().equals("Nikolaidou,Charitini") &&
a.getName().equals("Charitini") && a.getSurname().equals("Nikolaidou")
&& a.getRank() == 1 && a.getPid() == null));
Assertions
.assertTrue(
gr
.getAuthor()
.stream()
.anyMatch(
a -> a.getFullname().equals("Votsi,Nefta") &&
a.getName().equals("Nefta") && a.getSurname().equals("Votsi")
&& a.getRank() == 2 && a.getPid().getId().getScheme().equals(ModelConstants.ORCID)
&& a.getPid().getId().getValue().equals("0000-0001-6651-1178")
&& a.getPid().getProvenance() != null));
Assertions
.assertTrue(
gr
.getAuthor()
.stream()
.anyMatch(
a -> a.getFullname().equals("Sgardelis,Steanos") &&
a.getName().equals("Steanos") && a.getSurname().equals("Sgardelis")
&& a.getRank() == 3 && a.getPid().getId().getScheme().equals(ModelConstants.ORCID_PENDING)
&& a.getPid().getId().getValue().equals("0000-0001-6651-1178")
&& a.getPid().getProvenance() != null));
Assertions
.assertTrue(
gr
.getAuthor()
.stream()
.anyMatch(
a -> a.getFullname().equals("Halley,John") &&
a.getName().equals("John") && a.getSurname().equals("Halley")
&& a.getRank() == 4 && a.getPid() == null));
Assertions
.assertTrue(
gr
.getAuthor()
.stream()
.anyMatch(
a -> a.getFullname().equals("Pantis,John") &&
a.getName().equals("John") && a.getSurname().equals("Pantis")
&& a.getRank() == 5 && a.getPid().getId().getScheme().equals(ModelConstants.ORCID)
&& a.getPid().getId().getValue().equals("0000-0001-6651-1178")
&& a.getPid().getProvenance() != null));
Assertions
.assertTrue(
gr
.getAuthor()
.stream()
.anyMatch(
a -> a.getFullname().equals("Tsiafouli,Maria") &&
a.getName().equals("Maria") && a.getSurname().equals("Tsiafouli")
&& a.getRank() == 6 && a.getPid().getId().getScheme().equals(ModelConstants.ORCID_PENDING)
&& a.getPid().getId().getValue().equals("0000-0001-6651-1178")
&& a.getPid().getProvenance() != null));
Assertions.assertEquals("publication", gr.getType());
Assertions.assertEquals("eng", gr.getLanguage().getCode());
Assertions.assertEquals("English", gr.getLanguage().getLabel());
Assertions.assertEquals(1, gr.getCountry().size());
Assertions.assertEquals("IT", gr.getCountry().get(0).getCode());
Assertions.assertEquals("Italy", gr.getCountry().get(0).getLabel());
Assertions.assertTrue(gr.getCountry().get(0).getProvenance() == null);
Assertions.assertEquals(12, gr.getSubjects().size());
Assertions
.assertTrue(
gr
.getSubjects()
.stream()
.anyMatch(
s -> s.getSubject().getValue().equals("Ecosystem Services hotspots")
&& s.getSubject().getScheme().equals("ACM") && s.getProvenance() != null &&
s.getProvenance().getProvenance().equals("sysimport:crosswalk:repository")));
Assertions
.assertTrue(
gr
.getSubjects()
.stream()
.anyMatch(
s -> s.getSubject().getValue().equals("Natura 2000")
&& s.getSubject().getScheme().equals("") && s.getProvenance() != null &&
s.getProvenance().getProvenance().equals("sysimport:crosswalk:repository")));
Assertions
.assertEquals(
"Ecosystem Service capacity is higher in areas of multiple designation types",
gr.getMaintitle());
Assertions.assertEquals(null, gr.getSubtitle());
Assertions.assertEquals(1, gr.getDescription().size());
Assertions
.assertTrue(
gr
.getDescription()
.get(0)
.startsWith("The implementation of the Ecosystem Service (ES) concept into practice"));
Assertions
.assertTrue(
gr
.getDescription()
.get(0)
.endsWith(
"start complying with new standards and demands for nature conservation and environmental management."));
Assertions.assertEquals("2017-01-01", gr.getPublicationdate());
Assertions.assertEquals("Pensoft Publishers", gr.getPublisher());
Assertions.assertEquals(null, gr.getEmbargoenddate());
Assertions.assertEquals(1, gr.getSource().size());
Assertions.assertEquals("One Ecosystem 2: e13718", gr.getSource().get(0));
Assertions.assertEquals(1, gr.getFormat().size());
Assertions.assertEquals("text/html", gr.getFormat().get(0));
Assertions.assertEquals(0, gr.getContributor().size());
Assertions.assertEquals(0, gr.getCoverage().size());
Assertions.assertEquals(ModelConstants.ACCESS_RIGHT_OPEN, gr.getBestaccessright().getLabel());
Assertions
.assertEquals(
Constants.accessRightsCoarMap.get(ModelConstants.ACCESS_RIGHT_OPEN), gr.getBestaccessright().getCode());
Assertions.assertEquals(null, gr.getBestaccessright().getOpenAccessRoute());
Assertions.assertEquals("One Ecosystem", gr.getContainer().getName());
Assertions.assertEquals("2367-8194", gr.getContainer().getIssnOnline());
Assertions.assertEquals("", gr.getContainer().getIssnPrinted());
Assertions.assertEquals("", gr.getContainer().getIssnLinking());
Assertions.assertTrue(null == gr.getDocumentationUrl() || gr.getDocumentationUrl().size() == 0);
Assertions.assertTrue(null == gr.getCodeRepositoryUrl());
Assertions.assertEquals(null, gr.getProgrammingLanguage());
Assertions.assertTrue(null == gr.getContactperson() || gr.getContactperson().size() == 0);
Assertions.assertTrue(null == gr.getContactgroup() || gr.getContactgroup().size() == 0);
Assertions.assertTrue(null == gr.getTool() || gr.getTool().size() == 0);
Assertions.assertEquals(null, gr.getSize());
Assertions.assertEquals(null, gr.getVersion());
Assertions.assertTrue(null == gr.getGeolocation() || gr.getGeolocation().size() == 0);
Assertions.assertEquals("50|pensoft_____::00ea4a1cd53806a97d62ea6bf268f2a2", gr.getId());
Assertions.assertEquals(2, gr.getOriginalId().size());
Assertions
.assertTrue(
gr.getOriginalId().contains("50|pensoft_____::00ea4a1cd53806a97d62ea6bf268f2a2")
&& gr.getOriginalId().contains("10.3897/oneeco.2.e13718"));
Assertions.assertEquals(1, gr.getPid().size());
Assertions
.assertTrue(
gr.getPid().get(0).getScheme().equals("doi")
&& gr.getPid().get(0).getValue().equals("10.1016/j.triboint.2014.05.004"));
Assertions.assertEquals("2020-03-23T00:20:51.392Z", gr.getDateofcollection());
Assertions.assertEquals(1, gr.getInstance().size());
Instance instance = gr.getInstance().get(0);
Assertions.assertEquals(0, instance.getPid().size());
Assertions.assertEquals(1, instance.getAlternateIdentifier().size());
Assertions
.assertTrue(
instance.getAlternateIdentifier().get(0).getScheme().equals("doi")
&& instance.getAlternateIdentifier().get(0).getValue().equals("10.3897/oneeco.2.e13718"));
Assertions.assertEquals(null, instance.getLicense());
Assertions
.assertTrue(
instance
.getAccessright()
.getCode()
.equals(
Constants.accessRightsCoarMap
.get(ModelConstants.ACCESS_RIGHT_OPEN)));
Assertions.assertTrue(instance.getAccessright().getLabel().equals(ModelConstants.ACCESS_RIGHT_OPEN));
Assertions.assertTrue(instance.getAccessright().getOpenAccessRoute().equals(OpenAccessRoute.green));
Assertions.assertTrue(instance.getType().equals("Article"));
Assertions.assertEquals(2, instance.getUrl().size());
Assertions
.assertTrue(
instance.getUrl().contains("https://doi.org/10.3897/oneeco.2.e13718")
&& instance.getUrl().contains("https://oneecosystem.pensoft.net/article/13718/"));
Assertions.assertEquals("2017-01-01", instance.getPublicationdate());
Assertions.assertEquals(null, instance.getArticleprocessingcharge());
Assertions.assertEquals("peerReviewed", instance.getRefereed());
}
@Test
public void testDatasetDump() {
final String sourcePath = getClass()
.getResource("/eu/dnetlib/dhp/oa/graph/dump/resultDump/dataset_extendedinstance")
.getPath();
final String communityMapPath = getClass()
.getResource("/eu/dnetlib/dhp/oa/graph/dump/communityMapPath/communitymap.json")
.getPath();
DumpProducts dump = new DumpProducts();
dump
.run(
false, sourcePath, workingDir.toString() + "/result",
communityMapPath, Dataset.class,
GraphResult.class, Constants.DUMPTYPE.COMPLETE.getType());
final JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext());
JavaRDD<GraphResult> tmp = sc
.textFile(workingDir.toString() + "/result")
.map(item -> OBJECT_MAPPER.readValue(item, GraphResult.class));
org.apache.spark.sql.Dataset<GraphResult> verificationDataset = spark
.createDataset(tmp.rdd(), Encoders.bean(GraphResult.class));
Assertions.assertEquals(1, verificationDataset.count());
Assertions.assertEquals(1, verificationDataset.filter("type = 'dataset'").count());
// the common fields in the result have been already checked. Now checking only
// community specific fields
GraphResult gr = verificationDataset.first();
Assertions.assertEquals(2, gr.getGeolocation().size());
Assertions.assertEquals(2, gr.getGeolocation().stream().filter(gl -> gl.getBox().equals("")).count());
Assertions.assertEquals(1, gr.getGeolocation().stream().filter(gl -> gl.getPlace().equals("")).count());
Assertions.assertEquals(1, gr.getGeolocation().stream().filter(gl -> gl.getPoint().equals("")).count());
Assertions
.assertEquals(
1,
gr
.getGeolocation()
.stream()
.filter(gl -> gl.getPlace().equals("18 York St, Ottawa, ON K1N 5S6; Ottawa; Ontario; Canada"))
.count());
Assertions
.assertEquals(
1, gr.getGeolocation().stream().filter(gl -> gl.getPoint().equals("45.427242 -75.693904")).count());
Assertions
.assertEquals(
1,
gr
.getGeolocation()
.stream()
.filter(gl -> gl.getPoint().equals("") && !gl.getPlace().equals(""))
.count());
Assertions
.assertEquals(
1,
gr
.getGeolocation()
.stream()
.filter(gl -> !gl.getPoint().equals("") && gl.getPlace().equals(""))
.count());
Assertions.assertEquals("1024Gb", gr.getSize());
Assertions.assertEquals("1.01", gr.getVersion());
Assertions.assertEquals(null, gr.getContainer());
Assertions.assertEquals(null, gr.getCodeRepositoryUrl());
Assertions.assertEquals(null, gr.getProgrammingLanguage());
Assertions.assertEquals(null, gr.getDocumentationUrl());
Assertions.assertEquals(null, gr.getContactperson());
Assertions.assertEquals(null, gr.getContactgroup());
Assertions.assertEquals(null, gr.getTool());
}
@Test
public void testSoftwareDump() {
final String sourcePath = getClass()
.getResource("/eu/dnetlib/dhp/oa/graph/dump/resultDump/software_extendedinstance")
.getPath();
final String communityMapPath = getClass()
.getResource("/eu/dnetlib/dhp/oa/graph/dump/communityMapPath/communitymap.json")
.getPath();
DumpProducts dump = new DumpProducts();
dump
.run(
false, sourcePath, workingDir.toString() + "/result",
communityMapPath, Software.class,
GraphResult.class, Constants.DUMPTYPE.COMPLETE.getType());
final JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext());
JavaRDD<GraphResult> tmp = sc
.textFile(workingDir.toString() + "/result")
.map(item -> OBJECT_MAPPER.readValue(item, GraphResult.class));
org.apache.spark.sql.Dataset<GraphResult> verificationDataset = spark
.createDataset(tmp.rdd(), Encoders.bean(GraphResult.class));
Assertions.assertEquals(1, verificationDataset.count());
Assertions.assertEquals(1, verificationDataset.filter("type = 'software'").count());
GraphResult gr = verificationDataset.first();
Assertions.assertEquals(2, gr.getDocumentationUrl().size());
Assertions.assertTrue(gr.getDocumentationUrl().contains("doc_url_1"));
Assertions.assertTrue(gr.getDocumentationUrl().contains("doc_url_2"));
Assertions.assertEquals("code_repo", gr.getCodeRepositoryUrl());
Assertions.assertEquals("perl", gr.getProgrammingLanguage());
Assertions.assertEquals(null, gr.getContainer());
Assertions.assertEquals(null, gr.getContactperson());
Assertions.assertEquals(null, gr.getContactgroup());
Assertions.assertEquals(null, gr.getTool());
Assertions.assertEquals(null, gr.getGeolocation());
Assertions.assertEquals(null, gr.getSize());
Assertions.assertEquals(null, gr.getVersion());
}
@Test
public void testOrpDump() {
final String sourcePath = getClass()
.getResource("/eu/dnetlib/dhp/oa/graph/dump/resultDump/orp_extendedinstance")
.getPath();
final String communityMapPath = getClass()
.getResource("/eu/dnetlib/dhp/oa/graph/dump/communityMapPath/communitymap.json")
.getPath();
DumpProducts dump = new DumpProducts();
dump
.run(
false, sourcePath, workingDir.toString() + "/result",
communityMapPath, OtherResearchProduct.class,
GraphResult.class, Constants.DUMPTYPE.COMPLETE.getType());
final JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext());
JavaRDD<GraphResult> tmp = sc
.textFile(workingDir.toString() + "/result")
.map(item -> OBJECT_MAPPER.readValue(item, GraphResult.class));
org.apache.spark.sql.Dataset<GraphResult> verificationDataset = spark
.createDataset(tmp.rdd(), Encoders.bean(GraphResult.class));
Assertions.assertEquals(1, verificationDataset.count());
Assertions.assertEquals(1, verificationDataset.filter("type = 'other'").count());
GraphResult gr = verificationDataset.first();
Assertions.assertEquals(2, gr.getContactperson().size());
Assertions.assertTrue(gr.getContactperson().contains(("contact_person1")));
Assertions.assertTrue(gr.getContactperson().contains(("contact_person2")));
Assertions.assertEquals(1, gr.getContactgroup().size());
Assertions.assertTrue(gr.getContactgroup().contains(("contact_group")));
Assertions.assertEquals(2, gr.getTool().size());
Assertions.assertTrue(gr.getTool().contains("tool1"));
Assertions.assertTrue(gr.getTool().contains("tool2"));
Assertions.assertEquals(null, gr.getContainer());
Assertions.assertEquals(null, gr.getDocumentationUrl());
Assertions.assertEquals(null, gr.getCodeRepositoryUrl());
Assertions.assertEquals(null, gr.getProgrammingLanguage());
Assertions.assertEquals(null, gr.getGeolocation());
Assertions.assertEquals(null, gr.getSize());
Assertions.assertEquals(null, gr.getVersion());
}
@Test
public void testPublicationDumpCommunity() throws JsonProcessingException {
final String sourcePath = getClass()
.getResource("/eu/dnetlib/dhp/oa/graph/dump/resultDump/publication_extendedinstance")
.getPath();
final String communityMapPath = getClass()
.getResource("/eu/dnetlib/dhp/oa/graph/dump/communityMapPath/communitymap.json")
.getPath();
DumpProducts dump = new DumpProducts();
dump
.run(
false, sourcePath, workingDir.toString() + "/result", communityMapPath, Publication.class,
CommunityResult.class, Constants.DUMPTYPE.COMMUNITY.getType());
final JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext());
JavaRDD<CommunityResult> tmp = sc
.textFile(workingDir.toString() + "/result")
.map(item -> OBJECT_MAPPER.readValue(item, CommunityResult.class));
org.apache.spark.sql.Dataset<CommunityResult> verificationDataset = spark
.createDataset(tmp.rdd(), Encoders.bean(CommunityResult.class));
Assertions.assertEquals(1, verificationDataset.count());
Assertions.assertEquals(1, verificationDataset.filter("type = 'publication'").count());
// the common fields in the result have been already checked. Now checking only
// community specific fields
CommunityResult cr = verificationDataset.first();
Assertions.assertEquals(1, cr.getContext().size());
Assertions.assertEquals("dh-ch", cr.getContext().get(0).getCode());
Assertions.assertEquals("Digital Humanities and Cultural Heritage", cr.getContext().get(0).getLabel());
Assertions.assertEquals(1, cr.getContext().get(0).getProvenance().size());
Assertions.assertEquals("Inferred by OpenAIRE", cr.getContext().get(0).getProvenance().get(0).getProvenance());
Assertions.assertEquals("0.9", cr.getContext().get(0).getProvenance().get(0).getTrust());
Assertions.assertEquals(1, cr.getCollectedfrom().size());
Assertions
.assertEquals("10|openaire____::fdc7e0400d8c1634cdaf8051dbae23db", cr.getCollectedfrom().get(0).getKey());
Assertions.assertEquals("Pensoft", cr.getCollectedfrom().get(0).getValue());
Assertions.assertEquals(1, cr.getInstance().size());
Assertions
.assertEquals(
"10|openaire____::fdc7e0400d8c1634cdaf8051dbae23db",
cr.getInstance().get(0).getCollectedfrom().getKey());
Assertions.assertEquals("Pensoft", cr.getInstance().get(0).getCollectedfrom().getValue());
Assertions
.assertEquals(
"10|openaire____::e707e544b9a5bd23fc27fbfa65eb60dd", cr.getInstance().get(0).getHostedby().getKey());
Assertions.assertEquals("One Ecosystem", cr.getInstance().get(0).getHostedby().getValue());
}
@Test
public void testDataset() {
@ -151,7 +661,6 @@ public class DumpJobTest {
DumpProducts dump = new DumpProducts();
dump
.run(
// false, sourcePath, workingDir.toString() + "/result", communityMapPath, Dataset.class,
false, sourcePath, workingDir.toString() + "/result", communityMapPath, Dataset.class,
CommunityResult.class, Constants.DUMPTYPE.COMMUNITY.getType());
@ -166,10 +675,6 @@ public class DumpJobTest {
Assertions.assertEquals(90, verificationDataset.count());
// verificationDataset
// .filter("id = '50|DansKnawCris::1a960e20087cb46b93588e4e184e8a58'")
// .foreach((ForeachFunction<CommunityResult>) rec -> System.out.println(OBJECT_MAPPER.writeValueAsString(rec)));
Assertions
.assertTrue(
verificationDataset.filter("bestAccessright.code = 'c_abf2'").count() == verificationDataset
@ -198,8 +703,6 @@ public class DumpJobTest {
Assertions.assertTrue(verificationDataset.filter("type = 'dataset'").count() == 90);
//TODO verify value and name of the fields for vocab related value (i.e. accessright, bestaccessright)
}
@Test
@ -231,8 +734,6 @@ public class DumpJobTest {
Assertions.assertEquals(5, verificationDataset.count());
verificationDataset
.foreach((ForeachFunction<GraphResult>) res -> System.out.println(OBJECT_MAPPER.writeValueAsString(res)));
}
@Test
@ -249,7 +750,6 @@ public class DumpJobTest {
DumpProducts dump = new DumpProducts();
dump
.run(
// false, sourcePath, workingDir.toString() + "/result", communityMapPath, Dataset.class,
false, sourcePath, workingDir.toString() + "/result", communityMapPath, Dataset.class,
CommunityResult.class, Constants.DUMPTYPE.COMMUNITY.getType());
@ -264,7 +764,6 @@ public class DumpJobTest {
Assertions.assertEquals(0, verificationDataset.count());
verificationDataset.show(false);
}
@Test
@ -299,8 +798,6 @@ public class DumpJobTest {
Assertions.assertEquals(74, verificationDataset.filter("type = 'publication'").count());
//TODO verify value and name of the fields for vocab related value (i.e. accessright, bestaccessright)
}
@Test
@ -333,9 +830,6 @@ public class DumpJobTest {
Assertions.assertEquals(6, verificationDataset.count());
Assertions.assertEquals(6, verificationDataset.filter("type = 'software'").count());
verificationDataset.show(false);
//TODO verify value and name of the fields for vocab related value (i.e. accessright, bestaccessright)
}
@ -369,9 +863,6 @@ public class DumpJobTest {
Assertions.assertEquals(3, verificationDataset.count());
Assertions.assertEquals(3, verificationDataset.filter("type = 'other'").count());
verificationDataset.show(false);
//TODO verify value and name of the fields for vocab related value (i.e. accessright, bestaccessright)
}
@ -388,7 +879,6 @@ public class DumpJobTest {
DumpProducts dump = new DumpProducts();
dump
.run(
// false, sourcePath, workingDir.toString() + "/result", communityMapPath, Publication.class,
false, sourcePath, workingDir.toString() + "/result", communityMapPath, Publication.class,
CommunityResult.class, Constants.DUMPTYPE.COMMUNITY.getType());
@ -408,4 +898,51 @@ public class DumpJobTest {
}
@Test
public void testArticlePCA() {
final String sourcePath = getClass()
.getResource("/eu/dnetlib/dhp/oa/graph/dump/resultDump/publication_pca")
.getPath();
final String communityMapPath = getClass()
.getResource("/eu/dnetlib/dhp/oa/graph/dump/communityMapPath/communitymap.json")
.getPath();
DumpProducts dump = new DumpProducts();
dump
.run(
// false, sourcePath, workingDir.toString() + "/result", communityMapPath, Publication.class,
false, sourcePath, workingDir.toString() + "/result", communityMapPath, Publication.class,
GraphResult.class, Constants.DUMPTYPE.COMPLETE.getType());
final JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext());
JavaRDD<GraphResult> tmp = sc
.textFile(workingDir.toString() + "/result")
.map(item -> OBJECT_MAPPER.readValue(item, GraphResult.class));
org.apache.spark.sql.Dataset<GraphResult> verificationDataset = spark
.createDataset(tmp.rdd(), Encoders.bean(GraphResult.class));
Assertions.assertEquals(23, verificationDataset.count());
Assertions.assertEquals(23, verificationDataset.filter("type = 'publication'").count());
verificationDataset.createOrReplaceTempView("check");
org.apache.spark.sql.Dataset<Row> temp = spark
.sql(
"select id " +
"from check " +
"lateral view explode (instance) i as inst " +
"where inst.articleprocessingcharge is not null");
Assertions.assertTrue(temp.count() == 2);
Assertions.assertTrue(temp.filter("id = '50|datacite____::05c611fdfc93d7a2a703d1324e28104a'").count() == 1);
Assertions.assertTrue(temp.filter("id = '50|dedup_wf_001::01e6a28565ca01376b7548e530c6f6e8'").count() == 1);
}
}

View File

@ -223,4 +223,102 @@ public class PrepareResultProjectJobTest {
}
@Test
public void testMatchValidated() throws Exception {
final String sourcePath = getClass()
.getResource("/eu/dnetlib/dhp/oa/graph/dump/resultProject/match_validatedRels")
.getPath();
SparkPrepareResultProject.main(new String[] {
"-isSparkSessionManaged", Boolean.FALSE.toString(),
"-outputPath", workingDir.toString() + "/preparedInfo",
"-sourcePath", sourcePath
});
final JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext());
JavaRDD<ResultProject> tmp = sc
.textFile(workingDir.toString() + "/preparedInfo")
.map(item -> OBJECT_MAPPER.readValue(item, ResultProject.class));
org.apache.spark.sql.Dataset<ResultProject> verificationDataset = spark
.createDataset(tmp.rdd(), Encoders.bean(ResultProject.class));
assertEquals(2, verificationDataset.count());
assertEquals(
1,
verificationDataset.filter("resultId = '50|dedup_wf_001::e4805d005bfab0cd39a1642cbf477fdb'").count());
assertEquals(
1,
verificationDataset.filter("resultId = '50|dedup_wf_001::51b88f272ba9c3bb181af64e70255a80'").count());
verificationDataset.createOrReplaceTempView("dataset");
String query = "select resultId, MyT.id project , MyT.title title, MyT.acronym acronym , MyT.provenance.provenance provenance, "
+
"MyT.validated.validatedByFunder, MyT.validated.validationDate "
+ "from dataset "
+ "lateral view explode(projectsList) p as MyT ";
org.apache.spark.sql.Dataset<Row> resultExplodedProvenance = spark.sql(query);
assertEquals(3, resultExplodedProvenance.count());
assertEquals(3, resultExplodedProvenance.filter("validatedByFunder = true").count());
assertEquals(
2,
resultExplodedProvenance
.filter("resultId = '50|dedup_wf_001::e4805d005bfab0cd39a1642cbf477fdb'")
.count());
assertEquals(
1,
resultExplodedProvenance
.filter("resultId = '50|dedup_wf_001::51b88f272ba9c3bb181af64e70255a80'")
.count());
assertEquals(
2,
resultExplodedProvenance
.filter("project = '40|aka_________::0f7d119de1f656b5763a16acf876fed6'")
.count());
assertEquals(
1,
resultExplodedProvenance
.filter(
"project = '40|aka_________::0f7d119de1f656b5763a16acf876fed6' " +
"and resultId = '50|dedup_wf_001::e4805d005bfab0cd39a1642cbf477fdb' " +
"and validatedByFunder = true " +
"and validationDate = '2021-08-06'")
.count());
assertEquals(
1,
resultExplodedProvenance
.filter(
"project = '40|aka_________::0f7d119de1f656b5763a16acf876fed6' " +
"and resultId = '50|dedup_wf_001::51b88f272ba9c3bb181af64e70255a80' " +
"and validatedByFunder = true and validationDate = '2021-08-04'")
.count());
assertEquals(
1,
resultExplodedProvenance
.filter("project = '40|aka_________::03376222b28a3aebf2730ac514818d04'")
.count());
assertEquals(
1,
resultExplodedProvenance
.filter(
"project = '40|aka_________::03376222b28a3aebf2730ac514818d04' " +
"and resultId = '50|dedup_wf_001::e4805d005bfab0cd39a1642cbf477fdb' " +
"and validatedByFunder = true and validationDate = '2021-08-05'")
.count());
assertEquals(
3, resultExplodedProvenance.filter("provenance = 'sysimport:crosswalk:entityregistry'").count());
}
}

View File

@ -25,7 +25,7 @@ class QueryInformationSystemTest {
private static final String XQUERY = "for $x in collection('/db/DRIVER/ContextDSResources/ContextDSResourceType') "
+
" where $x//CONFIGURATION/context[./@type='community' or ./@type='ri'] " +
" and ($x//context/param[./@name = 'status']/text() = 'manager' or $x//context/param[./@name = 'status']/text() = 'all') "
" and ($x//context/param[./@name = 'status']/text() = 'all') "
+
" return " +
"<community> " +
@ -71,7 +71,7 @@ class QueryInformationSystemTest {
lenient().when(isLookUpService.quickSearchProfile(XQUERY)).thenReturn(communityMap);
queryInformationSystem = new QueryInformationSystem();
queryInformationSystem.setIsLookUp(isLookUpService);
map = queryInformationSystem.getCommunityMap();
map = queryInformationSystem.getCommunityMap(false, null);
}
@Test

View File

@ -79,7 +79,7 @@ public class SplitForCommunityTest {
final JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext());
JavaRDD<CommunityResult> tmp = sc
.textFile(workingDir.toString() + "/split/dh-ch")
.textFile(workingDir.toString() + "/split/Digital_Humanities_and_Cultural_Heritage")
.map(item -> OBJECT_MAPPER.readValue(item, CommunityResult.class));
org.apache.spark.sql.Dataset<CommunityResult> verificationDataset = spark
@ -92,7 +92,7 @@ public class SplitForCommunityTest {
1, verificationDataset.filter("id = '50|dedup_wf_001::51b88f272ba9c3bb181af64e70255a80'").count());
tmp = sc
.textFile(workingDir.toString() + "/split/egi")
.textFile(workingDir.toString() + "/split/EGI_Federation")
.map(item -> OBJECT_MAPPER.readValue(item, CommunityResult.class));
verificationDataset = spark
@ -105,7 +105,7 @@ public class SplitForCommunityTest {
1, verificationDataset.filter("id = '50|dedup_wf_001::e4805d005bfab0cd39a1642cbf477fdb'").count());
tmp = sc
.textFile(workingDir.toString() + "/split/ni")
.textFile(workingDir.toString() + "/split/Neuroinformatics")
.map(item -> OBJECT_MAPPER.readValue(item, CommunityResult.class));
verificationDataset = spark
@ -118,7 +118,7 @@ public class SplitForCommunityTest {
1, verificationDataset.filter("id = '50|datacite____::6b1e3a2fa60ed8c27317a66d6357f795'").count());
tmp = sc
.textFile(workingDir.toString() + "/split/science-innovation-policy")
.textFile(workingDir.toString() + "/split/Science_and_Innovation_Policy_Studies")
.map(item -> OBJECT_MAPPER.readValue(item, CommunityResult.class));
verificationDataset = spark

View File

@ -5,11 +5,15 @@ import java.io.IOException;
import java.nio.file.Files;
import java.nio.file.Path;
import java.util.HashMap;
import java.util.logging.Filter;
import java.util.stream.Collectors;
import org.apache.commons.io.FileUtils;
import org.apache.spark.SparkConf;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.api.java.function.FilterFunction;
import org.apache.spark.api.java.function.MapFunction;
import org.apache.spark.sql.Encoders;
import org.apache.spark.sql.Row;
import org.apache.spark.sql.SparkSession;
@ -25,6 +29,8 @@ import com.fasterxml.jackson.databind.ObjectMapper;
import eu.dnetlib.dhp.oa.graph.dump.community.SparkUpdateProjectInfo;
import eu.dnetlib.dhp.schema.dump.oaf.Result;
import eu.dnetlib.dhp.schema.dump.oaf.community.CommunityResult;
import eu.dnetlib.dhp.schema.dump.oaf.community.Funder;
import eu.dnetlib.dhp.schema.dump.oaf.community.Project;
public class UpdateProjectInfoTest {
@ -135,4 +141,96 @@ public class UpdateProjectInfoTest {
resultExplodedProvenance.show(false);
}
@Test
public void testValidatedRelation() throws Exception {
final String sourcePath = getClass()
.getResource("/eu/dnetlib/dhp/oa/graph/dump/addProjectInfo")
.getPath();
SparkUpdateProjectInfo.main(new String[] {
"-isSparkSessionManaged", Boolean.FALSE.toString(),
"-preparedInfoPath", sourcePath + "/preparedInfoValidated",
"-outputPath", workingDir.toString() + "/result",
"-sourcePath", sourcePath + "/publication_extendedmodel"
});
final JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext());
JavaRDD<CommunityResult> tmp = sc
.textFile(workingDir.toString() + "/result")
.map(item -> OBJECT_MAPPER.readValue(item, CommunityResult.class));
org.apache.spark.sql.Dataset<CommunityResult> verificationDataset = spark
.createDataset(tmp.rdd(), Encoders.bean(CommunityResult.class));
verificationDataset.show(false);
Assertions.assertEquals(2, verificationDataset.count());
verificationDataset.createOrReplaceTempView("dataset");
String query = "select id, MyT.code code, MyT.title title, MyT.funder.name funderName, MyT.funder.shortName funderShortName, "
+
"MyT.funder.jurisdiction funderJurisdiction, MyT.funder.fundingStream fundingStream, MyT.validated "
+ "from dataset " +
"lateral view explode(projects) p as MyT ";
org.apache.spark.sql.Dataset<Row> resultExplodedProvenance = spark.sql(query);
Assertions.assertEquals(2, resultExplodedProvenance.count());
resultExplodedProvenance.show(false);
Assertions
.assertEquals(
2,
resultExplodedProvenance.filter("id = '50|pensoft_____::00ea4a1cd53806a97d62ea6bf268f2a2'").count());
Assertions
.assertEquals(
1,
resultExplodedProvenance
.filter("id = '50|pensoft_____::00ea4a1cd53806a97d62ea6bf268f2a2' and code = '123455'")
.count());
Assertions
.assertEquals(
1,
resultExplodedProvenance
.filter("id = '50|pensoft_____::00ea4a1cd53806a97d62ea6bf268f2a2' and code = '119027'")
.count());
Project project = verificationDataset
.map(
(MapFunction<CommunityResult, Project>) cr -> cr
.getProjects()
.stream()
.filter(p -> p.getValidated() != null)
.collect(Collectors.toList())
.get(0),
Encoders.bean(Project.class))
.first();
Assertions.assertTrue(project.getFunder().getName().equals("Academy of Finland"));
Assertions.assertTrue(project.getFunder().getShortName().equals("AKA"));
Assertions.assertTrue(project.getFunder().getJurisdiction().equals("FI"));
Assertions.assertTrue(project.getFunder().getFundingStream() == null);
Assertions.assertTrue(project.getValidated().getValidationDate().equals("2021-08-06"));
project = verificationDataset
.map(
(MapFunction<CommunityResult, Project>) cr -> cr
.getProjects()
.stream()
.filter(p -> p.getValidated() == null)
.collect(Collectors.toList())
.get(0),
Encoders.bean(Project.class))
.first();
Assertions.assertTrue(project.getFunder().getName().equals("European Commission"));
Assertions.assertTrue(project.getFunder().getShortName().equals("EC"));
Assertions.assertTrue(project.getFunder().getJurisdiction().equals("EU"));
Assertions.assertTrue(project.getFunder().getFundingStream().equals("H2020"));
}
}

View File

@ -14,6 +14,7 @@ import com.google.gson.Gson;
import eu.dnetlib.dhp.schema.common.ModelSupport;
import eu.dnetlib.dhp.schema.dump.oaf.graph.Relation;
import eu.dnetlib.dhp.schema.oaf.Datasource;
import eu.dnetlib.dhp.schema.oaf.Project;
import eu.dnetlib.dhp.utils.DHPUtils;
class CreateRelationTest {
@ -203,6 +204,7 @@ class CreateRelationTest {
" <param name=\"suggestedAcknowledgement\"/>\n" +
" <param name=\"zenodoCommunity\">oac_ni</param>\n" +
" <param name=\"creationdate\">2018-03-01T12:00:00</param>\n" +
" <category claim=\"false\" id=\"ni::projects\" label=\"NI Content providers\"/>\n" +
" <category claim=\"false\" id=\"ni::contentproviders\" label=\"NI Content providers\">\n" +
" <concept claim=\"false\" id=\"ni::contentproviders::1\" label=\"OpenNeuro\">\n" +
" <param name=\"openaireId\">re3data_____::5b9bf9171d92df854cf3c520692e9122</param>\n" +
@ -376,7 +378,7 @@ class CreateRelationTest {
" <param name=\"rule\"/>\n" +
" <param name=\"CD_PROJECT_NUMBER\">675858</param>\n" +
" <param name=\"url\"/>\n" +
" <param name=\"funding\">H2020-EINFRA-2015-1</param>\n" +
" <param name=\"funding\">EC | H2020 | RIA</param>\n" +
" <param name=\"funder\">EC</param>\n" +
" <param name=\"acronym\">West-Life</param>\n" +
" </concept>\n" +
@ -437,7 +439,65 @@ class CreateRelationTest {
" <param name=\"suggestedAcknowledgement\"/>\n" +
" <param name=\"zenodoCommunity\">oaa_elixir-gr</param>\n" +
" <param name=\"creationdate\">2018-03-01T12:00:00</param>\n" +
" <category claim=\"false\" id=\"elixir-gr::projects\" label=\"ELIXIR GR Projects\"/>\n" +
" <category claim=\"false\" id=\"elixir-gr::projects\" label=\"ELIXIR GR Projects\">\n" +
" <concept claim=\"false\" id=\"ni::projects::12\" label=\"\">\n" +
" <param name=\"projectfullname\">BIO-INFORMATICS RESEARCH NETWORK COORDINATING CENTER (BIRN-CC)</param>\n"
+
" <param name=\"acronym\"/>\n" +
" <param name=\"CD_PROJECT_NUMBER\">1U24RR025736-01</param>\n" +
" <param name=\"funder\">NIH</param>\n" +
" </concept>\n" +
" <concept claim=\"false\" id=\"ni::projects::13\" label=\"\">\n" +
" <param name=\"projectfullname\">COLLABORATIVE RESEARCH: The Cognitive Neuroscience of Category Learning</param>\n"
+
" <param name=\"acronym\"/>\n" +
" <param name=\"CD_PROJECT_NUMBER\">0223843</param>\n" +
" <param name=\"funder\">NSF</param>\n" +
" </concept>\n" +
" <concept claim=\"false\" id=\"ni::projects::14\" label=\"\">\n" +
" <param name=\"projectfullname\">The Cognitive Atlas: Developing an Interdisciplinary Knowledge Base Through Socia</param>\n"
+
" <param name=\"acronym\"/>\n" +
" <param name=\"CD_PROJECT_NUMBER\">5R01MH082795-05</param>\n" +
" <param name=\"funder\">NIH</param>\n" +
" </concept>\n" +
" <concept claim=\"false\" id=\"ni::projects::15\" label=\"\">\n" +
" <param name=\"projectfullname\">Fragmented early life environmental and emotional / cognitive vulnerabilities</param>\n"
+
" <param name=\"acronym\"/>\n" +
" <param name=\"CD_PROJECT_NUMBER\">1P50MH096889-01A1</param>\n" +
" <param name=\"funder\">NIH</param>\n" +
" </concept>\n" +
" <concept claim=\"false\" id=\"ni::projects::16\" label=\"\">\n" +
" <param name=\"projectfullname\">Enhancement of the 1000 Functional Connectome Project</param>\n"
+
" <param name=\"acronym\"/>\n" +
" <param name=\"CD_PROJECT_NUMBER\">1R03MH096321-01A1</param>\n" +
" <param name=\"funder\">TUBITAK</param>\n" +
" </concept>\n" +
" <concept claim=\"false\" id=\"ni::projects::17\" label=\"\">\n" +
" <param name=\"projectfullname\">CRCNS Data Sharing: An open data repository for cognitive neuroscience: The OpenfMRI Project</param>\n"
+
" <param name=\"acronym\"/>\n" +
" <param name=\"CD_PROJECT_NUMBER\">1131441</param>\n" +
" <param name=\"funder\">NSF</param>\n" +
" </concept>\n" +
" <concept claim=\"false\" id=\"ni::projects::18\" label=\"\">\n" +
" <param name=\"projectfullname\">Enhancing Human Cortical Plasticity: Visual Psychophysics and fMRI</param>\n"
+
" <param name=\"acronym\"/>\n" +
" <param name=\"CD_PROJECT_NUMBER\">0121950</param>\n" +
" <param name=\"funder\">NSF</param>\n" +
" </concept>\n" +
" <concept claim=\"false\" id=\"ni::projects::18\" label=\"\">\n" +
" <param name=\"projectfullname\">Transforming statistical methodology for neuroimaging meta-analysis.</param>\n"
+
" <param name=\"acronym\"/>\n" +
" <param name=\"CD_PROJECT_NUMBER\">100309</param>\n" +
" <param name=\"funder\">WT</param>\n" +
" </concept>\n" +
" </category>" +
" <category claim=\"false\" id=\"elixir-gr::contentproviders\" label=\"Elixir-GR Content providers\">\n"
+
" <concept claim=\"false\" id=\"elixir-gr::contentproviders::1\" label=\"bio.tools\">\n" +
@ -566,4 +626,98 @@ class CreateRelationTest {
tmp.contains("10|doajarticles::2899208a99aa7d142646e0a80bfeef05"));
}
@Test
public void test2() {
List<ContextInfo> cInfoList = new ArrayList<>();
final Consumer<ContextInfo> consumer = ci -> cInfoList.add(ci);
queryInformationSystem
.getContextRelation(consumer, "projects", ModelSupport.getIdPrefix(Project.class));
cInfoList.forEach(c -> System.out.println(new Gson().toJson(c)));
List<Relation> rList = new ArrayList<>();
cInfoList.forEach(cInfo -> Process.getRelation(cInfo).forEach(rList::add));
Assertions.assertEquals(44, rList.size());
Assertions
.assertFalse(
rList
.stream()
.map(r -> r.getSource().getId())
.collect(Collectors.toSet())
.contains(
String
.format(
"%s|%s::%s", Constants.CONTEXT_ID,
Constants.CONTEXT_NS_PREFIX,
DHPUtils.md5("dh-ch"))));
Assertions
.assertEquals(
2,
rList
.stream()
.filter(
r -> r
.getSource()
.getId()
.equals(
String
.format(
"%s|%s::%s", Constants.CONTEXT_ID,
Constants.CONTEXT_NS_PREFIX,
DHPUtils.md5("clarin"))))
.collect(Collectors.toList())
.size());
Assertions
.assertEquals(
2,
rList
.stream()
.filter(
r -> r
.getTarget()
.getId()
.equals(
String
.format(
"%s|%s::%s", Constants.CONTEXT_ID,
Constants.CONTEXT_NS_PREFIX,
DHPUtils.md5("clarin"))))
.collect(Collectors.toList())
.size());
Set<String> tmp = rList
.stream()
.filter(
r -> r
.getSource()
.getId()
.equals(
String
.format(
"%s|%s::%s", Constants.CONTEXT_ID,
Constants.CONTEXT_NS_PREFIX,
DHPUtils.md5("clarin"))))
.map(r -> r.getTarget().getId())
.collect(Collectors.toSet());
Assertions
.assertTrue(
tmp.contains("40|corda__h2020::b5a4eb56bf84bef2ebc193306b4d423f") &&
tmp.contains("40|corda_______::ef782b2d85676aa3e5a907427feb18c4"));
rList.forEach(rel -> {
if (rel.getSource().getId().startsWith("40|")) {
String proj = rel.getSource().getId().substring(3);
Assertions.assertTrue(proj.substring(0, proj.indexOf("::")).length() == 12);
}
});
}
}

View File

@ -22,6 +22,7 @@ import org.slf4j.LoggerFactory;
import com.fasterxml.jackson.databind.ObjectMapper;
import eu.dnetlib.dhp.oa.graph.dump.exceptions.NoAvailableEntityTypeException;
import eu.dnetlib.dhp.schema.oaf.Datasource;
import eu.dnetlib.dhp.schema.oaf.Organization;
import eu.dnetlib.dhp.schema.oaf.Project;
@ -88,7 +89,7 @@ public class DumpOrganizationProjectDatasourceTest {
org.apache.spark.sql.Dataset<eu.dnetlib.dhp.schema.dump.oaf.graph.Organization> verificationDataset = spark
.createDataset(tmp.rdd(), Encoders.bean(eu.dnetlib.dhp.schema.dump.oaf.graph.Organization.class));
Assertions.assertEquals(34, verificationDataset.count());
Assertions.assertEquals(15, verificationDataset.count());
verificationDataset
.foreach(
@ -98,7 +99,7 @@ public class DumpOrganizationProjectDatasourceTest {
}
@Test
public void dumpProjectTest() {
public void dumpProjectTest() throws NoAvailableEntityTypeException {
final String sourcePath = getClass()
.getResource("/eu/dnetlib/dhp/oa/graph/dump/complete/project")
@ -127,7 +128,7 @@ public class DumpOrganizationProjectDatasourceTest {
}
@Test
public void dumpDatasourceTest() {
public void dumpDatasourceTest() throws NoAvailableEntityTypeException {
final String sourcePath = getClass()
.getResource("/eu/dnetlib/dhp/oa/graph/dump/complete/datasource")
.getPath();

View File

@ -10,6 +10,7 @@ import org.apache.commons.io.FileUtils;
import org.apache.spark.SparkConf;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.api.java.function.ForeachFunction;
import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Encoders;
import org.apache.spark.sql.Row;
@ -71,7 +72,7 @@ public class DumpRelationTest {
public void test1() throws Exception {
final String sourcePath = getClass()
.getResource("/eu/dnetlib/dhp/oa/graph/dump/relation")
.getResource("/eu/dnetlib/dhp/oa/graph/dump/relation/relation")
.getPath();
SparkDumpRelationJob.main(new String[] {
@ -93,6 +94,9 @@ public class DumpRelationTest {
verificationDataset.createOrReplaceTempView("table");
verificationDataset
.foreach((ForeachFunction<Relation>) r -> System.out.println(new ObjectMapper().writeValueAsString(r)));
Dataset<Row> check = spark
.sql(
"SELECT reltype.name, source.id source, source.type stype, target.id target,target.type ttype, provenance.provenance "
@ -127,4 +131,76 @@ public class DumpRelationTest {
.count());
}
@Test
public void test2() throws Exception {
final String sourcePath = getClass()
.getResource("/eu/dnetlib/dhp/oa/graph/dump/relation/relation_validated")
.getPath();
SparkDumpRelationJob.main(new String[] {
"-isSparkSessionManaged", Boolean.FALSE.toString(),
"-outputPath", workingDir.toString() + "/relation",
"-sourcePath", sourcePath
});
// dumpCommunityProducts.exec(MOCK_IS_LOOK_UP_URL,Boolean.FALSE, workingDir.toString()+"/dataset",sourcePath,"eu.dnetlib.dhp.schema.oaf.Dataset","eu.dnetlib.dhp.schema.dump.oaf.Dataset");
final JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext());
JavaRDD<Relation> tmp = sc
.textFile(workingDir.toString() + "/relation")
.map(item -> OBJECT_MAPPER.readValue(item, Relation.class));
org.apache.spark.sql.Dataset<Relation> verificationDataset = spark
.createDataset(tmp.rdd(), Encoders.bean(Relation.class));
verificationDataset.createOrReplaceTempView("table");
verificationDataset
.foreach((ForeachFunction<Relation>) r -> System.out.println(new ObjectMapper().writeValueAsString(r)));
Dataset<Row> check = spark
.sql(
"SELECT reltype.name, source.id source, source.type stype, target.id target,target.type ttype, provenance.provenance "
+
"from table ");
Assertions.assertEquals(20, check.filter("name = 'isProvidedBy'").count());
Assertions
.assertEquals(
20, check
.filter(
"name = 'isProvidedBy' and stype = 'datasource' and ttype = 'organization' and " +
"provenance = 'Harvested'")
.count());
Assertions.assertEquals(7, check.filter("name = 'isParticipant'").count());
Assertions
.assertEquals(
7, check
.filter(
"name = 'isParticipant' and stype = 'organization' and ttype = 'project' " +
"and provenance = 'Harvested'")
.count());
Assertions.assertEquals(1, check.filter("name = 'isAuthorInstitutionOf'").count());
Assertions
.assertEquals(
1, check
.filter(
"name = 'isAuthorInstitutionOf' and stype = 'organization' and ttype = 'result' " +
"and provenance = 'Inferred by OpenAIRE'")
.count());
Assertions.assertEquals(2, check.filter("name = 'isProducedBy'").count());
Assertions
.assertEquals(
2, check
.filter(
"name = 'isProducedBy' and stype = 'project' and ttype = 'result' " +
"and provenance = 'Harvested' and validated = true " +
"and validationDate = '2021-08-06'")
.count());
}
}

View File

@ -13,6 +13,7 @@ import org.junit.jupiter.api.extension.ExtendWith;
import org.mockito.Mock;
import org.mockito.junit.jupiter.MockitoExtension;
import eu.dnetlib.dhp.schema.common.ModelSupport;
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException;
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService;
@ -528,7 +529,8 @@ class QueryInformationSystemTest {
List<ContextInfo> cInfoList = new ArrayList<>();
final Consumer<ContextInfo> consumer = ci -> cInfoList.add(ci);
queryInformationSystem.execContextRelationQuery();
queryInformationSystem.getContextRelation(consumer, "contentproviders", "10|");
queryInformationSystem
.getContextRelation(consumer, "contentproviders", ModelSupport.entityIdPrefix.get("datasource"));
Assertions.assertEquals(5, cInfoList.size());
}
@ -539,7 +541,8 @@ class QueryInformationSystemTest {
List<ContextInfo> cInfoList = new ArrayList<>();
final Consumer<ContextInfo> consumer = ci -> cInfoList.add(ci);
queryInformationSystem.execContextRelationQuery();
queryInformationSystem.getContextRelation(consumer, "contentproviders", "10");
queryInformationSystem
.getContextRelation(consumer, "contentproviders", ModelSupport.entityIdPrefix.get("datasource"));
cInfoList.forEach(contextInfo -> {
switch (contextInfo.getId()) {

View File

@ -0,0 +1,100 @@
package eu.dnetlib.dhp.oa.graph.dump.complete;
import java.io.IOException;
import java.nio.file.Files;
import java.nio.file.Path;
import java.util.HashMap;
import org.apache.commons.io.FileUtils;
import org.apache.spark.SparkConf;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.api.java.function.ForeachFunction;
import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Encoders;
import org.apache.spark.sql.Row;
import org.apache.spark.sql.SparkSession;
import org.junit.jupiter.api.AfterAll;
import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.BeforeAll;
import org.junit.jupiter.api.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import com.fasterxml.jackson.databind.ObjectMapper;
import eu.dnetlib.dhp.schema.oaf.Relation;
import net.sf.saxon.expr.instruct.ForEach;
public class SelectRelationTest {
private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
private static SparkSession spark;
private static Path workingDir;
private static final Logger log = LoggerFactory
.getLogger(SelectRelationTest.class);
private static HashMap<String, String> map = new HashMap<>();
@BeforeAll
public static void beforeAll() throws IOException {
workingDir = Files
.createTempDirectory(SelectRelationTest.class.getSimpleName());
log.info("using work dir {}", workingDir);
SparkConf conf = new SparkConf();
conf.setAppName(SelectRelationTest.class.getSimpleName());
conf.setMaster("local[*]");
conf.set("spark.driver.host", "localhost");
conf.set("hive.metastore.local", "true");
conf.set("spark.ui.enabled", "false");
conf.set("spark.sql.warehouse.dir", workingDir.toString());
conf.set("hive.metastore.warehouse.dir", workingDir.resolve("warehouse").toString());
spark = SparkSession
.builder()
.appName(SelectRelationTest.class.getSimpleName())
.config(conf)
.getOrCreate();
}
@AfterAll
public static void afterAll() throws IOException {
FileUtils.deleteDirectory(workingDir.toFile());
spark.stop();
}
@Test
public void test1() throws Exception {
final String sourcePath = getClass()
.getResource("/eu/dnetlib/dhp/oa/graph/dump/selectrelations")
.getPath();
SparkSelectValidRelationsJob.main(new String[] {
"-isSparkSessionManaged", Boolean.FALSE.toString(),
"-outputPath", workingDir.toString() + "/relation",
"-sourcePath", sourcePath
});
// dumpCommunityProducts.exec(MOCK_IS_LOOK_UP_URL,Boolean.FALSE, workingDir.toString()+"/dataset",sourcePath,"eu.dnetlib.dhp.schema.oaf.Dataset","eu.dnetlib.dhp.schema.dump.oaf.Dataset");
final JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext());
JavaRDD<eu.dnetlib.dhp.schema.oaf.Relation> tmp = sc
.textFile(workingDir.toString() + "/relation")
.map(item -> OBJECT_MAPPER.readValue(item, eu.dnetlib.dhp.schema.oaf.Relation.class));
org.apache.spark.sql.Dataset<Relation> verificationDataset = spark
.createDataset(tmp.rdd(), Encoders.bean(eu.dnetlib.dhp.schema.oaf.Relation.class));
Assertions.assertTrue(verificationDataset.count() == 7);
}
}

View File

@ -75,8 +75,8 @@ public class ResultLinkedToProjectTest {
.getResource("/eu/dnetlib/dhp/oa/graph/dump/funderresource/nomatch/papers.json")
.getPath();
final String relationPath = getClass()
.getResource("/eu/dnetlib/dhp/oa/graph/dump/funderresource/nomatch/relations.json")
final String graphPath = getClass()
.getResource("/eu/dnetlib/dhp/oa/graph/dump/funderresource/nomatch")
.getPath();
SparkResultLinkedToProject.main(new String[] {
@ -84,7 +84,7 @@ public class ResultLinkedToProjectTest {
"-outputPath", workingDir.toString() + "/preparedInfo",
"-sourcePath", sourcePath,
"-resultTableName", "eu.dnetlib.dhp.schema.oaf.Publication",
"-relationPath", relationPath
"-graphPath", graphPath
});
@ -109,7 +109,7 @@ public class ResultLinkedToProjectTest {
.getPath();
final String relationPath = getClass()
.getResource("/eu/dnetlib/dhp/oa/graph/dump/funderresource/match/relations.json")
.getResource("/eu/dnetlib/dhp/oa/graph/dump/funderresource/match")
.getPath();
SparkResultLinkedToProject.main(new String[] {
@ -117,7 +117,7 @@ public class ResultLinkedToProjectTest {
"-outputPath", workingDir.toString() + "/preparedInfo",
"-sourcePath", sourcePath,
"-resultTableName", "eu.dnetlib.dhp.schema.oaf.Publication",
"-relationPath", relationPath
"-graphPath", relationPath
});

View File

@ -75,7 +75,7 @@ public class SplitPerFunderTest {
"-isSparkSessionManaged", Boolean.FALSE.toString(),
"-outputPath", workingDir.toString() + "/split",
"-sourcePath", sourcePath,
"-relationPath", sourcePath
"-graphPath", sourcePath
});
@ -137,5 +137,17 @@ public class SplitPerFunderTest {
.map(item -> OBJECT_MAPPER.readValue(item, CommunityResult.class));
Assertions.assertEquals(3, tmp.count());
// MZOS 1
tmp = sc
.textFile(workingDir.toString() + "/split/MZOS")
.map(item -> OBJECT_MAPPER.readValue(item, CommunityResult.class));
Assertions.assertEquals(1, tmp.count());
// CONICYT 0
tmp = sc
.textFile(workingDir.toString() + "/split/CONICYTF")
.map(item -> OBJECT_MAPPER.readValue(item, CommunityResult.class));
Assertions.assertEquals(0, tmp.count());
}
}

View File

@ -54,7 +54,7 @@ class TestApply extends java.io.Serializable{
assertTrue(pa.getInstance().get(0).getHostedby.getValue.equals("Academic Therapy"))
assertTrue(pa.getInstance().get(0).getAccessright.getClassid.equals("OPEN"))
assertTrue(pa.getInstance().get(0).getAccessright.getClassname.equals("Open Access"))
assertTrue(pa.getInstance().get(0).getAccessright.getOpenAccessRoute.equals(OpenAccessRoute.hybrid))
assertTrue(pa.getInstance().get(0).getAccessright.getOpenAccessRoute.equals(OpenAccessRoute.gold))
assertTrue(pa.getBestaccessright.getClassid.equals("OPEN"))
assertTrue(pa.getBestaccessright.getClassname.equals("Open Access"))

View File

@ -0,0 +1,2 @@
{"resultId":"50|pensoft_____::00ea4a1cd53806a97d62ea6bf268f2a2","projectsList":[{"id":"40|aka_________::0f7d119de1f656b5763a16acf876fed6","code":"123455","acronym":null,"title":"Business services for rural bioenergy entrepreneurship in Finland: a network analysis approach","funder":{"shortName":"AKA","name":"Academy of Finland","jurisdiction":"FI","fundingStream":null},"provenance":{"provenance":"sysimport:crosswalk:entityregistry","trust":"0.900000000000000022"},"validated":{"validationDate":"2021-08-06","validatedByFunder":true}},{"id":"40|aka_________::03376222b28a3aebf2730ac514818d04","code":"119027","acronym":null,"title":"EGFR Tyrosine Kinase Inhibitors and LKB1 Tumor Suppressor in Non-Small-Cell Lung Cancer","funder":{"shortName":"EC","name":"European Commission","jurisdiction":"EU","fundingStream":"H2020"},"provenance":{"provenance":"sysimport:crosswalk:entityregistry","trust":"0.900000000000000022"},"validated":null}]}
{"resultId":"50|dedup_wf_001::51b88f272ba9c3bb181af64e70255a80","projectsList":[{"id":"40|aka_________::0f7d119de1f656b5763a16acf876fed6","code":"123455","acronym":null,"title":"Business services for rural bioenergy entrepreneurship in Finland: a network analysis approach","funder":{"shortName":"AKA","name":"Academy of Finland","jurisdiction":"FI","fundingStream":null},"provenance":{"provenance":"sysimport:crosswalk:entityregistry","trust":"0.900000000000000022"},"validated":{"validationDate":"2021-08-04","validatedByFunder":true}}]}

View File

@ -1,8 +0,0 @@
{"collectedfrom":[{"key":"10|driver______::bee53aa31dc2cbb538c10c2b65fa5824","value":"DOAJ-Articles"}],"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"lastupdatetimestamp":1603715146539,"properties":[],"relClass":"isProducedBy","relType":"datasourceOrganization","source":"10|doajarticles::2baa9032dc058d3c8ff780c426b0c19f","subRelType":"provision","target":"40|cihr________::1e5e62235d094afd01cd56e65112fc63"}
{"collectedfrom":[{"key":"10|driver______::bee53aa31dc2cbb538c10c2b65fa5824","value":"DOAJ-Articles"}],"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"lastupdatetimestamp":1603715146539,"properties":[],"relClass":"isProducedBy","relType":"datasourceOrganization","source":"10|doajarticles::5ac587eb28411c351c2e357eb097fd3d","subRelType":"provision","target":"40|nwo_________::dc69ada721bf21ed51055b6421850d73"}
{"collectedfrom":[{"key":"10|driver______::bee53aa31dc2cbb538c10c2b65fa5824","value":"DOAJ-Articles"}],"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"lastupdatetimestamp":1603715146539,"properties":[],"relClass":"isProducedBy","relType":"datasourceOrganization","source":"10|doajarticles::690b3aaf177a4c70b81bacd8d023cbdc","subRelType":"provision","target":"40|nih_________::031bb5f2f70239b3210eda38b2079f67"}
{"collectedfrom":[{"key":"10|driver______::bee53aa31dc2cbb538c10c2b65fa5824","value":"DOAJ-Articles"}],"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"lastupdatetimestamp":1603715146539,"properties":[],"relClass":"isProducedBy","relType":"datasourceOrganization","source":"10|doajarticles::7a71f278237d1ab35088efda03fa007a","subRelType":"provision","target":"40|nsf_________::03748bcb5d754c951efec9700e18a56d"}
{"collectedfrom":[{"key":"10|driver______::bee53aa31dc2cbb538c10c2b65fa5824","value":"DOAJ-Articles"}],"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"lastupdatetimestamp":1603715146539,"properties":[],"relClass":"isProducedBy","relType":"datasourceOrganization","source":"10|doajarticles::8b75543067b50076e70764917e188178","subRelType":"provision","target":"40|snsf________::50cb15ff7a6a3f8531f063770179e346"}
{"collectedfrom":[{"key":"10|driver______::bee53aa31dc2cbb538c10c2b65fa5824","value":"DOAJ-Articles"}],"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"lastupdatetimestamp":1603715146539,"properties":[],"relClass":"isProducedBy","relType":"datasourceOrganization","source":"10|doajarticles::9f3ff882f023209d9ffb4dc32b77d376","subRelType":"provision","target":"40|corda_______::ffc1811633b3222e4764c7b0517f83e8"}
{"collectedfrom":[{"key":"10|driver______::bee53aa31dc2cbb538c10c2b65fa5824","value":"DOAJ-Articles"}],"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"lastupdatetimestamp":1603715146539,"properties":[],"relClass":"isProducedBy","relType":"datasourceOrganization","source":"10|doajarticles::b566fa319c3923454e1e8eb886ab62d2","subRelType":"provision","target":"40|nhmrc_______::4e6c928fef9851b37ec73f4f6daca35b"}
{"collectedfrom":[{"key":"10|driver______::bee53aa31dc2cbb538c10c2b65fa5824","value":"DOAJ-Articles"}],"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"lastupdatetimestamp":1603715146539,"properties":[],"relClass":"isProducedBy","relType":"datasourceOrganization","source":"10|doajarticles::e0554fb004a155bc23cfb43ee9fc8eae","subRelType":"provision","target":"40|corda__h2020::846b777af165fef7c904a81712a83b66"}

File diff suppressed because one or more lines are too long

View File

@ -22,8 +22,8 @@
{"collectedfrom":[{"key":"10|openaire____::21f8a223b9925c2f87c404096080b046","value":"Registry of Research Data Repository"}],"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"lastupdatetimestamp":1603715146539,"properties":[],"relClass":"isProvidedBy","relType":"datasourceOrganization","source":"10|re3data_____::cafe7980294aa5f935f433e7c8aab844","subRelType":"provision","target":"20|dedup_wf_001::2806db65ba8029ee196679cad067eff2"}
{"collectedfrom":[{"key":"10|openaire____::6ac933301a3933c8a22ceebea7000326","value":"Academy of Finland"}],"dataInfo":{"deletedbyinference":true,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900000000000000022"},"lastupdatetimestamp":1603715146539,"properties":[],"relClass":"isParticipant","relType":"projectOrganization","source":"20|aka_________::0cd5965141113df5739f1ac7ac7f6d37","subRelType":"participation","target":"40|aka_________::1bc716a1763110da3eb1af867de718a8"}
{"collectedfrom":[{"key":"10|openaire____::6ac933301a3933c8a22ceebea7000326","value":"Academy of Finland"}],"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900000000000000022"},"lastupdatetimestamp":1603715146539,"properties":[],"relClass":"isParticipant","relType":"projectOrganization","source":"20|aka_________::1e2df822bf0932ad0f77565789f22e17","subRelType":"participation","target":"40|aka_________::a6c805bcfd383bae043d8df38e79db78"}
{"collectedfrom":[],"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"iis::document_affiliations","inferred":true,"invisible":false,"provenanceaction":{"classid":"iis","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.8966"},"lastupdatetimestamp":1603564783812,"properties":[],"relClass":"isProducedBy","relType":"resultOrganization","target":"20|aka_________::2c3aab6bce7516338b4dbfb4f6f86db7","subRelType":"affiliation","source":"50|a89337edbe55::43e8b61e5e8d682545cb867be8118585"}
{"collectedfrom":[{"key":"10|openaire____::6ac933301a3933c8a22ceebea7000326","value":"Academy of Finland"}],"dataInfo":{"deletedbyinference":true,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900000000000000022"},"lastupdatetimestamp":1603715146539,"properties":[],"relClass":"isParticipant","relType":"projectOrganization","source":"20|aka_________::8bcd11b6bffc76f4d4f88d7a6728d614","subRelType":"participation","target":"40|aka_________::645123c3fe7bab557c36f0f9bb02a4cd"}
{"collectedfrom":[],"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"iis::document_affiliations","inferred":true,"invisible":false,"provenanceaction":{"classid":"iis","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.8966"},"lastupdatetimestamp":1603564783812,"properties":[],"relClass":"isProducedBy","relType":"resultOrganization","target":"40|aka_________::01bb7b48e29d732a1c7bc5150b9195c4","subRelType":"affiliation","source":"50|a89337edbe55::43e8b61e5e8d682545cb867be8118585"}
{"collectedfrom":[],"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"iis::document_affiliations","inferred":true,"invisible":false,"provenanceaction":{"classid":"iis","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.8966"},"lastupdatetimestamp":1603564783812,"properties":[],"relClass":"isProducedBy","relType":"resultOrganization","target":"40|aka_________::9d1af21dbd0f5bc719f71553d19a6b3a","subRelType":"affiliation","source":"50|a89337edbe55::43e8b61e5e8d682545cb867be8118585"}
{"collectedfrom":[{"key":"10|openaire____::6ac933301a3933c8a22ceebea7000326","value":"Academy of Finland"}],"dataInfo":{"deletedbyinference":true,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900000000000000022"},"lastupdatetimestamp":1603715146539,"properties":[],"relClass":"isParticipant","relType":"projectOrganization","source":"20|aka_________::8bcd11b6bffc76f4d4f88d7a6728d614","subRelType":"participation","target":"40|aka_________::fecf4f862a6b40dd2ccb1abc8fed5bc5"}
{"collectedfrom":[{"key":"10|openaire____::6ac933301a3933c8a22ceebea7000326","value":"Academy of Finland"}],"dataInfo":{"deletedbyinference":true,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900000000000000022"},"lastupdatetimestamp":1603715146539,"properties":[],"relClass":"isParticipant","relType":"projectOrganization","source":"20|aka_________::a30b9a45766293af38951d767e77a471","subRelType":"participation","target":"40|aka_________::68ce7288b1b036f73a1ff951c6524eba"}
{"collectedfrom":[{"key":"10|openaire____::6ac933301a3933c8a22ceebea7000326","value":"Academy of Finland"}],"dataInfo":{"deletedbyinference":true,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900000000000000022"},"lastupdatetimestamp":1603715146539,"properties":[],"relClass":"isParticipant","relType":"projectOrganization","source":"20|aka_________::a30b9a45766293af38951d767e77a471","subRelType":"participation","target":"40|aka_________::b733217d1cd609001dd3c75af419d872"}

View File

@ -0,0 +1,30 @@
{"collectedfrom":[{"key":"10|driver______::bee53aa31dc2cbb538c10c2b65fa5824","value":"DOAJ-Articles"}],"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"lastupdatetimestamp":1592688952862,"properties":[],"relClass":"isProducedBy","relType":"datasourceOrganization","source":"40|doajarticles::2baa9032dc058d3c8ff780c426b0c19f","subRelType":"provision","target":"50|dedup_wf_001::2899e571609779168222fdeb59cb916d", "validated":true, "validationDate":"2021-08-06"}
{"collectedfrom":[{"key":"10|driver______::bee53aa31dc2cbb538c10c2b65fa5824","value":"DOAJ-Articles"}],"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"lastupdatetimestamp":1592688952862,"properties":[],"relClass":"isProducedBy","relType":"datasourceOrganization","source":"40|doajarticles::2baa9032dc058d3c8ff780c426b0c19f","subRelType":"provision","target":"50|dedup_wf_001::396262ee936f3d3e26ff0e60bea6cae0", "validated":true, "validationDate":"2021-08-06"}
{"collectedfrom":[{"key":"10|driver______::bee53aa31dc2cbb538c10c2b65fa5824","value":"DOAJ-Articles"}],"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"lastupdatetimestamp":1592688952862,"properties":[],"relClass":"isProvidedBy","relType":"datasourceOrganization","source":"10|doajarticles::690b3aaf177a4c70b81bacd8d023cbdc","subRelType":"provision","target":"20|doajarticles::396262ee936f3d3e26ff0e60bea6cae0"}
{"collectedfrom":[{"key":"10|driver______::bee53aa31dc2cbb538c10c2b65fa5824","value":"DOAJ-Articles"}],"dataInfo":{"deletedbyinference":true,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"lastupdatetimestamp":1592688952862,"properties":[],"relClass":"isProvidedBy","relType":"datasourceOrganization","source":"10|doajarticles::7a71f278237d1ab35088efda03fa007a","subRelType":"provision","target":"20|doajarticles::03748bcb5d754c951efec9700e18a56d"}
{"collectedfrom":[{"key":"10|driver______::bee53aa31dc2cbb538c10c2b65fa5824","value":"DOAJ-Articles"}],"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"lastupdatetimestamp":1592688952862,"properties":[],"relClass":"isProvidedBy","relType":"datasourceOrganization","source":"10|doajarticles::8b75543067b50076e70764917e188178","subRelType":"provision","target":"20|doajarticles::50cb15ff7a6a3f8531f063770179e346"}
{"collectedfrom":[{"key":"10|driver______::bee53aa31dc2cbb538c10c2b65fa5824","value":"DOAJ-Articles"}],"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"lastupdatetimestamp":1592688952862,"properties":[],"relClass":"isProvidedBy","relType":"datasourceOrganization","source":"10|doajarticles::9f3ff882f023209d9ffb4dc32b77d376","subRelType":"provision","target":"20|doajarticles::ffc1811633b3222e4764c7b0517f83e8"}
{"collectedfrom":[{"key":"10|driver______::bee53aa31dc2cbb538c10c2b65fa5824","value":"DOAJ-Articles"}],"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"lastupdatetimestamp":1592688952862,"properties":[],"relClass":"isProvidedBy","relType":"datasourceOrganization","source":"10|doajarticles::b566fa319c3923454e1e8eb886ab62d2","subRelType":"provision","target":"20|dedup_wf_001::4e6c928fef9851b37ec73f4f6daca35b"}
{"collectedfrom":[{"key":"10|driver______::bee53aa31dc2cbb538c10c2b65fa5824","value":"DOAJ-Articles"}],"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"lastupdatetimestamp":1592688952862,"properties":[],"relClass":"isProvidedBy","relType":"datasourceOrganization","source":"10|doajarticles::e0554fb004a155bc23cfb43ee9fc8eae","subRelType":"provision","target":"20|dedup_wf_001::846b777af165fef7c904a81712a83b66"}
{"collectedfrom":[{"key":"10|driver______::bee53aa31dc2cbb538c10c2b65fa5824","value":"DOAJ-Articles"}],"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"lastupdatetimestamp":1592688952862,"properties":[],"relClass":"isProvidedBy","relType":"datasourceOrganization","source":"10|doajarticles::fbf7592ddbf2ad3cc0ed70c0f2e1d67c","subRelType":"provision","target":"20|dedup_wf_001::1b965e2c0c53e5526d269d63bcfa0ae6"}
{"collectedfrom":[{"key":"10|driver______::bee53aa31dc2cbb538c10c2b65fa5824","value":"DOAJ-Articles"}],"dataInfo":{"deletedbyinference":true,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"lastupdatetimestamp":1592688952862,"properties":[],"relClass":"isProvidedBy","relType":"datasourceOrganization","source":"10|doajarticles::fd4c399077127f0ba09b5205e2b78406","subRelType":"provision","target":"20|doajarticles::1cae0b82b56ccd97c2db1f698def7074"}
{"collectedfrom":[{"key":"10|infrastruct_::f66f1bd369679b5b077dcdf006089556","value":"OpenAIRE"}],"dataInfo":{"deletedbyinference":true,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"lastupdatetimestamp":1594398578323,"properties":[],"relClass":"isProvidedBy","relType":"datasourceOrganization","source":"10|openaire____::8f991165fae922e29ad55d592f568464","subRelType":"provision","target":"20|openaire____::ec653e804967133b9436fdd30d3ff51d"}
{"collectedfrom":[{"key":"10|openaire____::47ce9e9f4fad46e732cff06419ecaabb","value":"OpenDOAR"}],"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"lastupdatetimestamp":1592688952862,"properties":[],"relClass":"isProvidedBy","relType":"datasourceOrganization","source":"10|opendoar____::15231a7ce4ba789d13b722cc5c955834","subRelType":"provision","target":"20|dedup_wf_001::1ea4bcb1bae8c6befef1e7f1230f0f10"}
{"collectedfrom":[{"key":"10|openaire____::47ce9e9f4fad46e732cff06419ecaabb","value":"OpenDOAR"}],"dataInfo":{"deletedbyinference":true,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"lastupdatetimestamp":1592688952862,"properties":[],"relClass":"isProvidedBy","relType":"datasourceOrganization","source":"10|opendoar____::16d11e9595188dbad0418a85f0351aba","subRelType":"provision","target":"20|opendoar____::041abd8c990fc531ab9bd2674a0e2725"}
{"collectedfrom":[{"key":"10|openaire____::47ce9e9f4fad46e732cff06419ecaabb","value":"OpenDOAR"}],"dataInfo":{"deletedbyinference":true,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"lastupdatetimestamp":1592688952862,"properties":[],"relClass":"isProvidedBy","relType":"datasourceOrganization","source":"10|opendoar____::46d3f6029f6170ebccb28945964d09bf","subRelType":"provision","target":"20|opendoar____::a5fcb8eb25ebd6f7cd219e0fa1e6ddc1"}
{"collectedfrom":[{"key":"10|openaire____::47ce9e9f4fad46e732cff06419ecaabb","value":"OpenDOAR"}],"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"lastupdatetimestamp":1592688952862,"properties":[],"relClass":"isProvidedBy","relType":"datasourceOrganization","source":"10|opendoar____::7501e5d4da87ac39d782741cd794002d","subRelType":"provision","target":"20|dedup_wf_001::04e2c34ef4daa411ff2497afc807b612"}
{"collectedfrom":[{"key":"10|openaire____::47ce9e9f4fad46e732cff06419ecaabb","value":"OpenDOAR"}],"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"lastupdatetimestamp":1592688952862,"properties":[],"relClass":"isProvidedBy","relType":"datasourceOrganization","source":"10|opendoar____::75b9b6dc7fe44437c6e0a69fd863dbab","subRelType":"provision","target":"20|dedup_wf_001::ad30fbc9b3b6f5370e59e58c456b7e19"}
{"collectedfrom":[{"key":"10|openaire____::47ce9e9f4fad46e732cff06419ecaabb","value":"OpenDOAR"}],"dataInfo":{"deletedbyinference":true,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"lastupdatetimestamp":1592688952862,"properties":[],"relClass":"isProvidedBy","relType":"datasourceOrganization","source":"10|opendoar____::d35b05a832e2bb91f110d54e34e2da79","subRelType":"provision","target":"20|opendoar____::589618708434cfc5b830601ac4b339ee"}
{"collectedfrom":[{"key":"10|openaire____::47ce9e9f4fad46e732cff06419ecaabb","value":"OpenDOAR"}],"dataInfo":{"deletedbyinference":true,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"lastupdatetimestamp":1592688952862,"properties":[],"relClass":"isProvidedBy","relType":"datasourceOrganization","source":"10|opendoar____::fc2e6a440b94f64831840137698021e1","subRelType":"provision","target":"20|opendoar____::a82c7e358792e0018235b7f196fec4ed"}
{"collectedfrom":[{"key":"10|openaire____::21f8a223b9925c2f87c404096080b046","value":"Registry of Research Data Repository"}],"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"lastupdatetimestamp":1592688952862,"properties":[],"relClass":"isProvidedBy","relType":"datasourceOrganization","source":"10|re3data_____::48b3cc4b8c5951621730829c60b1c205","subRelType":"provision","target":"20|dedup_wf_001::5bc6fca7649010470f1cc11f6675ffb3"}
{"collectedfrom":[{"key":"10|openaire____::21f8a223b9925c2f87c404096080b046","value":"Registry of Research Data Repository"}],"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"lastupdatetimestamp":1592688952862,"properties":[],"relClass":"isProvidedBy","relType":"datasourceOrganization","source":"10|re3data_____::8e7a079ee6d2d4933db9b898c789c2f8","subRelType":"provision","target":"20|dedup_wf_001::0b0ae5b38aedc082a8b089abdf3b752f"}
{"collectedfrom":[{"key":"10|openaire____::21f8a223b9925c2f87c404096080b046","value":"Registry of Research Data Repository"}],"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"lastupdatetimestamp":1592688952862,"properties":[],"relClass":"isProvidedBy","relType":"datasourceOrganization","source":"10|re3data_____::b26c86eba2b3ad8b242a93b581e6ec8e","subRelType":"provision","target":"20|re3data_____::e841a40265d8d0a6739ac71f56328da3"}
{"collectedfrom":[{"key":"10|openaire____::21f8a223b9925c2f87c404096080b046","value":"Registry of Research Data Repository"}],"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"lastupdatetimestamp":1592688952862,"properties":[],"relClass":"isProvidedBy","relType":"datasourceOrganization","source":"10|re3data_____::cafe7980294aa5f935f433e7c8aab844","subRelType":"provision","target":"20|dedup_wf_001::2806db65ba8029ee196679cad067eff2"}
{"collectedfrom":[{"key":"10|openaire____::6ac933301a3933c8a22ceebea7000326","value":"Academy of Finland"}],"dataInfo":{"deletedbyinference":true,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900000000000000022"},"lastupdatetimestamp":1592688952862,"properties":[],"relClass":"isParticipant","relType":"projectOrganization","source":"20|aka_________::0cd5965141113df5739f1ac7ac7f6d37","subRelType":"participation","target":"40|aka_________::1bc716a1763110da3eb1af867de718a8"}
{"collectedfrom":[{"key":"10|openaire____::6ac933301a3933c8a22ceebea7000326","value":"Academy of Finland"}],"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900000000000000022"},"lastupdatetimestamp":1592688952862,"properties":[],"relClass":"isParticipant","relType":"projectOrganization","source":"20|aka_________::1e2df822bf0932ad0f77565789f22e17","subRelType":"participation","target":"40|aka_________::a6c805bcfd383bae043d8df38e79db78"}
{"collectedfrom":[],"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"iis::document_affiliations","inferred":true,"invisible":false,"provenanceaction":{"classid":"iis","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.8966"},"lastupdatetimestamp":1595258695262,"properties":[],"relClass":"isAuthorInstitutionOf","relType":"resultOrganization","source":"20|aka_________::2c3aab6bce7516338b4dbfb4f6f86db7","subRelType":"affiliation","target":"50|dedup_wf_001::02859c30f6c8bfbdd8c427068a6ec684"}
{"collectedfrom":[{"key":"10|openaire____::6ac933301a3933c8a22ceebea7000326","value":"Academy of Finland"}],"dataInfo":{"deletedbyinference":true,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900000000000000022"},"lastupdatetimestamp":1592688952862,"properties":[],"relClass":"isParticipant","relType":"projectOrganization","source":"20|aka_________::8bcd11b6bffc76f4d4f88d7a6728d614","subRelType":"participation","target":"40|aka_________::645123c3fe7bab557c36f0f9bb02a4cd"}
{"collectedfrom":[{"key":"10|openaire____::6ac933301a3933c8a22ceebea7000326","value":"Academy of Finland"}],"dataInfo":{"deletedbyinference":true,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900000000000000022"},"lastupdatetimestamp":1592688952862,"properties":[],"relClass":"isParticipant","relType":"projectOrganization","source":"20|aka_________::8bcd11b6bffc76f4d4f88d7a6728d614","subRelType":"participation","target":"40|aka_________::fecf4f862a6b40dd2ccb1abc8fed5bc5"}
{"collectedfrom":[{"key":"10|openaire____::6ac933301a3933c8a22ceebea7000326","value":"Academy of Finland"}],"dataInfo":{"deletedbyinference":true,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900000000000000022"},"lastupdatetimestamp":1592688952862,"properties":[],"relClass":"isParticipant","relType":"projectOrganization","source":"20|aka_________::a30b9a45766293af38951d767e77a471","subRelType":"participation","target":"40|aka_________::b733217d1cd609001dd3c75af419d872"}
{"collectedfrom":[{"key":"10|openaire____::6ac933301a3933c8a22ceebea7000326","value":"Academy of Finland"}],"dataInfo":{"deletedbyinference":true,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900000000000000022"},"lastupdatetimestamp":1592688952862,"properties":[],"relClass":"isParticipant","relType":"projectOrganization","source":"20|aka_________::a30b9a45766293af38951d767e77a471","subRelType":"participation","target":"40|aka_________::c33dee8231ad0374caf93e52c5a473e5"}
{"collectedfrom":[{"key":"10|openaire____::6ac933301a3933c8a22ceebea7000326","value":"Academy of Finland"}],"dataInfo":{"deletedbyinference":true,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900000000000000022"},"lastupdatetimestamp":1592688952862,"properties":[],"relClass":"isParticipant","relType":"projectOrganization","source":"20|aka_________::ddcb6d65425e4531a11c610488d42d81","subRelType":"participation","target":"40|aka_________::4d7027ac28c8ac9cc72b062ac4992b4e"}

File diff suppressed because one or more lines are too long

Some files were not shown because too many files have changed in this diff Show More