diff --git a/dhp-doc-resources/img/data_provision_workflow.png b/dhp-doc-resources/img/data_provision_workflow.png
new file mode 100644
index 0000000000..31979fbb49
Binary files /dev/null and b/dhp-doc-resources/img/data_provision_workflow.png differ
diff --git a/dhp-schemas/pom.xml b/dhp-schemas/pom.xml
index 89e52858be..8338f69e45 100644
--- a/dhp-schemas/pom.xml
+++ b/dhp-schemas/pom.xml
@@ -30,7 +30,12 @@
com.fasterxml.jackson.core
jackson-databind
-
+
+
+ com.google.guava
+ guava
+
+
junit
junit
diff --git a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/action/AtomicAction.java b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/action/AtomicAction.java
new file mode 100644
index 0000000000..0f9aa3adbc
--- /dev/null
+++ b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/action/AtomicAction.java
@@ -0,0 +1,38 @@
+package eu.dnetlib.dhp.schema.action;
+
+import com.fasterxml.jackson.databind.annotation.JsonDeserialize;
+import eu.dnetlib.dhp.schema.oaf.Oaf;
+
+import java.io.Serializable;
+
+@JsonDeserialize(using = AtomicActionDeserializer.class)
+public class AtomicAction implements Serializable {
+
+ private Class clazz;
+
+ private T payload;
+
+ public AtomicAction() {
+ }
+
+ public AtomicAction(Class clazz, T payload) {
+ this.clazz = clazz;
+ this.payload = payload;
+ }
+
+ public Class getClazz() {
+ return clazz;
+ }
+
+ public void setClazz(Class clazz) {
+ this.clazz = clazz;
+ }
+
+ public T getPayload() {
+ return payload;
+ }
+
+ public void setPayload(T payload) {
+ this.payload = payload;
+ }
+}
diff --git a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/action/AtomicActionDeserializer.java b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/action/AtomicActionDeserializer.java
new file mode 100644
index 0000000000..e6017288fa
--- /dev/null
+++ b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/action/AtomicActionDeserializer.java
@@ -0,0 +1,29 @@
+package eu.dnetlib.dhp.schema.action;
+
+import com.fasterxml.jackson.core.JsonParser;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.DeserializationContext;
+import com.fasterxml.jackson.databind.JsonDeserializer;
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import eu.dnetlib.dhp.schema.oaf.Oaf;
+
+import java.io.IOException;
+
+public class AtomicActionDeserializer extends JsonDeserializer {
+
+ @Override
+ public Object deserialize(JsonParser jp, DeserializationContext ctxt) throws IOException, JsonProcessingException {
+ JsonNode node = jp.getCodec().readTree(jp);
+ String classTag = node.get("clazz").asText();
+ JsonNode payload = node.get("payload");
+ ObjectMapper mapper = new ObjectMapper();
+
+ try {
+ final Class> clazz = Class.forName(classTag);
+ return new AtomicAction(clazz, (Oaf) mapper.readValue(payload.toString(), clazz));
+ } catch (ClassNotFoundException e) {
+ throw new IOException(e);
+ }
+ }
+}
diff --git a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Relation.java b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Relation.java
index 466062e8a9..6738b86938 100644
--- a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Relation.java
+++ b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Relation.java
@@ -6,7 +6,7 @@ import java.util.Objects;
import java.util.stream.Collectors;
import java.util.stream.Stream;
-import org.junit.Assert;
+import static com.google.common.base.Preconditions.checkArgument;
public class Relation extends Oaf {
@@ -71,11 +71,13 @@ public class Relation extends Oaf {
}
public void mergeFrom(final Relation r) {
- Assert.assertEquals("source ids must be equal", getSource(), r.getSource());
- Assert.assertEquals("target ids must be equal", getTarget(), r.getTarget());
- Assert.assertEquals("relType(s) must be equal", getRelType(), r.getRelType());
- Assert.assertEquals("subRelType(s) must be equal", getSubRelType(), r.getSubRelType());
- Assert.assertEquals("relClass(es) must be equal", getRelClass(), r.getRelClass());
+
+ checkArgument(Objects.equals(getSource(), r.getSource()),"source ids must be equal");
+ checkArgument(Objects.equals(getTarget(), r.getTarget()),"target ids must be equal");
+ checkArgument(Objects.equals(getRelType(), r.getRelType()),"relType(s) must be equal");
+ checkArgument(Objects.equals(getSubRelType(), r.getSubRelType()),"subRelType(s) must be equal");
+ checkArgument(Objects.equals(getRelClass(), r.getRelClass()),"relClass(es) must be equal");
+
setCollectedFrom(Stream.concat(getCollectedFrom().stream(), r.getCollectedFrom().stream())
.distinct() // relies on KeyValue.equals
.collect(Collectors.toList()));
@@ -85,18 +87,18 @@ public class Relation extends Oaf {
public boolean equals(Object o) {
if (this == o) return true;
if (o == null || getClass() != o.getClass()) return false;
- if (!super.equals(o)) return false;
Relation relation = (Relation) o;
- return Objects.equals(relType, relation.relType) &&
- Objects.equals(subRelType, relation.subRelType) &&
- Objects.equals(relClass, relation.relClass) &&
- Objects.equals(source, relation.source) &&
- Objects.equals(target, relation.target) &&
+ return relType.equals(relation.relType) &&
+ subRelType.equals(relation.subRelType) &&
+ relClass.equals(relation.relClass) &&
+ source.equals(relation.source) &&
+ target.equals(relation.target) &&
Objects.equals(collectedFrom, relation.collectedFrom);
}
@Override
public int hashCode() {
- return Objects.hash(super.hashCode(), relType, subRelType, relClass, source, target, collectedFrom);
+ return Objects.hash(relType, subRelType, relClass, source, target, collectedFrom);
}
+
}
diff --git a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Result.java b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Result.java
index 3c32d02c7c..d04272d522 100644
--- a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Result.java
+++ b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Result.java
@@ -5,7 +5,7 @@ import java.util.Comparator;
import java.util.List;
import java.util.Objects;
-public abstract class Result extends OafEntity implements Serializable {
+public class Result extends OafEntity implements Serializable {
private List author;
diff --git a/dhp-schemas/src/test/java/eu/dnetlib/dhp/schema/action/AtomicActionTest.java b/dhp-schemas/src/test/java/eu/dnetlib/dhp/schema/action/AtomicActionTest.java
new file mode 100644
index 0000000000..dcf20e3425
--- /dev/null
+++ b/dhp-schemas/src/test/java/eu/dnetlib/dhp/schema/action/AtomicActionTest.java
@@ -0,0 +1,37 @@
+package eu.dnetlib.dhp.schema.action;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import eu.dnetlib.dhp.schema.oaf.Relation;
+import org.apache.commons.lang3.StringUtils;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.IOException;
+
+public class AtomicActionTest {
+
+ @Test
+ public void serializationTest() throws IOException {
+
+ Relation rel = new Relation();
+ rel.setSource("1");
+ rel.setTarget("2");
+ rel.setRelType("resultResult");
+ rel.setSubRelType("dedup");
+ rel.setRelClass("merges");
+
+ AtomicAction aa1 = new AtomicAction(Relation.class, rel);
+
+ final ObjectMapper mapper = new ObjectMapper();
+ String json = mapper.writeValueAsString(aa1);
+
+ Assert.assertTrue(StringUtils.isNotBlank(json));
+
+ AtomicAction aa2 = mapper.readValue(json, AtomicAction.class);
+
+ Assert.assertEquals(aa1.getClazz(), aa2.getClazz());
+ Assert.assertEquals(aa1.getPayload(), aa2.getPayload());
+
+ }
+
+}
diff --git a/dhp-workflows/dhp-aggregation/pom.xml b/dhp-workflows/dhp-aggregation/pom.xml
index 78831073fd..09dac83499 100644
--- a/dhp-workflows/dhp-aggregation/pom.xml
+++ b/dhp-workflows/dhp-aggregation/pom.xml
@@ -24,6 +24,12 @@
eu.dnetlib.dhp
dhp-common
${project.version}
+
+
+ com.sun.xml.bind
+ jaxb-core
+
+
@@ -32,6 +38,49 @@
${project.version}
+
+ eu.dnetlib
+ dnet-actionmanager-common
+
+
+ eu.dnetlib
+ dnet-openaireplus-mapping-utils
+
+
+ saxonica
+ saxon
+
+
+ saxonica
+ saxon-dom
+
+
+ jgrapht
+ jgrapht
+
+
+ net.sf.ehcache
+ ehcache
+
+
+ org.springframework
+ spring-test
+
+
+ org.apache.*
+ *
+
+
+ apache
+ *
+
+
+
+
+ eu.dnetlib
+ dnet-openaire-data-protos
+
+
net.sf.saxon
Saxon-HE
@@ -55,6 +104,11 @@
org.mongodb
mongo-java-driver
+
+
+ org.apache.hadoop
+ hadoop-distcp
+
org.postgresql
diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/ExtractEntitiesFromHDFSJob.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/ExtractEntitiesFromHDFSJob.java
deleted file mode 100644
index 3b6fc9b5de..0000000000
--- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/ExtractEntitiesFromHDFSJob.java
+++ /dev/null
@@ -1,94 +0,0 @@
-package eu.dnetlib.dhp.migration;
-
-import java.io.IOException;
-import java.util.Arrays;
-import java.util.List;
-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.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.Text;
-import org.apache.spark.api.java.JavaRDD;
-import org.apache.spark.api.java.JavaSparkContext;
-import org.apache.spark.sql.SparkSession;
-
-import eu.dnetlib.dhp.application.ArgumentApplicationParser;
-import eu.dnetlib.dhp.schema.oaf.Dataset;
-import eu.dnetlib.dhp.schema.oaf.Datasource;
-import eu.dnetlib.dhp.schema.oaf.Organization;
-import eu.dnetlib.dhp.schema.oaf.OtherResearchProduct;
-import eu.dnetlib.dhp.schema.oaf.Project;
-import eu.dnetlib.dhp.schema.oaf.Publication;
-import eu.dnetlib.dhp.schema.oaf.Relation;
-import eu.dnetlib.dhp.schema.oaf.Software;
-import scala.Tuple2;
-
-public class ExtractEntitiesFromHDFSJob {
-
- private static final Log log = LogFactory.getLog(ExtractEntitiesFromHDFSJob.class);
-
- public static void main(final String[] args) throws Exception {
- final ArgumentApplicationParser parser = new ArgumentApplicationParser(
- IOUtils.toString(MigrateMongoMdstoresApplication.class
- .getResourceAsStream("/eu/dnetlib/dhp/migration/extract_entities_from_hdfs_parameters.json")));
- parser.parseArgument(args);
-
- final SparkSession spark = SparkSession
- .builder()
- .appName(ExtractEntitiesFromHDFSJob.class.getSimpleName())
- .master(parser.get("master"))
- .getOrCreate();
-
- try (final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext())) {
-
- final List sourcePaths = Arrays.stream(parser.get("sourcePaths").split(",")).filter(p -> exists(sc, p)).collect(Collectors.toList());
- final String targetPath = parser.get("graphRawPath");
-
- processEntity(sc, Publication.class, sourcePaths, targetPath);
- processEntity(sc, Dataset.class, sourcePaths, targetPath);
- processEntity(sc, Software.class, sourcePaths, targetPath);
- processEntity(sc, OtherResearchProduct.class, sourcePaths, targetPath);
- processEntity(sc, Datasource.class, sourcePaths, targetPath);
- processEntity(sc, Organization.class, sourcePaths, targetPath);
- processEntity(sc, Project.class, sourcePaths, targetPath);
- processEntity(sc, Relation.class, sourcePaths, targetPath);
- }
- }
-
- private static void processEntity(final JavaSparkContext sc, final Class> clazz, final List sourcePaths, final String targetPath) {
- final String type = clazz.getSimpleName().toLowerCase();
-
- log.info(String.format("Processing entities (%s) in files:", type));
- sourcePaths.forEach(log::info);
-
- JavaRDD inputRdd = sc.emptyRDD();
-
- for (final String sp : sourcePaths) {
- inputRdd = inputRdd.union(sc.sequenceFile(sp, Text.class, Text.class)
- .map(k -> new Tuple2<>(k._1().toString(), k._2().toString()))
- .filter(k -> isEntityType(k._1(), type))
- .map(Tuple2::_2));
- }
-
- inputRdd.saveAsTextFile(targetPath + "/" + type);
-
- }
-
- private static boolean isEntityType(final String item, final String type) {
- return StringUtils.substringAfter(item, ":").equalsIgnoreCase(type);
- }
-
- private static boolean exists(final JavaSparkContext context, final String pathToFile) {
- try {
- final FileSystem hdfs = org.apache.hadoop.fs.FileSystem.get(context.hadoopConfiguration());
- final Path path = new Path(pathToFile);
- return hdfs.exists(path);
- } catch (final IOException e) {
- throw new RuntimeException(e);
- }
- }
-}
diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/MigrateMongoMdstoresApplication.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/MigrateMongoMdstoresApplication.java
deleted file mode 100644
index 359fe7596b..0000000000
--- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/MigrateMongoMdstoresApplication.java
+++ /dev/null
@@ -1,45 +0,0 @@
-package eu.dnetlib.dhp.migration;
-
-import org.apache.commons.io.IOUtils;
-
-import eu.dnetlib.dhp.application.ArgumentApplicationParser;
-
-public class MigrateMongoMdstoresApplication {
-
- public static void main(final String[] args) throws Exception {
- final ArgumentApplicationParser parser = new ArgumentApplicationParser(
- IOUtils.toString(MigrateMongoMdstoresApplication.class.getResourceAsStream("/eu/dnetlib/dhp/migration/migrate_mongo_mstores_parameters.json")));
- parser.parseArgument(args);
-
- final String mongoBaseUrl = parser.get("mongoBaseUrl");
- final String mongoDb = parser.get("mongoDb");
-
- final String mdFormat = parser.get("mdFormat");
- final String mdLayout = parser.get("mdLayout");
- final String mdInterpretation = parser.get("mdInterpretation");
-
- final String hdfsPath = parser.get("hdfsPath");
- final String hdfsNameNode = parser.get("namenode");
- final String hdfsUser = parser.get("hdfsUser");
-
- final String dbUrl = parser.get("postgresUrl");
- final String dbUser = parser.get("postgresUser");
- final String dbPassword = parser.get("postgresPassword");
-
- if (mdFormat.equalsIgnoreCase("oaf")) {
- try (final OafMigrationExecutor mig =
- new OafMigrationExecutor(hdfsPath, hdfsNameNode, hdfsUser, mongoBaseUrl, mongoDb, dbUrl, dbUser, dbPassword)) {
- mig.processMdRecords(mdFormat, mdLayout, mdInterpretation);
- }
- } else if (mdFormat.equalsIgnoreCase("odf")) {
- try (final OdfMigrationExecutor mig =
- new OdfMigrationExecutor(hdfsPath, hdfsNameNode, hdfsUser, mongoBaseUrl, mongoDb, dbUrl, dbUser, dbPassword)) {
- mig.processMdRecords(mdFormat, mdLayout, mdInterpretation);
- }
- } else {
- throw new RuntimeException("Format not supported: " + mdFormat);
- }
-
- }
-
-}
diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/actions/LicenseComparator.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/actions/LicenseComparator.java
new file mode 100644
index 0000000000..9d0e82aca9
--- /dev/null
+++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/actions/LicenseComparator.java
@@ -0,0 +1,49 @@
+package eu.dnetlib.dhp.migration.actions;
+
+import eu.dnetlib.data.proto.FieldTypeProtos.Qualifier;
+
+import java.util.Comparator;
+
+public class LicenseComparator implements Comparator {
+
+ @Override
+ public int compare(Qualifier left, Qualifier right) {
+
+ if (left == null && right == null) return 0;
+ if (left == null) return 1;
+ if (right == null) return -1;
+
+ String lClass = left.getClassid();
+ String rClass = right.getClassid();
+
+ if (lClass.equals(rClass)) return 0;
+
+ if (lClass.equals("OPEN SOURCE")) return -1;
+ if (rClass.equals("OPEN SOURCE")) return 1;
+
+ if (lClass.equals("OPEN")) return -1;
+ if (rClass.equals("OPEN")) return 1;
+
+ if (lClass.equals("6MONTHS")) return -1;
+ if (rClass.equals("6MONTHS")) return 1;
+
+ if (lClass.equals("12MONTHS")) return -1;
+ if (rClass.equals("12MONTHS")) return 1;
+
+ if (lClass.equals("EMBARGO")) return -1;
+ if (rClass.equals("EMBARGO")) return 1;
+
+ if (lClass.equals("RESTRICTED")) return -1;
+ if (rClass.equals("RESTRICTED")) return 1;
+
+ if (lClass.equals("CLOSED")) return -1;
+ if (rClass.equals("CLOSED")) return 1;
+
+ if (lClass.equals("UNKNOWN")) return -1;
+ if (rClass.equals("UNKNOWN")) return 1;
+
+ // Else (but unlikely), lexicographical ordering will do.
+ return lClass.compareTo(rClass);
+ }
+
+}
diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/actions/MigrateActionSet.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/actions/MigrateActionSet.java
new file mode 100644
index 0000000000..487fac359c
--- /dev/null
+++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/actions/MigrateActionSet.java
@@ -0,0 +1,170 @@
+package eu.dnetlib.dhp.migration.actions;
+
+import com.google.common.base.Splitter;
+import com.google.common.collect.Lists;
+import eu.dnetlib.dhp.application.ArgumentApplicationParser;
+import eu.dnetlib.dhp.utils.ISLookupClientFactory;
+import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException;
+import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService;
+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.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.tools.DistCp;
+import org.apache.hadoop.tools.DistCpOptions;
+import org.apache.hadoop.util.ToolRunner;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.OutputStream;
+import java.util.*;
+import java.util.stream.Collectors;
+
+public class MigrateActionSet {
+
+ private static final Log log = LogFactory.getLog(MigrateActionSet.class);
+
+ private static final String SEPARATOR = "/";
+ private static final String TARGET_PATHS = "target_paths";
+ private static final String RAWSET_PREFIX = "rawset_";
+
+ private static Boolean DEFAULT_TRANSFORM_ONLY = false;
+
+ public static void main(String[] args) throws Exception {
+ final ArgumentApplicationParser parser = new ArgumentApplicationParser(
+ IOUtils.toString(MigrateActionSet.class.getResourceAsStream(
+ "/eu/dnetlib/dhp/migration/migrate_actionsets_parameters.json")));
+ parser.parseArgument(args);
+
+ new MigrateActionSet().run(parser);
+ }
+
+ private void run(ArgumentApplicationParser parser) throws Exception {
+
+ final String isLookupUrl = parser.get("isLookupUrl");
+ final String sourceNN = parser.get("sourceNameNode");
+ final String targetNN = parser.get("targetNameNode");
+ final String workDir = parser.get("workingDirectory");
+ final Integer distcp_num_maps = Integer.parseInt(parser.get("distcp_num_maps"));
+
+ final String distcp_memory_mb = parser.get("distcp_memory_mb");
+ final String distcp_task_timeout = parser.get("distcp_task_timeout");
+
+ final String transform_only_s = parser.get("transform_only");
+
+ log.info("transform only param: " + transform_only_s);
+
+ final Boolean transformOnly = Boolean.valueOf(parser.get("transform_only"));
+
+ log.info("transform only: " + transformOnly);
+
+ ISLookUpService isLookUp = ISLookupClientFactory.getLookUpService(isLookupUrl);
+
+ Configuration conf = getConfiguration(distcp_task_timeout, distcp_memory_mb, distcp_num_maps);
+ FileSystem targetFS = FileSystem.get(conf);
+
+ Configuration sourceConf = getConfiguration(distcp_task_timeout, distcp_memory_mb, distcp_num_maps);
+ sourceConf.set(FileSystem.FS_DEFAULT_NAME_KEY, sourceNN);
+ FileSystem sourceFS = FileSystem.get(sourceConf);
+
+ Properties props = new Properties();
+
+ List targetPaths = new ArrayList<>();
+
+ final List sourcePaths = getSourcePaths(sourceNN, isLookUp);
+ log.info(String.format("paths to process:\n%s", sourcePaths.stream().map(p -> p.toString()).collect(Collectors.joining("\n"))));
+ for(Path source : sourcePaths) {
+
+ if (!sourceFS.exists(source)) {
+ log.warn(String.format("skipping unexisting path: %s", source));
+ } else {
+
+ LinkedList pathQ = Lists.newLinkedList(Splitter.on(SEPARATOR).split(source.toUri().getPath()));
+
+ final String rawSet = pathQ.pollLast();
+ log.info(String.format("got RAWSET: %s", rawSet));
+
+ if (StringUtils.isNotBlank(rawSet) && rawSet.startsWith(RAWSET_PREFIX)) {
+
+ final String actionSetDirectory = pathQ.pollLast();
+
+ final Path targetPath = new Path(targetNN + workDir + SEPARATOR + actionSetDirectory + SEPARATOR + rawSet);
+
+ log.info(String.format("using TARGET PATH: %s", targetPath));
+
+ if (!transformOnly) {
+ if (targetFS.exists(targetPath)) {
+ targetFS.delete(targetPath, true);
+ }
+ runDistcp(distcp_num_maps, distcp_memory_mb, distcp_task_timeout, conf, source, targetPath);
+ }
+
+ targetPaths.add(targetPath);
+ }
+ }
+ }
+
+ props.setProperty(TARGET_PATHS, targetPaths
+ .stream()
+ .map(p -> p.toString())
+ .collect(Collectors.joining(",")));
+ File file = new File(System.getProperty("oozie.action.output.properties"));
+
+ try(OutputStream os = new FileOutputStream(file)) {
+ props.store(os, "");
+ }
+ System.out.println(file.getAbsolutePath());
+ }
+
+ private void runDistcp(Integer distcp_num_maps, String distcp_memory_mb, String distcp_task_timeout, Configuration conf, Path source, Path targetPath) throws Exception {
+
+ final DistCpOptions op = new DistCpOptions(source, targetPath);
+ op.setMaxMaps(distcp_num_maps);
+ op.preserve(DistCpOptions.FileAttribute.BLOCKSIZE);
+ op.preserve(DistCpOptions.FileAttribute.REPLICATION);
+ op.preserve(DistCpOptions.FileAttribute.CHECKSUMTYPE);
+
+ int res = ToolRunner.run(new DistCp(conf, op), new String[]{
+ "-Dmapred.task.timeout=" + distcp_task_timeout,
+ "-Dmapreduce.map.memory.mb=" + distcp_memory_mb,
+ "-pb",
+ "-m " + distcp_num_maps,
+ source.toString(),
+ targetPath.toString()});
+
+ if (res != 0) {
+ throw new RuntimeException(String.format("distcp exited with code %s", res));
+ }
+ }
+
+ private Configuration getConfiguration(String distcp_task_timeout, String distcp_memory_mb, Integer distcp_num_maps) {
+ final Configuration conf = new Configuration();
+ conf.set("dfs.webhdfs.socket.connect-timeout", distcp_task_timeout);
+ conf.set("dfs.webhdfs.socket.read-timeout", distcp_task_timeout);
+ conf.set("dfs.http.client.retry.policy.enabled", "true");
+ conf.set("mapred.task.timeout", distcp_task_timeout);
+ conf.set("mapreduce.map.memory.mb", distcp_memory_mb);
+ conf.set("mapred.map.tasks", String.valueOf(distcp_num_maps));
+ return conf;
+ }
+
+ private List getSourcePaths(String sourceNN, ISLookUpService isLookUp) throws ISLookUpException {
+ String XQUERY = "distinct-values(\n" +
+ "let $basePath := collection('/db/DRIVER/ServiceResources/ActionManagerServiceResourceType')//SERVICE_PROPERTIES/PROPERTY[@key = 'basePath']/@value/string()\n" +
+ "for $x in collection('/db/DRIVER/ActionManagerSetDSResources/ActionManagerSetDSResourceType') \n" +
+ "let $setDir := $x//SET/@directory/string()\n" +
+ "let $rawSet := $x//RAW_SETS/LATEST/@id/string()\n" +
+ "return concat($basePath, '/', $setDir, '/', $rawSet))";
+
+ log.info(String.format("running xquery:\n%s", XQUERY));
+ return isLookUp.quickSearchProfile(XQUERY)
+ .stream()
+ .map(p -> sourceNN + p)
+ .map(Path::new)
+ .collect(Collectors.toList());
+ }
+
+}
diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/actions/ProtoConverter.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/actions/ProtoConverter.java
new file mode 100644
index 0000000000..a7e70ee813
--- /dev/null
+++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/actions/ProtoConverter.java
@@ -0,0 +1,580 @@
+package eu.dnetlib.dhp.migration.actions;
+
+import com.google.common.collect.Lists;
+import com.googlecode.protobuf.format.JsonFormat;
+import eu.dnetlib.data.proto.*;
+import eu.dnetlib.dhp.schema.oaf.*;
+import org.apache.commons.lang3.StringUtils;
+
+import java.io.Serializable;
+import java.util.List;
+import java.util.Optional;
+import java.util.stream.Collectors;
+
+public class ProtoConverter implements Serializable {
+
+ public static final String UNKNOWN = "UNKNOWN";
+ public static final String NOT_AVAILABLE = "not available";
+ public static final String DNET_ACCESS_MODES = "dnet:access_modes";
+
+ public static Oaf convert(OafProtos.Oaf oaf) {
+ try {
+ switch (oaf.getKind()) {
+ case entity:
+ return convertEntity(oaf);
+ case relation:
+ return convertRelation(oaf);
+ default:
+ throw new IllegalArgumentException("invalid kind " + oaf.getKind());
+ }
+ } catch (Throwable e) {
+ throw new RuntimeException("error on getting " + JsonFormat.printToString(oaf), e);
+ }
+ }
+
+ private static Relation convertRelation(OafProtos.Oaf oaf) {
+ final OafProtos.OafRel r = oaf.getRel();
+ final Relation rel = new Relation();
+ rel.setDataInfo(mapDataInfo(oaf.getDataInfo()));
+ rel.setLastupdatetimestamp(oaf.getLastupdatetimestamp());
+ rel.setSource(r.getSource());
+ rel.setTarget(r.getTarget());
+ rel.setRelType(r.getRelType().toString());
+ rel.setSubRelType(r.getSubRelType().toString());
+ rel.setRelClass(r.getRelClass());
+ rel.setCollectedFrom(r.getCollectedfromCount() > 0 ?
+ r.getCollectedfromList().stream()
+ .map(kv -> mapKV(kv))
+ .collect(Collectors.toList()) : null);
+ return rel;
+ }
+
+ private static OafEntity convertEntity(OafProtos.Oaf oaf) {
+
+ switch (oaf.getEntity().getType()) {
+ case result:
+ final Result r = convertResult(oaf);
+ r.setInstance(convertInstances(oaf));
+ return r;
+ case project:
+ return convertProject(oaf);
+ case datasource:
+ return convertDataSource(oaf);
+ case organization:
+ return convertOrganization(oaf);
+ default:
+ throw new RuntimeException("received unknown type");
+ }
+ }
+
+ private static List convertInstances(OafProtos.Oaf oaf) {
+
+ final ResultProtos.Result r = oaf.getEntity().getResult();
+ if (r.getInstanceCount() > 0) {
+ return r.getInstanceList()
+ .stream()
+ .map(i -> convertInstance(i))
+ .collect(Collectors.toList());
+ }
+ return Lists.newArrayList();
+ }
+
+ private static Instance convertInstance(ResultProtos.Result.Instance ri) {
+ final Instance i = new Instance();
+ i.setAccessright(mapQualifier(ri.getAccessright()));
+ i.setCollectedfrom(mapKV(ri.getCollectedfrom()));
+ i.setDateofacceptance(mapStringField(ri.getDateofacceptance()));
+ i.setDistributionlocation(ri.getDistributionlocation());
+ i.setHostedby(mapKV(ri.getHostedby()));
+ i.setInstancetype(mapQualifier(ri.getInstancetype()));
+ i.setLicense(mapStringField(ri.getLicense()));
+ i.setUrl(ri.getUrlList());
+ i.setRefereed(mapStringField(ri.getRefereed()));
+ i.setProcessingchargeamount(mapStringField(ri.getProcessingchargeamount()));
+ i.setProcessingchargecurrency(mapStringField(ri.getProcessingchargecurrency()));
+ return i;
+ }
+
+ private static Organization convertOrganization(OafProtos.Oaf oaf) {
+ final OrganizationProtos.Organization.Metadata m = oaf.getEntity().getOrganization().getMetadata();
+ final Organization org = setOaf(new Organization(), oaf);
+ setEntity(org, oaf);
+ org.setLegalshortname(mapStringField(m.getLegalshortname()));
+ org.setLegalname(mapStringField(m.getLegalname()));
+ org.setAlternativeNames(m.getAlternativeNamesList().
+ stream()
+ .map(ProtoConverter::mapStringField)
+ .collect(Collectors.toList()));
+ org.setWebsiteurl(mapStringField(m.getWebsiteurl()));
+ org.setLogourl(mapStringField(m.getLogourl()));
+ org.setEclegalbody(mapStringField(m.getEclegalbody()));
+ org.setEclegalperson(mapStringField(m.getEclegalperson()));
+ org.setEcnonprofit(mapStringField(m.getEcnonprofit()));
+ org.setEcresearchorganization(mapStringField(m.getEcresearchorganization()));
+ org.setEchighereducation(mapStringField(m.getEchighereducation()));
+ org.setEcinternationalorganizationeurinterests(mapStringField(m.getEcinternationalorganizationeurinterests()));
+ org.setEcinternationalorganization(mapStringField(m.getEcinternationalorganization()));
+ org.setEcenterprise(mapStringField(m.getEcenterprise()));
+ org.setEcsmevalidated(mapStringField(m.getEcsmevalidated()));
+ org.setEcnutscode(mapStringField(m.getEcnutscode()));
+ org.setCountry(mapQualifier(m.getCountry()));
+
+ return org;
+ }
+
+ private static Datasource convertDataSource(OafProtos.Oaf oaf) {
+ final DatasourceProtos.Datasource.Metadata m = oaf.getEntity().getDatasource().getMetadata();
+ final Datasource datasource = setOaf(new Datasource(), oaf);
+ setEntity(datasource, oaf);
+ datasource.setAccessinfopackage(m.getAccessinfopackageList()
+ .stream()
+ .map(ProtoConverter::mapStringField)
+ .collect(Collectors.toList()));
+ datasource.setCertificates(mapStringField(m.getCertificates()));
+ datasource.setCitationguidelineurl(mapStringField(m.getCitationguidelineurl()));
+ datasource.setContactemail(mapStringField(m.getContactemail()));
+ datasource.setDatabaseaccessrestriction(mapStringField(m.getDatabaseaccessrestriction()));
+ datasource.setDatabaseaccesstype(mapStringField(m.getDatabaseaccesstype()));
+ datasource.setDataprovider(mapBoolField(m.getDataprovider()));
+ datasource.setDatasourcetype(mapQualifier(m.getDatasourcetype()));
+ datasource.setDatauploadrestriction(mapStringField(m.getDatauploadrestriction()));
+ datasource.setCitationguidelineurl(mapStringField(m.getCitationguidelineurl()));
+ datasource.setDatauploadtype(mapStringField(m.getDatauploadtype()));
+ datasource.setDateofvalidation(mapStringField(m.getDateofvalidation()));
+ datasource.setDescription(mapStringField(m.getDescription()));
+ datasource.setEnglishname(mapStringField(m.getEnglishname()));
+ datasource.setLatitude(mapStringField(m.getLatitude()));
+ datasource.setLongitude(mapStringField(m.getLongitude()));
+ datasource.setLogourl(mapStringField(m.getLogourl()));
+ datasource.setMissionstatementurl(mapStringField(m.getMissionstatementurl()));
+ datasource.setNamespaceprefix(mapStringField(m.getNamespaceprefix()));
+ datasource.setOdcontenttypes(m.getOdcontenttypesList()
+ .stream()
+ .map(ProtoConverter::mapStringField)
+ .collect(Collectors.toList()));
+ datasource.setOdlanguages(m.getOdlanguagesList()
+ .stream()
+ .map(ProtoConverter::mapStringField)
+ .collect(Collectors.toList()));
+ datasource.setOdnumberofitems(mapStringField(m.getOdnumberofitems()));
+ datasource.setOdnumberofitemsdate(mapStringField(m.getOdnumberofitemsdate()));
+ datasource.setOdpolicies(mapStringField(m.getOdpolicies()));
+ datasource.setOfficialname(mapStringField(m.getOfficialname()));
+ datasource.setOpenairecompatibility(mapQualifier(m.getOpenairecompatibility()));
+ datasource.setPidsystems(mapStringField(m.getPidsystems()));
+ datasource.setPolicies(m.getPoliciesList()
+ .stream()
+ .map(ProtoConverter::mapKV)
+ .collect(Collectors.toList()));
+ datasource.setQualitymanagementkind(mapStringField(m.getQualitymanagementkind()));
+ datasource.setReleaseenddate(mapStringField(m.getReleaseenddate()));
+ datasource.setServiceprovider(mapBoolField(m.getServiceprovider()));
+ datasource.setReleasestartdate(mapStringField(m.getReleasestartdate()));
+ datasource.setSubjects(m.getSubjectsList()
+ .stream()
+ .map(ProtoConverter::mapStructuredProperty)
+ .collect(Collectors.toList()));
+ datasource.setVersioning(mapBoolField(m.getVersioning()));
+ datasource.setWebsiteurl(mapStringField(m.getWebsiteurl()));
+ datasource.setJournal(mapJournal(m.getJournal()));
+
+
+ return datasource;
+ }
+
+ private static Project convertProject(OafProtos.Oaf oaf) {
+ final ProjectProtos.Project.Metadata m = oaf.getEntity().getProject().getMetadata();
+ final Project project = setOaf(new Project(), oaf);
+ setEntity(project, oaf);
+ project.setAcronym(mapStringField(m.getAcronym()));
+ project.setCallidentifier(mapStringField(m.getCallidentifier()));
+ project.setCode(mapStringField(m.getCode()));
+ project.setContactemail(mapStringField(m.getContactemail()));
+ project.setContactfax(mapStringField(m.getContactfax()));
+ project.setContactfullname(mapStringField(m.getContactfullname()));
+ project.setContactphone(mapStringField(m.getContactphone()));
+ project.setContracttype(mapQualifier(m.getContracttype()));
+ project.setCurrency(mapStringField(m.getCurrency()));
+ project.setDuration(mapStringField(m.getDuration()));
+ project.setEcarticle29_3(mapStringField(m.getEcarticle293()));
+ project.setEcsc39(mapStringField(m.getEcsc39()));
+ project.setOamandatepublications(mapStringField(m.getOamandatepublications()));
+ project.setStartdate(mapStringField(m.getStartdate()));
+ project.setEnddate(mapStringField(m.getEnddate()));
+ project.setFundedamount(m.getFundedamount());
+ project.setTotalcost(m.getTotalcost());
+ project.setKeywords(mapStringField(m.getKeywords()));
+ project.setSubjects(m.getSubjectsList().stream()
+ .map(sp -> mapStructuredProperty(sp))
+ .collect(Collectors.toList()));
+ project.setTitle(mapStringField(m.getTitle()));
+ project.setWebsiteurl(mapStringField(m.getWebsiteurl()));
+ project.setFundingtree(m.getFundingtreeList().stream()
+ .map(f -> mapStringField(f))
+ .collect(Collectors.toList()));
+ project.setJsonextrainfo(mapStringField(m.getJsonextrainfo()));
+ project.setSummary(mapStringField(m.getSummary()));
+ project.setOptional1(mapStringField(m.getOptional1()));
+ project.setOptional2(mapStringField(m.getOptional2()));
+ return project;
+ }
+
+ private static Result convertResult(OafProtos.Oaf oaf) {
+ switch (oaf.getEntity().getResult().getMetadata().getResulttype().getClassid()) {
+ case "dataset":
+ return createDataset(oaf);
+ case "publication":
+ return createPublication(oaf);
+ case "software":
+ return createSoftware(oaf);
+ case "other":
+ return createORP(oaf);
+ default:
+ Result result = setOaf(new Result(), oaf);
+ setEntity(result, oaf);
+ return setResult(result, oaf);
+ }
+ }
+
+ private static Software createSoftware(OafProtos.Oaf oaf) {
+ ResultProtos.Result.Metadata m = oaf.getEntity().getResult().getMetadata();
+ Software software = setOaf(new Software(), oaf);
+ setEntity(software, oaf);
+ setResult(software, oaf);
+
+ software.setDocumentationUrl(m.getDocumentationUrlList()
+ .stream()
+ .map(ProtoConverter::mapStringField)
+ .collect(Collectors.toList()));
+ software.setLicense(m.getLicenseList()
+ .stream()
+ .map(ProtoConverter::mapStructuredProperty)
+ .collect(Collectors.toList()));
+ software.setCodeRepositoryUrl(mapStringField(m.getCodeRepositoryUrl()));
+ software.setProgrammingLanguage(mapQualifier(m.getProgrammingLanguage()));
+ return software;
+ }
+
+ private static OtherResearchProduct createORP(OafProtos.Oaf oaf) {
+ ResultProtos.Result.Metadata m = oaf.getEntity().getResult().getMetadata();
+ OtherResearchProduct otherResearchProducts = setOaf(new OtherResearchProduct(), oaf);
+ setEntity(otherResearchProducts, oaf);
+ setResult(otherResearchProducts, oaf);
+ otherResearchProducts.setContactperson(m.getContactpersonList()
+ .stream()
+ .map(ProtoConverter::mapStringField)
+ .collect(Collectors.toList()));
+ otherResearchProducts.setContactgroup(m.getContactgroupList()
+ .stream()
+ .map(ProtoConverter::mapStringField)
+ .collect(Collectors.toList()));
+ otherResearchProducts.setTool(m.getToolList()
+ .stream()
+ .map(ProtoConverter::mapStringField)
+ .collect(Collectors.toList()));
+
+ return otherResearchProducts;
+ }
+
+ private static Publication createPublication(OafProtos.Oaf oaf) {
+
+ ResultProtos.Result.Metadata m = oaf.getEntity().getResult().getMetadata();
+ Publication publication = setOaf(new Publication(), oaf);
+ setEntity(publication, oaf);
+ setResult(publication, oaf);
+ publication.setJournal(mapJournal(m.getJournal()));
+ return publication;
+ }
+
+ private static Dataset createDataset(OafProtos.Oaf oaf) {
+
+ ResultProtos.Result.Metadata m = oaf.getEntity().getResult().getMetadata();
+ Dataset dataset = setOaf(new Dataset(), oaf);
+ setEntity(dataset, oaf);
+ setResult(dataset, oaf);
+ dataset.setStoragedate(mapStringField(m.getStoragedate()));
+ dataset.setDevice(mapStringField(m.getDevice()));
+ dataset.setSize(mapStringField(m.getSize()));
+ dataset.setVersion(mapStringField(m.getVersion()));
+ dataset.setLastmetadataupdate(mapStringField(m.getLastmetadataupdate()));
+ dataset.setMetadataversionnumber(mapStringField(m.getMetadataversionnumber()));
+ dataset.setGeolocation(m.getGeolocationList()
+ .stream()
+ .map(ProtoConverter::mapGeolocation)
+ .collect(Collectors.toList()));
+ return dataset;
+
+ }
+
+ public static T setOaf(T oaf, OafProtos.Oaf o) {
+ oaf.setDataInfo(mapDataInfo(o.getDataInfo()));
+ oaf.setLastupdatetimestamp(o.getLastupdatetimestamp());
+ return oaf;
+ }
+
+ public static T setEntity(T entity, OafProtos.Oaf oaf) {
+ //setting Entity fields
+ final OafProtos.OafEntity e = oaf.getEntity();
+ entity.setId(e.getId());
+ entity.setOriginalId(e.getOriginalIdList());
+ entity.setCollectedfrom(e.getCollectedfromList()
+ .stream()
+ .map(ProtoConverter::mapKV)
+ .collect(Collectors.toList()));
+ entity.setPid(e.getPidList().stream()
+ .map(ProtoConverter::mapStructuredProperty)
+ .collect(Collectors.toList()));
+ entity.setDateofcollection(e.getDateofcollection());
+ entity.setDateoftransformation(e.getDateoftransformation());
+ entity.setExtraInfo(e.getExtraInfoList()
+ .stream()
+ .map(ProtoConverter::mapExtraInfo)
+ .collect(Collectors.toList()));
+ return entity;
+ }
+
+ public static T setResult(T entity, OafProtos.Oaf oaf) {
+ //setting Entity fields
+ final ResultProtos.Result.Metadata m = oaf.getEntity().getResult().getMetadata();
+ entity.setAuthor(m.getAuthorList()
+ .stream()
+ .map(ProtoConverter::mapAuthor)
+ .collect(Collectors.toList()));
+ entity.setResulttype(mapQualifier(m.getResulttype()));
+ entity.setLanguage(mapQualifier(m.getLanguage()));
+ entity.setCountry(m.getCountryList()
+ .stream()
+ .map(ProtoConverter::mapQualifierAsCountry)
+ .collect(Collectors.toList()));
+ entity.setSubject(m.getSubjectList()
+ .stream()
+ .map(ProtoConverter::mapStructuredProperty)
+ .collect(Collectors.toList()));
+ entity.setTitle(m.getTitleList()
+ .stream()
+ .map(ProtoConverter::mapStructuredProperty)
+ .collect(Collectors.toList()));
+ entity.setRelevantdate(m.getRelevantdateList()
+ .stream()
+ .map(ProtoConverter::mapStructuredProperty)
+ .collect(Collectors.toList()));
+ entity.setDescription(m.getDescriptionList()
+ .stream()
+ .map(ProtoConverter::mapStringField)
+ .collect(Collectors.toList()));
+ entity.setDateofacceptance(mapStringField(m.getDateofacceptance()));
+ entity.setPublisher(mapStringField(m.getPublisher()));
+ entity.setEmbargoenddate(mapStringField(m.getEmbargoenddate()));
+ entity.setSource(m.getSourceList()
+ .stream()
+ .map(ProtoConverter::mapStringField)
+ .collect(Collectors.toList()));
+ entity.setFulltext(m.getFulltextList()
+ .stream()
+ .map(ProtoConverter::mapStringField)
+ .collect(Collectors.toList()));
+ entity.setFormat(m.getFormatList()
+ .stream()
+ .map(ProtoConverter::mapStringField)
+ .collect(Collectors.toList()));
+ entity.setContributor(m.getContributorList()
+ .stream()
+ .map(ProtoConverter::mapStringField)
+ .collect(Collectors.toList()));
+ entity.setResourcetype(mapQualifier(m.getResourcetype()));
+ entity.setCoverage(m.getCoverageList()
+ .stream()
+ .map(ProtoConverter::mapStringField)
+ .collect(Collectors.toList()));
+ entity.setContext(m.getContextList()
+ .stream()
+ .map(ProtoConverter::mapContext)
+ .collect(Collectors.toList()));
+
+ entity.setBestaccessright(getBestAccessRights(oaf.getEntity().getResult().getInstanceList()));
+
+ return entity;
+ }
+
+ private static Qualifier getBestAccessRights(List instanceList) {
+ if (instanceList != null) {
+ final Optional min = instanceList.stream()
+ .map(i -> i.getAccessright()).min(new LicenseComparator());
+
+ final Qualifier rights = min.isPresent() ? mapQualifier(min.get()) : new Qualifier();
+
+ if (StringUtils.isBlank(rights.getClassid())) {
+ rights.setClassid(UNKNOWN);
+ }
+ if (StringUtils.isBlank(rights.getClassname()) || UNKNOWN.equalsIgnoreCase(rights.getClassname())) {
+ rights.setClassname(NOT_AVAILABLE);
+ }
+ if (StringUtils.isBlank(rights.getSchemeid())) {
+ rights.setSchemeid(DNET_ACCESS_MODES);
+ }
+ if (StringUtils.isBlank(rights.getSchemename())) {
+ rights.setSchemename(DNET_ACCESS_MODES);
+ }
+
+ return rights;
+ }
+ return null;
+ }
+
+ private static Context mapContext(ResultProtos.Result.Context context) {
+
+ final Context entity = new Context();
+ entity.setId(context.getId());
+ entity.setDataInfo(context.getDataInfoList()
+ .stream()
+ .map(ProtoConverter::mapDataInfo)
+ .collect(Collectors.toList()));
+ return entity;
+ }
+
+
+ public static KeyValue mapKV(FieldTypeProtos.KeyValue kv) {
+ final KeyValue keyValue = new KeyValue();
+ keyValue.setKey(kv.getKey());
+ keyValue.setValue(kv.getValue());
+ keyValue.setDataInfo(mapDataInfo(kv.getDataInfo()));
+ return keyValue;
+ }
+
+ public static DataInfo mapDataInfo(FieldTypeProtos.DataInfo d) {
+ final DataInfo dataInfo = new DataInfo();
+ dataInfo.setDeletedbyinference(d.getDeletedbyinference());
+ dataInfo.setInferenceprovenance(d.getInferenceprovenance());
+ dataInfo.setInferred(d.getInferred());
+ dataInfo.setInvisible(d.getInvisible());
+ dataInfo.setProvenanceaction(mapQualifier(d.getProvenanceaction()));
+ dataInfo.setTrust(d.getTrust());
+ return dataInfo;
+ }
+
+ public static Qualifier mapQualifier(FieldTypeProtos.Qualifier q) {
+ final Qualifier qualifier = new Qualifier();
+ qualifier.setClassid(q.getClassid());
+ qualifier.setClassname(q.getClassname());
+ qualifier.setSchemeid(q.getSchemeid());
+ qualifier.setSchemename(q.getSchemename());
+ return qualifier;
+ }
+
+ public static Country mapQualifierAsCountry(FieldTypeProtos.Qualifier q) {
+ final Country c = new Country();
+ c.setClassid(q.getClassid());
+ c.setClassname(q.getClassname());
+ c.setSchemeid(q.getSchemeid());
+ c.setSchemename(q.getSchemename());
+ c.setDataInfo(mapDataInfo(q.getDataInfo()));
+ return c;
+ }
+
+ public static StructuredProperty mapStructuredProperty(FieldTypeProtos.StructuredProperty sp) {
+ final StructuredProperty structuredProperty = new StructuredProperty();
+ structuredProperty.setValue(sp.getValue());
+ structuredProperty.setQualifier(mapQualifier(sp.getQualifier()));
+ structuredProperty.setDataInfo(mapDataInfo(sp.getDataInfo()));
+ return structuredProperty;
+ }
+
+ public static ExtraInfo mapExtraInfo(FieldTypeProtos.ExtraInfo extraInfo) {
+ final ExtraInfo entity = new ExtraInfo();
+ entity.setName(extraInfo.getName());
+ entity.setTypology(extraInfo.getTypology());
+ entity.setProvenance(extraInfo.getProvenance());
+ entity.setTrust(extraInfo.getTrust());
+ entity.setValue(extraInfo.getValue());
+ return entity;
+ }
+
+ public static OAIProvenance mapOAIProvenance(FieldTypeProtos.OAIProvenance oaiProvenance) {
+ final OAIProvenance entity = new OAIProvenance();
+ entity.setOriginDescription(mapOriginalDescription(oaiProvenance.getOriginDescription()));
+ return entity;
+ }
+
+ public static OriginDescription mapOriginalDescription(FieldTypeProtos.OAIProvenance.OriginDescription originDescription) {
+ final OriginDescription originDescriptionResult = new OriginDescription();
+ originDescriptionResult.setHarvestDate(originDescription.getHarvestDate());
+ originDescriptionResult.setAltered(originDescription.getAltered());
+ originDescriptionResult.setBaseURL(originDescription.getBaseURL());
+ originDescriptionResult.setIdentifier(originDescription.getIdentifier());
+ originDescriptionResult.setDatestamp(originDescription.getDatestamp());
+ originDescriptionResult.setMetadataNamespace(originDescription.getMetadataNamespace());
+ return originDescriptionResult;
+ }
+
+ public static Field mapStringField(FieldTypeProtos.StringField s) {
+ final Field stringField = new Field<>();
+ stringField.setValue(s.getValue());
+ stringField.setDataInfo(mapDataInfo(s.getDataInfo()));
+ return stringField;
+ }
+
+ public static Field mapBoolField(FieldTypeProtos.BoolField b) {
+ final Field booleanField = new Field<>();
+ booleanField.setValue(b.getValue());
+ booleanField.setDataInfo(mapDataInfo(b.getDataInfo()));
+ return booleanField;
+ }
+
+ public static Field mapIntField(FieldTypeProtos.IntField b) {
+ final Field entity = new Field<>();
+ entity.setValue(b.getValue());
+ entity.setDataInfo(mapDataInfo(b.getDataInfo()));
+ return entity;
+ }
+
+ public static Journal mapJournal(FieldTypeProtos.Journal j) {
+ final Journal journal = new Journal();
+ journal.setConferencedate(j.getConferencedate());
+ journal.setConferenceplace(j.getConferenceplace());
+ journal.setEdition(j.getEdition());
+ journal.setEp(j.getEp());
+ journal.setIss(j.getIss());
+ journal.setIssnLinking(j.getIssnLinking());
+ journal.setIssnOnline(j.getIssnOnline());
+ journal.setIssnPrinted(j.getIssnPrinted());
+ journal.setName(j.getName());
+ journal.setSp(j.getSp());
+ journal.setVol(j.getVol());
+ journal.setDataInfo(mapDataInfo(j.getDataInfo()));
+ return journal;
+ }
+
+ public static Author mapAuthor(FieldTypeProtos.Author author) {
+ final Author entity = new Author();
+ entity.setFullname(author.getFullname());
+ entity.setName(author.getName());
+ entity.setSurname(author.getSurname());
+ entity.setRank(author.getRank());
+ entity.setPid(author.getPidList()
+ .stream()
+ .map(kv -> {
+ final StructuredProperty sp = new StructuredProperty();
+ sp.setValue(kv.getValue());
+ final Qualifier q = new Qualifier();
+ q.setClassid(kv.getKey());
+ q.setClassname(kv.getKey());
+ sp.setQualifier(q);
+ return sp;
+ })
+ .collect(Collectors.toList()));
+ entity.setAffiliation(author.getAffiliationList()
+ .stream()
+ .map(ProtoConverter::mapStringField)
+ .collect(Collectors.toList()));
+ return entity;
+
+ }
+
+ public static GeoLocation mapGeolocation(ResultProtos.Result.GeoLocation geoLocation) {
+ final GeoLocation entity = new GeoLocation();
+ entity.setPoint(geoLocation.getPoint());
+ entity.setBox(geoLocation.getBox());
+ entity.setPlace(geoLocation.getPlace());
+ return entity;
+ }
+}
diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/actions/TransformActions.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/actions/TransformActions.java
new file mode 100644
index 0000000000..19a0cb5c9d
--- /dev/null
+++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/actions/TransformActions.java
@@ -0,0 +1,194 @@
+package eu.dnetlib.dhp.migration.actions;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.base.Splitter;
+import com.google.common.collect.Lists;
+import com.google.protobuf.InvalidProtocolBufferException;
+import eu.dnetlib.dhp.schema.action.AtomicAction;
+import eu.dnetlib.data.proto.OafProtos;
+import eu.dnetlib.dhp.application.ArgumentApplicationParser;
+import eu.dnetlib.dhp.schema.oaf.*;
+import eu.dnetlib.dhp.utils.ISLookupClientFactory;
+import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException;
+import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService;
+
+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.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.compress.GzipCodec;
+import org.apache.hadoop.mapred.SequenceFileOutputFormat;
+import org.apache.spark.SparkConf;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.sql.SparkSession;
+import scala.Tuple2;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.LinkedList;
+
+public class TransformActions implements Serializable {
+
+ private static final Log log = LogFactory.getLog(TransformActions.class);
+ private static final String SEPARATOR = "/";
+
+ public static void main(String[] args) throws Exception {
+ final ArgumentApplicationParser parser = new ArgumentApplicationParser(
+ IOUtils.toString(MigrateActionSet.class.getResourceAsStream(
+ "/eu/dnetlib/dhp/migration/transform_actionsets_parameters.json")));
+ parser.parseArgument(args);
+
+ new TransformActions().run(parser);
+ }
+
+ private void run(ArgumentApplicationParser parser) throws ISLookUpException, IOException {
+
+ final String isLookupUrl = parser.get("isLookupUrl");
+ log.info("isLookupUrl: " + isLookupUrl);
+
+ final String inputPaths = parser.get("inputPaths");
+
+ if (StringUtils.isBlank(inputPaths)) {
+ throw new RuntimeException("empty inputPaths");
+ }
+ log.info("inputPaths: " + inputPaths);
+
+ final String targetBaseDir = getTargetBaseDir(isLookupUrl);
+
+ try(SparkSession spark = getSparkSession(parser)) {
+ final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext());
+ final FileSystem fs = FileSystem.get(spark.sparkContext().hadoopConfiguration());
+
+ for(String sourcePath : Lists.newArrayList(Splitter.on(",").split(inputPaths))) {
+
+ LinkedList pathQ = Lists.newLinkedList(Splitter.on(SEPARATOR).split(sourcePath));
+
+ final String rawset = pathQ.pollLast();
+ final String actionSetDirectory = pathQ.pollLast();
+
+ final Path targetDirectory = new Path(targetBaseDir + SEPARATOR + actionSetDirectory + SEPARATOR + rawset);
+
+ if (fs.exists(targetDirectory)) {
+ log.info(String.format("found target directory '%s", targetDirectory));
+ fs.delete(targetDirectory, true);
+ log.info(String.format("deleted target directory '%s", targetDirectory));
+ }
+
+ log.info(String.format("transforming actions from '%s' to '%s'", sourcePath, targetDirectory));
+
+ sc.sequenceFile(sourcePath, Text.class, Text.class)
+ .mapToPair(a -> new Tuple2<>(a._1(), eu.dnetlib.actionmanager.actions.AtomicAction.fromJSON(a._2().toString())))
+ .mapToPair(a -> new Tuple2<>(a._1(), transformAction(a._1().toString(), a._2())))
+ .filter(t -> StringUtils.isNotBlank(t._2().toString()))
+ .saveAsHadoopFile(targetDirectory.toString(), Text.class, Text.class, SequenceFileOutputFormat.class, GzipCodec.class);
+ }
+ }
+ }
+
+ private Text transformAction(String atomicaActionId, eu.dnetlib.actionmanager.actions.AtomicAction aa) throws InvalidProtocolBufferException, JsonProcessingException {
+ final Text out = new Text();
+ final ObjectMapper mapper = new ObjectMapper();
+ if (aa.getTargetValue() != null && aa.getTargetValue().length > 0) {
+ out.set(mapper.writeValueAsString(doTransform(aa)));
+ } else {
+ if (atomicaActionId.contains("dedupSimilarity")) {
+ out.set(mapper.writeValueAsString(getRelationAtomicAction(atomicaActionId)));
+ }
+ }
+
+ return out;
+ }
+
+ private AtomicAction getRelationAtomicAction(String atomicaActionId) {
+ final String[] splitId = atomicaActionId.split("@");
+
+ String source = splitId[0];
+ String target = splitId[2];
+
+ String[] relSemantic = splitId[1].split("_");
+
+ Relation rel = new Relation();
+ rel.setSource(source);
+ rel.setTarget(target);
+ rel.setRelType(relSemantic[0]);
+ rel.setSubRelType(relSemantic[1]);
+ rel.setRelClass(relSemantic[2]);
+
+ DataInfo d = new DataInfo();
+ d.setDeletedbyinference(false);
+ d.setInferenceprovenance("deduplication");
+ d.setInferred(true);
+ d.setInvisible(false);
+ Qualifier provenanceaction = new Qualifier();
+
+ provenanceaction.setClassid("deduplication");
+ provenanceaction.setClassname("deduplication");
+ provenanceaction.setSchemeid("dnet:provenanceActions");
+ provenanceaction.setSchemename("dnet:provenanceActions");
+
+ d.setProvenanceaction(provenanceaction);
+
+ rel.setDataInfo(d);
+
+ return new AtomicAction<>(Relation.class, rel);
+ }
+
+ private AtomicAction doTransform(eu.dnetlib.actionmanager.actions.AtomicAction aa) throws InvalidProtocolBufferException {
+ final OafProtos.Oaf proto_oaf = OafProtos.Oaf.parseFrom(aa.getTargetValue());
+ final Oaf oaf = ProtoConverter.convert(proto_oaf);
+ switch (proto_oaf.getKind()) {
+ case entity:
+ switch (proto_oaf.getEntity().getType()) {
+ case datasource:
+ return new AtomicAction<>(Datasource.class, (Datasource) oaf);
+ case organization:
+ return new AtomicAction<>(Organization.class, (Organization) oaf);
+ case project:
+ return new AtomicAction<>(Project.class, (Project) oaf);
+ case result:
+ final String resulttypeid = proto_oaf.getEntity().getResult().getMetadata().getResulttype().getClassid();
+ switch (resulttypeid) {
+ case "publication":
+ return new AtomicAction<>(Publication.class, (Publication) oaf);
+ case "software":
+ return new AtomicAction<>(Software.class, (Software) oaf);
+ case "other":
+ return new AtomicAction<>(OtherResearchProduct.class, (OtherResearchProduct) oaf);
+ case "dataset":
+ return new AtomicAction<>(Dataset.class, (Dataset) oaf);
+ default:
+ // can be an update, where the resulttype is not specified
+ return new AtomicAction<>(Result.class, (Result) oaf);
+ }
+ default:
+ throw new IllegalArgumentException("invalid entity type: " + proto_oaf.getEntity().getType());
+ }
+ case relation:
+ return new AtomicAction<>(Relation.class, (Relation) oaf);
+ default:
+ throw new IllegalArgumentException("invalid kind: " + proto_oaf.getKind());
+ }
+ }
+
+ private String getTargetBaseDir(String isLookupUrl) throws ISLookUpException {
+ ISLookUpService isLookUp = ISLookupClientFactory.getLookUpService(isLookupUrl);
+ String XQUERY = "collection('/db/DRIVER/ServiceResources/ActionManagerServiceResourceType')//SERVICE_PROPERTIES/PROPERTY[@key = 'basePath']/@value/string()";
+ return isLookUp.getResourceProfileByQuery(XQUERY);
+ }
+
+ private static SparkSession getSparkSession(ArgumentApplicationParser parser) {
+ SparkConf conf = new SparkConf();
+
+ return SparkSession
+ .builder()
+ .appName(TransformActions.class.getSimpleName())
+ .master(parser.get("master"))
+ .config(conf)
+ .enableHiveSupport()
+ .getOrCreate();
+ }
+}
diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/MigrateDbEntitiesApplication.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/step1/MigrateDbEntitiesApplication.java
similarity index 94%
rename from dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/MigrateDbEntitiesApplication.java
rename to dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/step1/MigrateDbEntitiesApplication.java
index 1ccfd09ef0..5e54c2b86d 100644
--- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/MigrateDbEntitiesApplication.java
+++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/step1/MigrateDbEntitiesApplication.java
@@ -1,4 +1,14 @@
-package eu.dnetlib.dhp.migration;
+package eu.dnetlib.dhp.migration.step1;
+
+import static eu.dnetlib.dhp.migration.utils.OafMapperUtils.asString;
+import static eu.dnetlib.dhp.migration.utils.OafMapperUtils.createOpenaireId;
+import static eu.dnetlib.dhp.migration.utils.OafMapperUtils.dataInfo;
+import static eu.dnetlib.dhp.migration.utils.OafMapperUtils.field;
+import static eu.dnetlib.dhp.migration.utils.OafMapperUtils.journal;
+import static eu.dnetlib.dhp.migration.utils.OafMapperUtils.listFields;
+import static eu.dnetlib.dhp.migration.utils.OafMapperUtils.listKeyValues;
+import static eu.dnetlib.dhp.migration.utils.OafMapperUtils.qualifier;
+import static eu.dnetlib.dhp.migration.utils.OafMapperUtils.structuredProperty;
import java.io.Closeable;
import java.io.IOException;
@@ -17,6 +27,8 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
+import eu.dnetlib.dhp.migration.utils.AbstractMigrationApplication;
+import eu.dnetlib.dhp.migration.utils.DbClient;
import eu.dnetlib.dhp.schema.oaf.Context;
import eu.dnetlib.dhp.schema.oaf.DataInfo;
import eu.dnetlib.dhp.schema.oaf.Dataset;
@@ -34,7 +46,7 @@ import eu.dnetlib.dhp.schema.oaf.Result;
import eu.dnetlib.dhp.schema.oaf.Software;
import eu.dnetlib.dhp.schema.oaf.StructuredProperty;
-public class MigrateDbEntitiesApplication extends AbstractMigrationExecutor implements Closeable {
+public class MigrateDbEntitiesApplication extends AbstractMigrationApplication implements Closeable {
private static final Qualifier ENTITYREGISTRY_PROVENANCE_ACTION =
qualifier("sysimport:crosswalk:entityregistry", "sysimport:crosswalk:entityregistry", "dnet:provenance_actions", "dnet:provenance_actions");
@@ -56,12 +68,10 @@ public class MigrateDbEntitiesApplication extends AbstractMigrationExecutor impl
final String dbPassword = parser.get("postgresPassword");
final String hdfsPath = parser.get("hdfsPath");
- final String hdfsNameNode = parser.get("namenode");
- final String hdfsUser = parser.get("hdfsUser");
final boolean processClaims = parser.get("action") != null && parser.get("action").equalsIgnoreCase("claims");
- try (final MigrateDbEntitiesApplication smdbe = new MigrateDbEntitiesApplication(hdfsPath, hdfsNameNode, hdfsUser, dbUrl, dbUser, dbPassword)) {
+ try (final MigrateDbEntitiesApplication smdbe = new MigrateDbEntitiesApplication(hdfsPath, dbUrl, dbUser, dbPassword)) {
if (processClaims) {
log.info("Processing claims...");
smdbe.execute("queryClaims.sql", smdbe::processClaims);
@@ -85,9 +95,9 @@ public class MigrateDbEntitiesApplication extends AbstractMigrationExecutor impl
}
}
- public MigrateDbEntitiesApplication(final String hdfsPath, final String hdfsNameNode, final String hdfsUser, final String dbUrl, final String dbUser,
+ public MigrateDbEntitiesApplication(final String hdfsPath, final String dbUrl, final String dbUser,
final String dbPassword) throws Exception {
- super(hdfsPath, hdfsNameNode, hdfsUser);
+ super(hdfsPath);
this.dbClient = new DbClient(dbUrl, dbUser, dbPassword);
this.lastUpdateTimestamp = new Date().getTime();
}
@@ -105,7 +115,7 @@ public class MigrateDbEntitiesApplication extends AbstractMigrationExecutor impl
final Datasource ds = new Datasource();
- ds.setId(createOpenaireId(10, rs.getString("datasourceid")));
+ ds.setId(createOpenaireId(10, rs.getString("datasourceid"), true));
ds.setOriginalId(Arrays.asList(rs.getString("datasourceid")));
ds.setCollectedfrom(listKeyValues(rs.getString("collectedfromid"), rs.getString("collectedfromname")));
ds.setPid(new ArrayList<>());
@@ -212,7 +222,7 @@ public class MigrateDbEntitiesApplication extends AbstractMigrationExecutor impl
final Project p = new Project();
- p.setId(createOpenaireId(40, rs.getString("projectid")));
+ p.setId(createOpenaireId(40, rs.getString("projectid"), true));
p.setOriginalId(Arrays.asList(rs.getString("projectid")));
p.setCollectedfrom(listKeyValues(rs.getString("collectedfromid"), rs.getString("collectedfromname")));
p.setPid(new ArrayList<>());
@@ -302,7 +312,7 @@ public class MigrateDbEntitiesApplication extends AbstractMigrationExecutor impl
final Organization o = new Organization();
- o.setId(createOpenaireId(20, rs.getString("organizationid")));
+ o.setId(createOpenaireId(20, rs.getString("organizationid"), true));
o.setOriginalId(Arrays.asList(rs.getString("organizationid")));
o.setCollectedfrom(listKeyValues(rs.getString("collectedfromid"), rs.getString("collectedfromname")));
o.setPid(new ArrayList<>());
@@ -366,8 +376,8 @@ public class MigrateDbEntitiesApplication extends AbstractMigrationExecutor impl
try {
final DataInfo info = prepareDataInfo(rs);
- final String orgId = createOpenaireId(20, rs.getString("organization"));
- final String dsId = createOpenaireId(10, rs.getString("datasource"));
+ final String orgId = createOpenaireId(20, rs.getString("organization"), true);
+ final String dsId = createOpenaireId(10, rs.getString("datasource"), true);
final List collectedFrom = listKeyValues(rs.getString("collectedfromid"), rs.getString("collectedfromname"));
final Relation r1 = new Relation();
@@ -415,8 +425,8 @@ public class MigrateDbEntitiesApplication extends AbstractMigrationExecutor impl
try {
final DataInfo info = prepareDataInfo(rs);
- final String orgId = createOpenaireId(20, rs.getString("resporganization"));
- final String projectId = createOpenaireId(40, rs.getString("project"));
+ final String orgId = createOpenaireId(20, rs.getString("resporganization"), true);
+ final String projectId = createOpenaireId(40, rs.getString("project"), true);
final List collectedFrom = listKeyValues(rs.getString("collectedfromid"), rs.getString("collectedfromname"));
final Relation r1 = new Relation();
@@ -481,14 +491,14 @@ public class MigrateDbEntitiesApplication extends AbstractMigrationExecutor impl
} else {
r = new Publication();
}
- r.setId(createOpenaireId(50, rs.getString("target_id")));
+ r.setId(createOpenaireId(50, rs.getString("target_id"), false));
r.setLastupdatetimestamp(lastUpdateTimestamp);
r.setContext(prepareContext(rs.getString("source_id"), info));
r.setDataInfo(info);
emitOaf(r);
} else {
- final String sourceId = createOpenaireId(rs.getString("source_type"), rs.getString("source_id"));
- final String targetId = createOpenaireId(rs.getString("target_type"), rs.getString("target_id"));
+ final String sourceId = createOpenaireId(rs.getString("source_type"), rs.getString("source_id"), false);
+ final String targetId = createOpenaireId(rs.getString("target_type"), rs.getString("target_id"), false);
final Relation r1 = new Relation();
final Relation r2 = new Relation();
diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/step1/MigrateMongoMdstoresApplication.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/step1/MigrateMongoMdstoresApplication.java
new file mode 100644
index 0000000000..b1de31326b
--- /dev/null
+++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/step1/MigrateMongoMdstoresApplication.java
@@ -0,0 +1,67 @@
+package eu.dnetlib.dhp.migration.step1;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.Map;
+import java.util.Map.Entry;
+
+import org.apache.commons.io.IOUtils;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+import eu.dnetlib.dhp.application.ArgumentApplicationParser;
+import eu.dnetlib.dhp.migration.utils.AbstractMigrationApplication;
+import eu.dnetlib.dhp.migration.utils.MdstoreClient;
+
+public class MigrateMongoMdstoresApplication extends AbstractMigrationApplication implements Closeable {
+
+ private static final Log log = LogFactory.getLog(MigrateMongoMdstoresApplication.class);
+
+ private final MdstoreClient mdstoreClient;
+
+ public static void main(final String[] args) throws Exception {
+ final ArgumentApplicationParser parser = new ArgumentApplicationParser(
+ IOUtils.toString(MigrateMongoMdstoresApplication.class.getResourceAsStream("/eu/dnetlib/dhp/migration/migrate_mongo_mstores_parameters.json")));
+ parser.parseArgument(args);
+
+ final String mongoBaseUrl = parser.get("mongoBaseUrl");
+ final String mongoDb = parser.get("mongoDb");
+
+ final String mdFormat = parser.get("mdFormat");
+ final String mdLayout = parser.get("mdLayout");
+ final String mdInterpretation = parser.get("mdInterpretation");
+
+ final String hdfsPath = parser.get("hdfsPath");
+
+ try (MigrateMongoMdstoresApplication app = new MigrateMongoMdstoresApplication(hdfsPath, mongoBaseUrl, mongoDb)) {
+ app.execute(mdFormat, mdLayout, mdInterpretation);
+ }
+
+ }
+
+ public MigrateMongoMdstoresApplication(final String hdfsPath, final String mongoBaseUrl, final String mongoDb) throws Exception {
+ super(hdfsPath);
+ this.mdstoreClient = new MdstoreClient(mongoBaseUrl, mongoDb);
+ }
+
+ public void execute(final String format, final String layout, final String interpretation) {
+ final Map colls = mdstoreClient.validCollections(format, layout, interpretation);
+ log.info("Found " + colls.size() + " mdstores");
+
+ for (final Entry entry : colls.entrySet()) {
+ log.info("Processing mdstore " + entry.getKey() + " (collection: " + entry.getValue() + ")");
+ final String currentColl = entry.getValue();
+
+ for (final String xml : mdstoreClient.listRecords(currentColl)) {
+ emit(xml, "native_" + format);
+ }
+ }
+ }
+
+ @Override
+ public void close() throws IOException {
+ super.close();
+ mdstoreClient.close();
+ }
+
+}
diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/AbstractMongoExecutor.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/step2/AbstractMdRecordToOafMapper.java
similarity index 80%
rename from dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/AbstractMongoExecutor.java
rename to dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/step2/AbstractMdRecordToOafMapper.java
index 0595726d45..7c3000fbad 100644
--- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/AbstractMongoExecutor.java
+++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/step2/AbstractMdRecordToOafMapper.java
@@ -1,20 +1,24 @@
-package eu.dnetlib.dhp.migration;
+package eu.dnetlib.dhp.migration.step2;
+
+import static eu.dnetlib.dhp.migration.utils.OafMapperUtils.createOpenaireId;
+import static eu.dnetlib.dhp.migration.utils.OafMapperUtils.dataInfo;
+import static eu.dnetlib.dhp.migration.utils.OafMapperUtils.field;
+import static eu.dnetlib.dhp.migration.utils.OafMapperUtils.journal;
+import static eu.dnetlib.dhp.migration.utils.OafMapperUtils.keyValue;
+import static eu.dnetlib.dhp.migration.utils.OafMapperUtils.listFields;
+import static eu.dnetlib.dhp.migration.utils.OafMapperUtils.oaiIProvenance;
+import static eu.dnetlib.dhp.migration.utils.OafMapperUtils.qualifier;
+import static eu.dnetlib.dhp.migration.utils.OafMapperUtils.structuredProperty;
-import java.io.IOException;
-import java.sql.SQLException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Date;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
-import java.util.Map.Entry;
import org.apache.commons.lang3.StringUtils;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
import org.dom4j.Document;
-import org.dom4j.DocumentException;
import org.dom4j.DocumentFactory;
import org.dom4j.DocumentHelper;
import org.dom4j.Node;
@@ -37,11 +41,9 @@ import eu.dnetlib.dhp.schema.oaf.Result;
import eu.dnetlib.dhp.schema.oaf.Software;
import eu.dnetlib.dhp.schema.oaf.StructuredProperty;
-public abstract class AbstractMongoExecutor extends AbstractMigrationExecutor {
+public abstract class AbstractMdRecordToOafMapper {
- protected final Map code2name = new HashMap<>();
-
- protected final MdstoreClient mdstoreClient;
+ protected final Map code2name;
protected static final Qualifier MAIN_TITLE_QUALIFIER = qualifier("main title", "main title", "dnet:dataCite_title", "dnet:dataCite_title");
@@ -51,79 +53,36 @@ public abstract class AbstractMongoExecutor extends AbstractMigrationExecutor {
protected static final Qualifier SOFTWARE_RESULTTYPE_QUALIFIER = qualifier("software", "software", "dnet:result_typologies", "dnet:result_typologies");
protected static final Qualifier OTHER_RESULTTYPE_QUALIFIER = qualifier("other", "other", "dnet:result_typologies", "dnet:result_typologies");
- private static final Log log = LogFactory.getLog(AbstractMongoExecutor.class);
-
- public AbstractMongoExecutor(final String hdfsPath, final String hdfsNameNode, final String hdfsUser, final String mongoBaseUrl,
- final String mongoDb, final String dbUrl, final String dbUser,
- final String dbPassword) throws Exception {
-
- super(hdfsPath, hdfsNameNode, hdfsUser);
-
- this.mdstoreClient = new MdstoreClient(mongoBaseUrl, mongoDb);
- loadClassNames(dbUrl, dbUser, dbPassword);
-
- final Map nsContext = new HashMap<>();
-
- registerNamespaces(nsContext);
-
- DocumentFactory.getInstance().setXPathNamespaceURIs(nsContext);
+ protected AbstractMdRecordToOafMapper(final Map code2name) {
+ this.code2name = code2name;
}
- private void loadClassNames(final String dbUrl, final String dbUser, final String dbPassword) throws IOException {
+ public List processMdRecord(final String xml) {
+ try {
+ final Map nsContext = new HashMap<>();
+ nsContext.put("dr", "http://www.driver-repository.eu/namespace/dr");
+ nsContext.put("dri", "http://www.driver-repository.eu/namespace/dri");
+ nsContext.put("oaf", "http://namespace.openaire.eu/oaf");
+ nsContext.put("oai", "http://www.openarchives.org/OAI/2.0/");
+ nsContext.put("prov", "http://www.openarchives.org/OAI/2.0/provenance");
+ nsContext.put("dc", "http://purl.org/dc/elements/1.1/");
+ nsContext.put("datacite", "http://datacite.org/schema/kernel-3");
+ DocumentFactory.getInstance().setXPathNamespaceURIs(nsContext);
- log.info("Loading vocabulary terms from db...");
+ final Document doc = DocumentHelper.parseText(xml);
- try (DbClient dbClient = new DbClient(dbUrl, dbUser, dbPassword)) {
- code2name.clear();
- dbClient.processResults("select code, name from class", rs -> {
- try {
- code2name.put(rs.getString("code"), rs.getString("name"));
- } catch (final SQLException e) {
- e.printStackTrace();
- }
- });
+ final String type = doc.valueOf("//dr:CobjCategory/@type");
+ final KeyValue collectedFrom = keyValue(doc.valueOf("//oaf:collectedFrom/@id"), doc.valueOf("//oaf:collectedFrom/@name"));
+ final KeyValue hostedBy = StringUtils.isBlank(doc.valueOf("//oaf:hostedBy/@id")) ? collectedFrom
+ : keyValue(doc.valueOf("//oaf:hostedBy/@id"), doc.valueOf("//oaf:hostedBy/@name"));
+
+ final DataInfo info = prepareDataInfo(doc);
+ final long lastUpdateTimestamp = new Date().getTime();
+
+ return createOafs(doc, type, collectedFrom, hostedBy, info, lastUpdateTimestamp);
+ } catch (final Exception e) {
+ throw new RuntimeException(e);
}
-
- log.info("Found " + code2name.size() + " terms.");
-
- }
-
- public void processMdRecords(final String mdFormat, final String mdLayout, final String mdInterpretation) throws DocumentException {
-
- log.info(String.format("Searching mdstores (format: %s, layout: %s, interpretation: %s)", mdFormat, mdLayout, mdInterpretation));
-
- final Map colls = mdstoreClient.validCollections(mdFormat, mdLayout, mdInterpretation);
- log.info("Found " + colls.size() + " mdstores");
-
- for (final Entry entry : colls.entrySet()) {
- log.info("Processing mdstore " + entry.getKey() + " (collection: " + entry.getValue() + ")");
- final String currentColl = entry.getValue();
-
- for (final String xml : mdstoreClient.listRecords(currentColl)) {
- final Document doc = DocumentHelper.parseText(xml);
-
- final String type = doc.valueOf("//dr:CobjCategory/@type");
- final KeyValue collectedFrom = keyValue(doc.valueOf("//oaf:collectedFrom/@id"), doc.valueOf("//oaf:collectedFrom/@name"));
- final KeyValue hostedBy = StringUtils.isBlank(doc.valueOf("//oaf:hostedBy/@id")) ? collectedFrom
- : keyValue(doc.valueOf("//oaf:hostedBy/@id"), doc.valueOf("//oaf:hostedBy/@name"));
-
- final DataInfo info = prepareDataInfo(doc);
- final long lastUpdateTimestamp = new Date().getTime();
-
- for (final Oaf oaf : createOafs(doc, type, collectedFrom, hostedBy, info, lastUpdateTimestamp)) {
- emitOaf(oaf);
- }
- }
- }
- log.info("All Done.");
- }
-
- protected void registerNamespaces(final Map nsContext) {
- nsContext.put("dr", "http://www.driver-repository.eu/namespace/dr");
- nsContext.put("dri", "http://www.driver-repository.eu/namespace/dri");
- nsContext.put("oaf", "http://namespace.openaire.eu/oaf");
- nsContext.put("oai", "http://www.openarchives.org/OAI/2.0/");
- nsContext.put("prov", "http://www.openarchives.org/OAI/2.0/provenance");
}
protected List createOafs(final Document doc,
@@ -194,10 +153,10 @@ public abstract class AbstractMongoExecutor extends AbstractMigrationExecutor {
final List res = new ArrayList<>();
- final String docId = createOpenaireId(50, doc.valueOf("//dri:objIdentifier"));
+ final String docId = createOpenaireId(50, doc.valueOf("//dri:objIdentifier"), false);
for (final Object o : doc.selectNodes("//oaf:projectid")) {
- final String projectId = createOpenaireId(40, ((Node) o).getText());
+ final String projectId = createOpenaireId(40, ((Node) o).getText(), true);
final Relation r1 = new Relation();
r1.setRelType("resultProject");
@@ -238,7 +197,7 @@ public abstract class AbstractMongoExecutor extends AbstractMigrationExecutor {
final long lastUpdateTimestamp) {
r.setDataInfo(info);
r.setLastupdatetimestamp(lastUpdateTimestamp);
- r.setId(createOpenaireId(50, doc.valueOf("//dri:objIdentifier")));
+ r.setId(createOpenaireId(50, doc.valueOf("//dri:objIdentifier"), false));
r.setOriginalId(Arrays.asList(doc.valueOf("//dri:objIdentifier")));
r.setCollectedfrom(Arrays.asList(collectedFrom));
r.setPid(prepareListStructProps(doc, "//oaf:identifier", "@identifierType", "dnet:pid_types", "dnet:pid_types", info));
@@ -432,10 +391,4 @@ public abstract class AbstractMongoExecutor extends AbstractMigrationExecutor {
return res;
}
- @Override
- public void close() throws IOException {
- super.close();
- mdstoreClient.close();
- }
-
}
diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/step2/GenerateEntitiesApplication.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/step2/GenerateEntitiesApplication.java
new file mode 100644
index 0000000000..7f907b0c8c
--- /dev/null
+++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/step2/GenerateEntitiesApplication.java
@@ -0,0 +1,173 @@
+package eu.dnetlib.dhp.migration.step2;
+
+import java.io.IOException;
+import java.sql.SQLException;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+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.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.compress.GzipCodec;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.sql.SparkSession;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+
+import eu.dnetlib.dhp.application.ArgumentApplicationParser;
+import eu.dnetlib.dhp.migration.step1.MigrateMongoMdstoresApplication;
+import eu.dnetlib.dhp.migration.utils.DbClient;
+import eu.dnetlib.dhp.schema.oaf.Dataset;
+import eu.dnetlib.dhp.schema.oaf.Datasource;
+import eu.dnetlib.dhp.schema.oaf.Oaf;
+import eu.dnetlib.dhp.schema.oaf.Organization;
+import eu.dnetlib.dhp.schema.oaf.OtherResearchProduct;
+import eu.dnetlib.dhp.schema.oaf.Project;
+import eu.dnetlib.dhp.schema.oaf.Publication;
+import eu.dnetlib.dhp.schema.oaf.Relation;
+import eu.dnetlib.dhp.schema.oaf.Software;
+import scala.Tuple2;
+
+public class GenerateEntitiesApplication {
+
+ private static final Log log = LogFactory.getLog(GenerateEntitiesApplication.class);
+
+ public static void main(final String[] args) throws Exception {
+ final ArgumentApplicationParser parser = new ArgumentApplicationParser(
+ IOUtils.toString(MigrateMongoMdstoresApplication.class
+ .getResourceAsStream("/eu/dnetlib/dhp/migration/generate_entities_parameters.json")));
+
+ parser.parseArgument(args);
+
+ final String sourcePaths = parser.get("sourcePaths");
+ final String targetPath = parser.get("targetPath");
+
+ final String dbUrl = parser.get("postgresUrl");
+ final String dbUser = parser.get("postgresUser");
+ final String dbPassword = parser.get("postgresPassword");
+
+ final Map code2name = loadClassNames(dbUrl, dbUser, dbPassword);
+
+ try (final SparkSession spark = newSparkSession(parser); final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext())) {
+ final List existingSourcePaths = Arrays.stream(sourcePaths.split(",")).filter(p -> exists(sc, p)).collect(Collectors.toList());
+ generateEntities(sc, code2name, existingSourcePaths, targetPath);
+ }
+ }
+
+ private static SparkSession newSparkSession(final ArgumentApplicationParser parser) {
+ return SparkSession
+ .builder()
+ .appName(GenerateEntitiesApplication.class.getSimpleName())
+ .master(parser.get("master"))
+ .getOrCreate();
+ }
+
+ private static void generateEntities(final JavaSparkContext sc,
+ final Map code2name,
+ final List sourcePaths,
+ final String targetPath) {
+
+ log.info("Generate entities from files:");
+ sourcePaths.forEach(log::info);
+
+ JavaRDD inputRdd = sc.emptyRDD();
+
+ for (final String sp : sourcePaths) {
+ inputRdd = inputRdd.union(sc.sequenceFile(sp, Text.class, Text.class)
+ .map(k -> new Tuple2<>(k._1().toString(), k._2().toString()))
+ .map(k -> convertToListOaf(k._1(), k._2(), code2name))
+ .flatMap(list -> list.iterator())
+ .map(oaf -> oaf.getClass().getSimpleName().toLowerCase() + "|" + convertToJson(oaf)));
+ }
+
+ inputRdd.saveAsTextFile(targetPath, GzipCodec.class);
+
+ }
+
+ private static List convertToListOaf(final String id, final String s, final Map code2name) {
+ final String type = StringUtils.substringAfter(id, ":");
+
+ switch (type.toLowerCase()) {
+ case "native_oaf":
+ return new OafToOafMapper(code2name).processMdRecord(s);
+ case "native_odf":
+ return new OdfToOafMapper(code2name).processMdRecord(s);
+ case "datasource":
+ return Arrays.asList(convertFromJson(s, Datasource.class));
+ case "organization":
+ return Arrays.asList(convertFromJson(s, Organization.class));
+ case "project":
+ return Arrays.asList(convertFromJson(s, Project.class));
+ case "relation":
+ return Arrays.asList(convertFromJson(s, Relation.class));
+ case "publication":
+ return Arrays.asList(convertFromJson(s, Publication.class));
+ case "dataset":
+ return Arrays.asList(convertFromJson(s, Dataset.class));
+ case "software":
+ return Arrays.asList(convertFromJson(s, Software.class));
+ case "otherresearchproducts":
+ default:
+ return Arrays.asList(convertFromJson(s, OtherResearchProduct.class));
+ }
+
+ }
+
+ private static Map loadClassNames(final String dbUrl, final String dbUser, final String dbPassword) throws IOException {
+
+ log.info("Loading vocabulary terms from db...");
+
+ final Map map = new HashMap<>();
+
+ try (DbClient dbClient = new DbClient(dbUrl, dbUser, dbPassword)) {
+ dbClient.processResults("select code, name from class", rs -> {
+ try {
+ map.put(rs.getString("code"), rs.getString("name"));
+ } catch (final SQLException e) {
+ e.printStackTrace();
+ }
+ });
+ }
+
+ log.info("Found " + map.size() + " terms.");
+
+ return map;
+
+ }
+
+ private static String convertToJson(final Oaf oaf) {
+ try {
+ return new ObjectMapper().writeValueAsString(oaf);
+ } catch (final Exception e) {
+ throw new RuntimeException(e);
+ }
+ }
+
+ private static Oaf convertFromJson(final String s, final Class extends Oaf> clazz) {
+ try {
+ return new ObjectMapper().readValue(s, clazz);
+ } catch (final Exception e) {
+ log.error("Error parsing object of class: " + clazz);
+ log.error(s);
+ throw new RuntimeException(e);
+ }
+ }
+
+ private static boolean exists(final JavaSparkContext context, final String pathToFile) {
+ try {
+ final FileSystem hdfs = org.apache.hadoop.fs.FileSystem.get(context.hadoopConfiguration());
+ final Path path = new Path(pathToFile);
+ return hdfs.exists(path);
+ } catch (final IOException e) {
+ throw new RuntimeException(e);
+ }
+ }
+}
diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/OafMigrationExecutor.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/step2/OafToOafMapper.java
similarity index 89%
rename from dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/OafMigrationExecutor.java
rename to dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/step2/OafToOafMapper.java
index c325682905..110abc4864 100644
--- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/OafMigrationExecutor.java
+++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/step2/OafToOafMapper.java
@@ -1,16 +1,17 @@
-package eu.dnetlib.dhp.migration;
+package eu.dnetlib.dhp.migration.step2;
+
+import static eu.dnetlib.dhp.migration.utils.OafMapperUtils.createOpenaireId;
+import static eu.dnetlib.dhp.migration.utils.OafMapperUtils.field;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import java.util.Map;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
import org.dom4j.Document;
import org.dom4j.Node;
-import eu.dnetlib.dhp.migration.pace.PacePerson;
+import eu.dnetlib.dhp.migration.utils.PacePerson;
import eu.dnetlib.dhp.schema.oaf.Author;
import eu.dnetlib.dhp.schema.oaf.DataInfo;
import eu.dnetlib.dhp.schema.oaf.Field;
@@ -22,20 +23,10 @@ import eu.dnetlib.dhp.schema.oaf.Qualifier;
import eu.dnetlib.dhp.schema.oaf.Relation;
import eu.dnetlib.dhp.schema.oaf.StructuredProperty;
-public class OafMigrationExecutor extends AbstractMongoExecutor {
+public class OafToOafMapper extends AbstractMdRecordToOafMapper {
- private static final Log log = LogFactory.getLog(OafMigrationExecutor.class);
-
- public OafMigrationExecutor(final String hdfsPath, final String hdfsNameNode, final String hdfsUser, final String mongoBaseUrl, final String mongoDb,
- final String dbUrl, final String dbUser,
- final String dbPassword) throws Exception {
- super(hdfsPath, hdfsNameNode, hdfsUser, mongoBaseUrl, mongoDb, dbUrl, dbUser, dbPassword);
- }
-
- @Override
- protected void registerNamespaces(final Map nsContext) {
- super.registerNamespaces(nsContext);
- nsContext.put("dc", "http://purl.org/dc/elements/1.1/");
+ public OafToOafMapper(final Map code2name) {
+ super(code2name);
}
@Override
@@ -211,12 +202,12 @@ public class OafMigrationExecutor extends AbstractMongoExecutor {
final KeyValue collectedFrom,
final DataInfo info,
final long lastUpdateTimestamp) {
- final String docId = createOpenaireId(50, doc.valueOf("//dri:objIdentifier"));
+ final String docId = createOpenaireId(50, doc.valueOf("//dri:objIdentifier"), false);
final List res = new ArrayList<>();
for (final Object o : doc.selectNodes("//*[local-name()='relatedDataset']")) {
- final String otherId = createOpenaireId(50, ((Node) o).getText());
+ final String otherId = createOpenaireId(50, ((Node) o).getText(), false);
final Relation r1 = new Relation();
r1.setRelType("resultResult");
diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/OdfMigrationExecutor.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/step2/OdfToOafMapper.java
similarity index 73%
rename from dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/OdfMigrationExecutor.java
rename to dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/step2/OdfToOafMapper.java
index 457534085a..b4868b8f9b 100644
--- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/OdfMigrationExecutor.java
+++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/step2/OdfToOafMapper.java
@@ -1,4 +1,8 @@
-package eu.dnetlib.dhp.migration;
+package eu.dnetlib.dhp.migration.step2;
+
+import static eu.dnetlib.dhp.migration.utils.OafMapperUtils.createOpenaireId;
+import static eu.dnetlib.dhp.migration.utils.OafMapperUtils.field;
+import static eu.dnetlib.dhp.migration.utils.OafMapperUtils.structuredProperty;
import java.util.ArrayList;
import java.util.Arrays;
@@ -6,8 +10,6 @@ import java.util.List;
import java.util.Map;
import org.apache.commons.lang3.StringUtils;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
import org.dom4j.Document;
import org.dom4j.Node;
@@ -22,38 +24,28 @@ import eu.dnetlib.dhp.schema.oaf.Qualifier;
import eu.dnetlib.dhp.schema.oaf.Relation;
import eu.dnetlib.dhp.schema.oaf.StructuredProperty;
-public class OdfMigrationExecutor extends AbstractMongoExecutor {
+public class OdfToOafMapper extends AbstractMdRecordToOafMapper {
- private static final Log log = LogFactory.getLog(OdfMigrationExecutor.class);
-
- public OdfMigrationExecutor(final String hdfsPath, final String hdfsNameNode, final String hdfsUser, final String mongoBaseUrl, final String mongoDb,
- final String dbUrl, final String dbUser,
- final String dbPassword) throws Exception {
- super(hdfsPath, hdfsNameNode, hdfsUser, mongoBaseUrl, mongoDb, dbUrl, dbUser, dbPassword);
- }
-
- @Override
- protected void registerNamespaces(final Map nsContext) {
- super.registerNamespaces(nsContext);
- nsContext.put("dc", "http://datacite.org/schema/kernel-3");
+ public OdfToOafMapper(final Map code2name) {
+ super(code2name);
}
@Override
protected List prepareTitles(final Document doc, final DataInfo info) {
- return prepareListStructProps(doc, "//dc:title", MAIN_TITLE_QUALIFIER, info);
+ return prepareListStructProps(doc, "//datacite:title", MAIN_TITLE_QUALIFIER, info);
}
@Override
protected List prepareAuthors(final Document doc, final DataInfo info) {
final List res = new ArrayList<>();
int pos = 1;
- for (final Object o : doc.selectNodes("//dc:creator")) {
+ for (final Object o : doc.selectNodes("//datacite:creator")) {
final Node n = (Node) o;
final Author author = new Author();
- author.setFullname(n.valueOf("./dc:creatorName"));
- author.setName(n.valueOf("./dc:givenName"));
- author.setSurname(n.valueOf("./dc:familyName"));
- author.setAffiliation(prepareListFields(doc, "./dc:affiliation", info));
+ author.setFullname(n.valueOf("./datacite:creatorName"));
+ author.setName(n.valueOf("./datacite:givenName"));
+ author.setSurname(n.valueOf("./datacite:familyName"));
+ author.setAffiliation(prepareListFields(doc, "./datacite:affiliation", info));
author.setPid(preparePids(doc, info));
author.setRank(pos++);
res.add(author);
@@ -63,7 +55,7 @@ public class OdfMigrationExecutor extends AbstractMongoExecutor {
private List preparePids(final Document doc, final DataInfo info) {
final List res = new ArrayList<>();
- for (final Object o : doc.selectNodes("./dc:nameIdentifier")) {
+ for (final Object o : doc.selectNodes("./datacite:nameIdentifier")) {
res.add(structuredProperty(((Node) o).getText(), prepareQualifier((Node) o, "./@nameIdentifierScheme", "dnet:pid_types", "dnet:pid_types"), info));
}
return res;
@@ -72,7 +64,7 @@ public class OdfMigrationExecutor extends AbstractMongoExecutor {
@Override
protected List prepareInstances(final Document doc, final DataInfo info, final KeyValue collectedfrom, final KeyValue hostedby) {
final List res = new ArrayList<>();
- for (final Object o : doc.selectNodes("//dc:alternateIdentifier[@alternateIdentifierType='URL']")) {
+ for (final Object o : doc.selectNodes("//datacite:alternateIdentifier[@alternateIdentifierType='URL']")) {
final Instance instance = new Instance();
instance.setUrl(Arrays.asList(((Node) o).getText().trim()));
instance.setInstancetype(prepareQualifier(doc, "//dr:CobjCategory", "dnet:publication_resource", "dnet:publication_resource"));
@@ -98,7 +90,7 @@ public class OdfMigrationExecutor extends AbstractMongoExecutor {
@Override
protected List prepareRelevantDates(final Document doc, final DataInfo info) {
final List res = new ArrayList<>();
- for (final Object o : doc.selectNodes("//dc:date")) {
+ for (final Object o : doc.selectNodes("//datacite:date")) {
final String dateType = ((Node) o).valueOf("@dateType");
if (StringUtils.isBlank(dateType) && !dateType.equalsIgnoreCase("Accepted") && !dateType.equalsIgnoreCase("Issued")
&& !dateType.equalsIgnoreCase("Updated") && !dateType.equalsIgnoreCase("Available")) {
@@ -115,32 +107,32 @@ public class OdfMigrationExecutor extends AbstractMongoExecutor {
@Override
protected List> prepareContributors(final Document doc, final DataInfo info) {
- return prepareListFields(doc, "//dc:contributorName", info);
+ return prepareListFields(doc, "//datacite:contributorName", info);
}
@Override
protected List> prepareFormats(final Document doc, final DataInfo info) {
- return prepareListFields(doc, "//dc:format", info);
+ return prepareListFields(doc, "//datacite:format", info);
}
@Override
protected Field preparePublisher(final Document doc, final DataInfo info) {
- return prepareField(doc, "//dc:publisher", info);
+ return prepareField(doc, "//datacite:publisher", info);
}
@Override
protected List> prepareDescriptions(final Document doc, final DataInfo info) {
- return prepareListFields(doc, "//dc:description[@descriptionType='Abstract']", info);
+ return prepareListFields(doc, "//datacite:description[@descriptionType='Abstract']", info);
}
@Override
protected List prepareSubjects(final Document doc, final DataInfo info) {
- return prepareListStructProps(doc, "//dc:subject", info);
+ return prepareListStructProps(doc, "//datacite:subject", info);
}
@Override
protected Qualifier prepareLanguages(final Document doc) {
- return prepareQualifier(doc, "//dc:language", "dnet:languages", "dnet:languages");
+ return prepareQualifier(doc, "//datacite:language", "dnet:languages", "dnet:languages");
}
@Override
@@ -150,17 +142,17 @@ public class OdfMigrationExecutor extends AbstractMongoExecutor {
@Override
protected List> prepareOtherResearchProductContactGroups(final Document doc, final DataInfo info) {
- return prepareListFields(doc, "//dc:contributor[@contributorType='ContactGroup']/dc:contributorName", info);
+ return prepareListFields(doc, "//datacite:contributor[@contributorType='ContactGroup']/datacite:contributorName", info);
}
@Override
protected List> prepareOtherResearchProductContactPersons(final Document doc, final DataInfo info) {
- return prepareListFields(doc, "//dc:contributor[@contributorType='ContactPerson']/dc:contributorName", info);
+ return prepareListFields(doc, "//datacite:contributor[@contributorType='ContactPerson']/datacite:contributorName", info);
}
@Override
protected Qualifier prepareSoftwareProgrammingLanguage(final Document doc, final DataInfo info) {
- return prepareQualifier(doc, "//dc:format", "dnet:programming_languages", "dnet:programming_languages");
+ return prepareQualifier(doc, "//datacite:format", "dnet:programming_languages", "dnet:programming_languages");
}
@Override
@@ -175,7 +167,7 @@ public class OdfMigrationExecutor extends AbstractMongoExecutor {
@Override
protected List> prepareSoftwareDocumentationUrls(final Document doc, final DataInfo info) {
- return prepareListFields(doc, "//dc:relatedIdentifier[@relatedIdentifierType='URL' and @relationType='IsDocumentedBy']", info);
+ return prepareListFields(doc, "//datacite:relatedIdentifier[@relatedIdentifierType='URL' and @relationType='IsDocumentedBy']", info);
}
// DATASETS
@@ -184,11 +176,11 @@ public class OdfMigrationExecutor extends AbstractMongoExecutor {
protected List prepareDatasetGeoLocations(final Document doc, final DataInfo info) {
final List res = new ArrayList<>();
- for (final Object o : doc.selectNodes("//dc:geoLocation")) {
+ for (final Object o : doc.selectNodes("//datacite:geoLocation")) {
final GeoLocation loc = new GeoLocation();
- loc.setBox(((Node) o).valueOf("./dc:geoLocationBox"));
- loc.setPlace(((Node) o).valueOf("./dc:geoLocationPlace"));
- loc.setPoint(((Node) o).valueOf("./dc:geoLocationPoint"));
+ loc.setBox(((Node) o).valueOf("./datacite:geoLocationBox"));
+ loc.setPlace(((Node) o).valueOf("./datacite:geoLocationPlace"));
+ loc.setPoint(((Node) o).valueOf("./datacite:geoLocationPoint"));
res.add(loc);
}
return res;
@@ -201,17 +193,17 @@ public class OdfMigrationExecutor extends AbstractMongoExecutor {
@Override
protected Field prepareDatasetLastMetadataUpdate(final Document doc, final DataInfo info) {
- return prepareField(doc, "//dc:date[@dateType='Updated']", info);
+ return prepareField(doc, "//datacite:date[@dateType='Updated']", info);
}
@Override
protected Field prepareDatasetVersion(final Document doc, final DataInfo info) {
- return prepareField(doc, "//dc:version", info);
+ return prepareField(doc, "//datacite:version", info);
}
@Override
protected Field prepareDatasetSize(final Document doc, final DataInfo info) {
- return prepareField(doc, "//dc:size", info);
+ return prepareField(doc, "//datacite:size", info);
}
@Override
@@ -221,18 +213,18 @@ public class OdfMigrationExecutor extends AbstractMongoExecutor {
@Override
protected Field prepareDatasetStorageDate(final Document doc, final DataInfo info) {
- return prepareField(doc, "//dc:date[@dateType='Issued']", info);
+ return prepareField(doc, "//datacite:date[@dateType='Issued']", info);
}
@Override
protected List addOtherResultRels(final Document doc, final KeyValue collectedFrom, final DataInfo info, final long lastUpdateTimestamp) {
- final String docId = createOpenaireId(50, doc.valueOf("//dri:objIdentifier"));
+ final String docId = createOpenaireId(50, doc.valueOf("//dri:objIdentifier"), false);
final List res = new ArrayList<>();
- for (final Object o : doc.selectNodes("//*[local-name() = 'resource']//*[local-name()='relatedIdentifier' and ./@relatedIdentifierType='OPENAIRE']")) {
- final String otherId = createOpenaireId(50, ((Node) o).getText());
+ for (final Object o : doc.selectNodes("//datacite:relatedIdentifier[@relatedIdentifierType='OPENAIRE']")) {
+ final String otherId = createOpenaireId(50, ((Node) o).getText(), false);
final String type = ((Node) o).valueOf("@relationType");
if (type.equals("IsSupplementTo")) {
diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/step3/DispatchEntitiesApplication.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/step3/DispatchEntitiesApplication.java
new file mode 100644
index 0000000000..4ee24cba0a
--- /dev/null
+++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/step3/DispatchEntitiesApplication.java
@@ -0,0 +1,71 @@
+package eu.dnetlib.dhp.migration.step3;
+
+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.io.compress.GzipCodec;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.sql.SparkSession;
+
+import eu.dnetlib.dhp.application.ArgumentApplicationParser;
+import eu.dnetlib.dhp.migration.step1.MigrateMongoMdstoresApplication;
+import eu.dnetlib.dhp.schema.oaf.Dataset;
+import eu.dnetlib.dhp.schema.oaf.Datasource;
+import eu.dnetlib.dhp.schema.oaf.Organization;
+import eu.dnetlib.dhp.schema.oaf.OtherResearchProduct;
+import eu.dnetlib.dhp.schema.oaf.Project;
+import eu.dnetlib.dhp.schema.oaf.Publication;
+import eu.dnetlib.dhp.schema.oaf.Relation;
+import eu.dnetlib.dhp.schema.oaf.Software;
+
+public class DispatchEntitiesApplication {
+
+ private static final Log log = LogFactory.getLog(DispatchEntitiesApplication.class);
+
+ public static void main(final String[] args) throws Exception {
+ final ArgumentApplicationParser parser = new ArgumentApplicationParser(
+ IOUtils.toString(MigrateMongoMdstoresApplication.class
+ .getResourceAsStream("/eu/dnetlib/dhp/migration/dispatch_entities_parameters.json")));
+ parser.parseArgument(args);
+
+ try (final SparkSession spark = newSparkSession(parser); final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext())) {
+
+ final String sourcePath = parser.get("sourcePath");
+ final String targetPath = parser.get("graphRawPath");
+
+ processEntity(sc, Publication.class, sourcePath, targetPath);
+ processEntity(sc, Dataset.class, sourcePath, targetPath);
+ processEntity(sc, Software.class, sourcePath, targetPath);
+ processEntity(sc, OtherResearchProduct.class, sourcePath, targetPath);
+ processEntity(sc, Datasource.class, sourcePath, targetPath);
+ processEntity(sc, Organization.class, sourcePath, targetPath);
+ processEntity(sc, Project.class, sourcePath, targetPath);
+ processEntity(sc, Relation.class, sourcePath, targetPath);
+ }
+ }
+
+ private static SparkSession newSparkSession(final ArgumentApplicationParser parser) {
+ return SparkSession
+ .builder()
+ .appName(DispatchEntitiesApplication.class.getSimpleName())
+ .master(parser.get("master"))
+ .getOrCreate();
+ }
+
+ private static void processEntity(final JavaSparkContext sc, final Class> clazz, final String sourcePath, final String targetPath) {
+ final String type = clazz.getSimpleName().toLowerCase();
+
+ log.info(String.format("Processing entities (%s) in file: %s", type, sourcePath));
+
+ sc.textFile(sourcePath)
+ .filter(l -> isEntityType(l, type))
+ .map(l -> StringUtils.substringAfter(l, "|"))
+ .saveAsTextFile(targetPath + "/" + type, GzipCodec.class); // use repartition(XXX) ???
+ }
+
+ private static boolean isEntityType(final String line, final String type) {
+ return StringUtils.substringBefore(line, "|").equalsIgnoreCase(type);
+ }
+
+}
diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/utils/AbstractMigrationApplication.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/utils/AbstractMigrationApplication.java
new file mode 100644
index 0000000000..8eb444562c
--- /dev/null
+++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/utils/AbstractMigrationApplication.java
@@ -0,0 +1,77 @@
+package eu.dnetlib.dhp.migration.utils;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.concurrent.atomic.AtomicInteger;
+
+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.codehaus.jackson.map.ObjectMapper;
+
+import eu.dnetlib.dhp.schema.oaf.Oaf;
+
+public class AbstractMigrationApplication implements Closeable {
+
+ private final AtomicInteger counter = new AtomicInteger(0);
+
+ private final Text key = new Text();
+
+ private final Text value = new Text();
+
+ private final SequenceFile.Writer writer;
+
+ private final ObjectMapper objectMapper = new ObjectMapper();
+
+ private static final Log log = LogFactory.getLog(AbstractMigrationApplication.class);
+
+ public AbstractMigrationApplication(final String hdfsPath) throws Exception {
+
+ log.info(String.format("Creating SequenceFile Writer, hdfsPath=%s", hdfsPath));
+
+ this.writer = SequenceFile.createWriter(getConf(), SequenceFile.Writer.file(new Path(hdfsPath)), SequenceFile.Writer
+ .keyClass(Text.class), SequenceFile.Writer.valueClass(Text.class));
+ }
+
+ private Configuration getConf() throws IOException {
+ final Configuration conf = new Configuration();
+ /*
+ * conf.set("fs.defaultFS", hdfsNameNode); conf.set("fs.hdfs.impl", org.apache.hadoop.hdfs.DistributedFileSystem.class.getName());
+ * conf.set("fs.file.impl", org.apache.hadoop.fs.LocalFileSystem.class.getName()); System.setProperty("HADOOP_USER_NAME", hdfsUser);
+ * System.setProperty("hadoop.home.dir", "/"); FileSystem.get(URI.create(hdfsNameNode), conf);
+ */
+ return conf;
+ }
+
+ protected void emit(final String s, final String type) {
+ try {
+ key.set(counter.getAndIncrement() + ":" + type);
+ value.set(s);
+ writer.append(key, value);
+ } catch (final Exception e) {
+ throw new RuntimeException(e);
+ }
+ }
+
+ protected void emitOaf(final Oaf oaf) {
+ try {
+ emit(objectMapper.writeValueAsString(oaf), oaf.getClass().getSimpleName().toLowerCase());
+ } catch (final Exception e) {
+ throw new RuntimeException(e);
+ }
+ }
+
+ public ObjectMapper getObjectMapper() {
+ return objectMapper;
+ }
+
+ @Override
+ public void close() throws IOException {
+ writer.hflush();
+ writer.close();
+ }
+
+}
diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/DbClient.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/utils/DbClient.java
similarity index 78%
rename from dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/DbClient.java
rename to dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/utils/DbClient.java
index 9ac0089d25..8e97843464 100644
--- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/DbClient.java
+++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/utils/DbClient.java
@@ -1,4 +1,4 @@
-package eu.dnetlib.dhp.migration;
+package eu.dnetlib.dhp.migration.utils;
import java.io.Closeable;
import java.io.IOException;
@@ -28,8 +28,8 @@ public class DbClient implements Closeable {
StringUtils.isNoneBlank(login, password) ? DriverManager.getConnection(address, login, password) : DriverManager.getConnection(address);
this.connection.setAutoCommit(false);
} catch (final Exception e) {
- log.error(e.getClass().getName() + ": " + e.getMessage());
- throw new RuntimeException(e);
+ log.error("Connection to postgresDB failed");
+ throw new RuntimeException("Connection to postgresDB failed", e);
}
log.info("Opened database successfully");
}
@@ -44,10 +44,12 @@ public class DbClient implements Closeable {
consumer.accept(rs);
}
} catch (final SQLException e) {
- throw new RuntimeException(e);
+ log.error("Error executing sql query: " + sql, e);
+ throw new RuntimeException("Error executing sql query", e);
}
} catch (final SQLException e1) {
- throw new RuntimeException(e1);
+ log.error("Error preparing sql statement", e1);
+ throw new RuntimeException("Error preparing sql statement", e1);
}
}
diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/MdstoreClient.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/utils/MdstoreClient.java
similarity index 98%
rename from dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/MdstoreClient.java
rename to dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/utils/MdstoreClient.java
index 87dadfc7aa..612503da74 100644
--- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/MdstoreClient.java
+++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/utils/MdstoreClient.java
@@ -1,4 +1,4 @@
-package eu.dnetlib.dhp.migration;
+package eu.dnetlib.dhp.migration.utils;
import java.io.Closeable;
import java.io.IOException;
diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/AbstractMigrationExecutor.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/utils/OafMapperUtils.java
similarity index 65%
rename from dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/AbstractMigrationExecutor.java
rename to dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/utils/OafMapperUtils.java
index b0db3c76fd..8e51c1858e 100644
--- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/AbstractMigrationExecutor.java
+++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/utils/OafMapperUtils.java
@@ -1,24 +1,12 @@
-package eu.dnetlib.dhp.migration;
+package eu.dnetlib.dhp.migration.utils;
-import java.io.Closeable;
-import java.io.IOException;
-import java.net.URI;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import java.util.Objects;
-import java.util.concurrent.atomic.AtomicInteger;
import java.util.stream.Collectors;
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.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.SequenceFile;
-import org.apache.hadoop.io.Text;
-import org.codehaus.jackson.map.ObjectMapper;
import eu.dnetlib.dhp.schema.oaf.DataInfo;
import eu.dnetlib.dhp.schema.oaf.ExtraInfo;
@@ -26,60 +14,12 @@ import eu.dnetlib.dhp.schema.oaf.Field;
import eu.dnetlib.dhp.schema.oaf.Journal;
import eu.dnetlib.dhp.schema.oaf.KeyValue;
import eu.dnetlib.dhp.schema.oaf.OAIProvenance;
-import eu.dnetlib.dhp.schema.oaf.Oaf;
import eu.dnetlib.dhp.schema.oaf.OriginDescription;
import eu.dnetlib.dhp.schema.oaf.Qualifier;
import eu.dnetlib.dhp.schema.oaf.StructuredProperty;
import eu.dnetlib.dhp.utils.DHPUtils;
-public class AbstractMigrationExecutor implements Closeable {
-
- private final AtomicInteger counter = new AtomicInteger(0);
-
- private final Text key = new Text();
-
- private final Text value = new Text();
-
- private final ObjectMapper objectMapper = new ObjectMapper();
-
- private final SequenceFile.Writer writer;
-
- private static final Log log = LogFactory.getLog(AbstractMigrationExecutor.class);
-
- public AbstractMigrationExecutor(final String hdfsPath, final String hdfsNameNode, final String hdfsUser) throws Exception {
-
- log.info(String.format("Creating SequenceFile Writer, hdfsPath=%s, nameNode=%s, user=%s", hdfsPath, hdfsNameNode, hdfsUser));
-
- this.writer = SequenceFile.createWriter(getConf(hdfsNameNode, hdfsUser), SequenceFile.Writer.file(new Path(hdfsPath)), SequenceFile.Writer
- .keyClass(Text.class), SequenceFile.Writer.valueClass(Text.class));
- }
-
- private Configuration getConf(final String hdfsNameNode, final String hdfsUser) throws IOException {
- final Configuration conf = new Configuration();
- conf.set("fs.defaultFS", hdfsNameNode);
- conf.set("fs.hdfs.impl", org.apache.hadoop.hdfs.DistributedFileSystem.class.getName());
- conf.set("fs.file.impl", org.apache.hadoop.fs.LocalFileSystem.class.getName());
- System.setProperty("HADOOP_USER_NAME", hdfsUser);
- System.setProperty("hadoop.home.dir", "/");
- FileSystem.get(URI.create(hdfsNameNode), conf);
- return conf;
- }
-
- protected void emitOaf(final Oaf oaf) {
- try {
- key.set(counter.getAndIncrement() + ":" + oaf.getClass().getSimpleName().toLowerCase());
- value.set(objectMapper.writeValueAsString(oaf));
- writer.append(key, value);
- } catch (final Exception e) {
- throw new RuntimeException(e);
- }
- }
-
- @Override
- public void close() throws IOException {
- writer.hflush();
- writer.close();
- }
+public class OafMapperUtils {
public static KeyValue keyValue(final String k, final String v) {
final KeyValue kv = new KeyValue();
@@ -223,28 +163,33 @@ public class AbstractMigrationExecutor implements Closeable {
return d;
}
- public static String createOpenaireId(final int prefix, final String originalId) {
- final String nsPrefix = StringUtils.substringBefore(originalId, "::");
- final String rest = StringUtils.substringAfter(originalId, "::");
- return String.format("%s|%s::%s", prefix, nsPrefix, DHPUtils.md5(rest));
+ public static String createOpenaireId(final int prefix, final String originalId, final boolean to_md5) {
+ if (to_md5) {
+ final String nsPrefix = StringUtils.substringBefore(originalId, "::");
+ final String rest = StringUtils.substringAfter(originalId, "::");
+ return String.format("%s|%s::%s", prefix, nsPrefix, DHPUtils.md5(rest));
+ } else {
+ return String.format("%s|%s", prefix, originalId);
+ }
}
- public static String createOpenaireId(final String type, final String originalId) {
+ public static String createOpenaireId(final String type, final String originalId, final boolean to_md5) {
switch (type) {
case "datasource":
- return createOpenaireId(10, originalId);
+ return createOpenaireId(10, originalId, to_md5);
case "organization":
- return createOpenaireId(20, originalId);
+ return createOpenaireId(20, originalId, to_md5);
case "person":
- return createOpenaireId(30, originalId);
+ return createOpenaireId(30, originalId, to_md5);
case "project":
- return createOpenaireId(40, originalId);
+ return createOpenaireId(40, originalId, to_md5);
default:
- return createOpenaireId(50, originalId);
+ return createOpenaireId(50, originalId, to_md5);
}
}
public static String asString(final Object o) {
return o == null ? "" : o.toString();
}
+
}
diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/pace/PacePerson.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/utils/PacePerson.java
similarity index 99%
rename from dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/pace/PacePerson.java
rename to dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/utils/PacePerson.java
index 927f5641bf..69e128e63c 100644
--- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/pace/PacePerson.java
+++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/migration/utils/PacePerson.java
@@ -1,4 +1,4 @@
-package eu.dnetlib.dhp.migration.pace;
+package eu.dnetlib.dhp.migration.utils;
import java.nio.charset.Charset;
import java.text.Normalizer;
diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/extract_entities_from_hdfs_parameters.json b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/dispatch_entities_parameters.json
similarity index 70%
rename from dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/extract_entities_from_hdfs_parameters.json
rename to dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/dispatch_entities_parameters.json
index 0039493e7b..8c81290ca2 100644
--- a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/extract_entities_from_hdfs_parameters.json
+++ b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/dispatch_entities_parameters.json
@@ -1,8 +1,8 @@
[
{
"paramName": "s",
- "paramLongName": "sourcePaths",
- "paramDescription": "the HDFS source paths which contains the sequential file (comma separated)",
+ "paramLongName": "sourcePath",
+ "paramDescription": "the source path",
"paramRequired": true
},
{
diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/generate_entities_parameters.json b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/generate_entities_parameters.json
new file mode 100644
index 0000000000..53ee010c45
--- /dev/null
+++ b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/generate_entities_parameters.json
@@ -0,0 +1,39 @@
+[
+ {
+ "paramName": "s",
+ "paramLongName": "sourcePaths",
+ "paramDescription": "the HDFS source paths which contains the sequential file (comma separated)",
+ "paramRequired": true
+ },
+ {
+ "paramName": "mt",
+ "paramLongName": "master",
+ "paramDescription": "should be local or yarn",
+ "paramRequired": true
+ },
+ {
+ "paramName": "t",
+ "paramLongName": "targetPath",
+ "paramDescription": "the path of the target file",
+ "paramRequired": true
+ },
+ {
+ "paramName": "pgurl",
+ "paramLongName": "postgresUrl",
+ "paramDescription": "postgres url, example: jdbc:postgresql://localhost:5432/testdb",
+ "paramRequired": true
+ },
+ {
+ "paramName": "pguser",
+ "paramLongName": "postgresUser",
+ "paramDescription": "postgres user",
+ "paramRequired": false
+ },
+ {
+ "paramName": "pgpasswd",
+ "paramLongName": "postgresPassword",
+ "paramDescription": "postgres password",
+ "paramRequired": false
+ }
+
+]
\ No newline at end of file
diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/migrate_actionsets_parameters.json b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/migrate_actionsets_parameters.json
new file mode 100644
index 0000000000..c4910ec61b
--- /dev/null
+++ b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/migrate_actionsets_parameters.json
@@ -0,0 +1,10 @@
+[
+ {"paramName":"is", "paramLongName":"isLookupUrl", "paramDescription": "URL of the isLookUp Service", "paramRequired": true},
+ {"paramName":"sn", "paramLongName":"sourceNameNode", "paramDescription": "nameNode of the source cluster", "paramRequired": true},
+ {"paramName":"tn", "paramLongName":"targetNameNode", "paramDescription": "namoNode of the target cluster", "paramRequired": true},
+ {"paramName":"w", "paramLongName":"workingDirectory", "paramDescription": "working directory", "paramRequired": true},
+ {"paramName":"nm", "paramLongName":"distcp_num_maps", "paramDescription": "maximum number of map tasks used in the distcp process", "paramRequired": true},
+ {"paramName":"mm", "paramLongName":"distcp_memory_mb", "paramDescription": "memory for distcp action copying actionsets from remote cluster", "paramRequired": true},
+ {"paramName":"tt", "paramLongName":"distcp_task_timeout", "paramDescription": "timeout for distcp copying actions from remote cluster", "paramRequired": true},
+ {"paramName":"tr", "paramLongName":"transform_only", "paramDescription": "activate tranform-only mode. Only apply transformation step", "paramRequired": true}
+]
diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/migrate_db_entities_parameters.json b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/migrate_db_entities_parameters.json
index 4506e2ae1a..cb13ff0242 100644
--- a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/migrate_db_entities_parameters.json
+++ b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/migrate_db_entities_parameters.json
@@ -6,31 +6,19 @@
"paramRequired": true
},
{
- "paramName": "n",
- "paramLongName": "namenode",
- "paramDescription": "the Name Node URI",
- "paramRequired": true
- },
- {
- "paramName": "u",
- "paramLongName": "hdfsUser",
- "paramDescription": "the user wich create the hdfs seq file",
- "paramRequired": true
- },
- {
- "paramName": "dburl",
+ "paramName": "pgurl",
"paramLongName": "postgresUrl",
"paramDescription": "postgres url, example: jdbc:postgresql://localhost:5432/testdb",
"paramRequired": true
},
{
- "paramName": "dbuser",
+ "paramName": "pguser",
"paramLongName": "postgresUser",
"paramDescription": "postgres user",
"paramRequired": false
},
{
- "paramName": "dbpasswd",
+ "paramName": "pgpasswd",
"paramLongName": "postgresPassword",
"paramDescription": "postgres password",
"paramRequired": false
diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/migrate_mongo_mstores_parameters.json b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/migrate_mongo_mstores_parameters.json
index 5738daa762..ee1a6ac4ee 100644
--- a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/migrate_mongo_mstores_parameters.json
+++ b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/migrate_mongo_mstores_parameters.json
@@ -5,18 +5,6 @@
"paramDescription": "the path where storing the sequential file",
"paramRequired": true
},
- {
- "paramName": "n",
- "paramLongName": "namenode",
- "paramDescription": "the Name Node URI",
- "paramRequired": true
- },
- {
- "paramName": "u",
- "paramLongName": "hdfsUser",
- "paramDescription": "the user wich create the hdfs seq file",
- "paramRequired": true
- },
{
"paramName": "mongourl",
"paramLongName": "mongoBaseUrl",
@@ -24,7 +12,7 @@
"paramRequired": true
},
{
- "paramName": "db",
+ "paramName": "mongodb",
"paramLongName": "mongoDb",
"paramDescription": "mongo database",
"paramRequired": true
@@ -46,23 +34,5 @@
"paramLongName": "mdInterpretation",
"paramDescription": "metadata interpretation",
"paramRequired": true
- },
- {
- "paramName": "pgurl",
- "paramLongName": "postgresUrl",
- "paramDescription": "postgres url, example: jdbc:postgresql://localhost:5432/testdb",
- "paramRequired": true
- },
- {
- "paramName": "pguser",
- "paramLongName": "postgresUser",
- "paramDescription": "postgres user",
- "paramRequired": false
- },
- {
- "paramName": "pgpasswd",
- "paramLongName": "postgresPassword",
- "paramDescription": "postgres password",
- "paramRequired": false
}
]
\ No newline at end of file
diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/oozie_app/workflow.xml b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/oozie_app/workflow.xml
deleted file mode 100644
index 6589633217..0000000000
--- a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/oozie_app/workflow.xml
+++ /dev/null
@@ -1,206 +0,0 @@
-
-
-
- workingPath
- the base path to store hdfs file
-
-
- graphRawPath
- the graph Raw base path
-
-
-
- postgresURL
- the postgres URL to access to the database
-
-
- postgresUser
- the user postgres
-
-
- postgresPassword
- the password postgres
-
-
- mongourl
- mongoDB url, example: mongodb://[username:password@]host[:port]
-
-
- mongoDb
- mongo database
-
-
- sparkDriverMemory
- memory for driver process
-
-
- sparkExecutorMemory
- memory for individual executor
-
-
- sparkExecutorCores
- number of cores used by single executor
-
-
-
-
-
-
- Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}]
-
-
-
-
-
-
-
-
-
-
-
-
-
- ${jobTracker}
- ${nameNode}
- eu.dnetlib.dhp.migration.MigrateDbEntitiesApplication
- -p${workingPath}/db_entities
- -n${nameNode}
- -u${hdfsUser}
- -dburl${postgresURL}
- -dbuser${postgresUser}
- -dbpasswd${postgresPassword}
-
-
-
-
-
-
-
- ${jobTracker}
- ${nameNode}
- eu.dnetlib.dhp.migration.MigrateDbEntitiesApplication
- -p${workingPath}/db_claims
- -n${nameNode}
- -u${hdfsUser}
- -dburl${postgresURL}
- -dbuser${postgresUser}
- -dbpasswd${postgresPassword}
- -aclaims
-
-
-
-
-
-
-
- ${jobTracker}
- ${nameNode}
- eu.dnetlib.dhp.migration.MigrateMongoMdstoresApplication
- -p${workingPath}/odf_entities
- -n${nameNode}
- -u${hdfsUser}
- -mongourl${mongourl}
- -db${mongoDb}
- -fODF
- -lstore
- -icleaned
- -pgurl${postgresURL}
- -pguser${postgresUser}
- -pgpasswd${postgresPassword}
-
-
-
-
-
-
-
- ${jobTracker}
- ${nameNode}
- eu.dnetlib.dhp.migration.MigrateMongoMdstoresApplication
- -p${workingPath}/oaf_entities
- -n${nameNode}
- -u${hdfsUser}
- -mongourl${mongourl}
- -db${mongoDb}
- -fOAF
- -lstore
- -icleaned
- -pgurl${postgresURL}
- -pguser${postgresUser}
- -pgpasswd${postgresPassword}
-
-
-
-
-
-
-
- ${jobTracker}
- ${nameNode}
- eu.dnetlib.dhp.migration.MigrateMongoMdstoresApplication
- -p${workingPath}/odf_claims
- -n${nameNode}
- -u${hdfsUser}
- -mongourl${mongourl}
- -db${mongoDb}
- -fODF
- -lstore
- -iclaim
- -pgurl${postgresURL}
- -pguser${postgresUser}
- -pgpasswd${postgresPassword}
-
-
-
-
-
-
-
- ${jobTracker}
- ${nameNode}
- eu.dnetlib.dhp.migration.MigrateMongoMdstoresApplication
- -p${workingPath}/oaf_claims
- -n${nameNode}
- -u${hdfsUser}
- -mongourl${mongourl}
- -db${mongoDb}
- -fOAF
- -lstore
- -iclaim
- -pgurl${postgresURL}
- -pguser${postgresUser}
- -pgpasswd${postgresPassword}
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
- ${jobTracker}
- ${nameNode}
- yarn-cluster
- cluster
- ExtractEntities
- eu.dnetlib.dhp.migration.ExtractEntitiesFromHDFSJob
- dhp-aggregation-${projectVersion}.jar
- --executor-memory ${sparkExecutorMemory} --executor-cores ${sparkExecutorCores} --driver-memory=${sparkDriverMemory} --conf spark.extraListeners="com.cloudera.spark.lineage.NavigatorAppListener" --conf spark.sql.queryExecutionListeners="com.cloudera.spark.lineage.NavigatorQueryListener" --conf spark.sql.warehouse.dir="/user/hive/warehouse"
- -mt yarn-cluster
- -s${workingPath}/db_entities,${workingPath}/oaf_entities,${workingPath}/odf_entities
- -g${graphRawPath}
-
-
-
-
-
-
-
\ No newline at end of file
diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/transform_actionsets_parameters.json b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/transform_actionsets_parameters.json
new file mode 100644
index 0000000000..ce72f53ca6
--- /dev/null
+++ b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/transform_actionsets_parameters.json
@@ -0,0 +1,5 @@
+[
+ {"paramName":"mt", "paramLongName":"master", "paramDescription": "should be local or yarn", "paramRequired": true},
+ {"paramName":"is", "paramLongName":"isLookupUrl", "paramDescription": "URL of the isLookUp Service", "paramRequired": true},
+ {"paramName":"i", "paramLongName":"inputPaths", "paramDescription": "URL of the isLookUp Service", "paramRequired": true}
+]
diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/wfs/actions/oozie_app/config-default.xml b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/wfs/actions/oozie_app/config-default.xml
new file mode 100644
index 0000000000..9637ebdc62
--- /dev/null
+++ b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/wfs/actions/oozie_app/config-default.xml
@@ -0,0 +1,30 @@
+
+
+ jobTracker
+ yarnRM
+
+
+ nameNode
+ hdfs://nameservice1
+
+
+ sourceNN
+ webhdfs://namenode2.hadoop.dm.openaire.eu:50071
+
+
+ oozie.use.system.libpath
+ true
+
+
+ oozie.action.sharelib.for.spark
+ spark2
+
+
+ spark2YarnHistoryServerAddress
+ http://iis-cdh5-test-gw.ocean.icm.edu.pl:18088
+
+
+ spark2EventLogDir
+ /user/spark/applicationHistory
+
+
\ No newline at end of file
diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/wfs/actions/oozie_app/workflow.xml b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/wfs/actions/oozie_app/workflow.xml
new file mode 100644
index 0000000000..ed01c8de4c
--- /dev/null
+++ b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/wfs/actions/oozie_app/workflow.xml
@@ -0,0 +1,122 @@
+
+
+
+ sourceNN
+ the source name node
+
+
+ isLookupUrl
+ the isLookup service endpoint
+
+
+ workingDirectory
+ /tmp/actionsets
+ working directory
+
+
+ distcp_memory_mb
+ 6144
+ memory for distcp copying actionsets from remote cluster
+
+
+ distcp_task_timeout
+ 60000000
+ timeout for distcp copying actions from remote cluster
+
+
+ distcp_num_maps
+ 1
+ mmaximum number of map tasks used in the distcp process
+
+
+ transform_only
+ activate tranform-only mode. Only apply transformation step
+
+
+ sparkDriverMemory
+ memory for driver process
+
+
+ sparkExecutorMemory
+ memory for individual executor
+
+
+ sparkExecutorCores
+ number of cores used by single executor
+
+
+ spark2YarnHistoryServerAddress
+ spark 2.* yarn history server address
+
+
+ spark2EventLogDir
+ spark 2.* event log dir location
+
+
+
+
+ ${jobTracker}
+ ${nameNode}
+
+
+ mapreduce.job.queuename
+ ${queueName}
+
+
+ oozie.launcher.mapred.job.queue.name
+ ${oozieLauncherQueueName}
+
+
+
+
+
+
+
+
+ eu.dnetlib.dhp.migration.actions.MigrateActionSet
+ -Dmapred.task.timeout=${distcp_task_timeout}
+ -is${isLookupUrl}
+ -sn${sourceNN}
+ -tn${nameNode}
+ -w${workingDirectory}
+ -nm${distcp_num_maps}
+ -mm${distcp_memory_mb}
+ -tt${distcp_task_timeout}
+ -tr${transform_only}
+
+
+
+
+
+
+
+
+ yarn
+ cluster
+ transform_actions
+ eu.dnetlib.dhp.migration.actions.TransformActions
+ dhp-aggregation-${projectVersion}.jar
+
+ --executor-cores ${sparkExecutorCores}
+ --executor-memory ${sparkExecutorMemory}
+ --driver-memory=${sparkDriverMemory}
+ --conf spark.extraListeners="com.cloudera.spark.lineage.NavigatorAppListener"
+ --conf spark.sql.queryExecutionListeners="com.cloudera.spark.lineage.NavigatorQueryListener"
+ --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
+ --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
+
+ -mtyarn
+ -is${isLookupUrl}
+ --inputPaths${wf:actionData('migrate_actionsets')['target_paths']}
+
+
+
+
+
+
+ migrate_actions failed, error message[${wf:errorMessage(wf:lastErrorNode())}]
+
+
+
+
+
\ No newline at end of file
diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/oozie_app/config-default.xml b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/wfs/claims/oozie_app/config-default.xml
similarity index 80%
rename from dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/oozie_app/config-default.xml
rename to dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/wfs/claims/oozie_app/config-default.xml
index 51e48d8f75..2e0ed9aeea 100644
--- a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/oozie_app/config-default.xml
+++ b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/wfs/claims/oozie_app/config-default.xml
@@ -15,8 +15,4 @@
oozie.action.sharelib.for.spark
spark2
-
- hdfsUser
- dnet
-
\ No newline at end of file
diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/wfs/claims/oozie_app/workflow.xml b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/wfs/claims/oozie_app/workflow.xml
new file mode 100644
index 0000000000..1ac456976d
--- /dev/null
+++ b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/wfs/claims/oozie_app/workflow.xml
@@ -0,0 +1,169 @@
+
+
+
+ migrationClaimsPathStep1
+ the base path to store hdfs file
+
+
+ migrationClaimsPathStep2
+ the temporary path to store entities before dispatching
+
+
+ migrationClaimsPathStep3
+ the graph Raw base path
+
+
+ postgresURL
+ the postgres URL to access to the database
+
+
+ postgresUser
+ the user postgres
+
+
+ postgresPassword
+ the password postgres
+
+
+ mongoURL
+ mongoDB url, example: mongodb://[username:password@]host[:port]
+
+
+ mongoDb
+ mongo database
+
+
+ sparkDriverMemory
+ memory for driver process
+
+
+ sparkExecutorMemory
+ memory for individual executor
+
+
+ sparkExecutorCores
+ number of cores used by single executor
+
+
+
+
+
+
+ Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}]
+
+
+
+
+
+
+
+
+
+
+
+
+
+ ${jobTracker}
+ ${nameNode}
+ eu.dnetlib.dhp.migration.step1.MigrateDbEntitiesApplication
+ -p${migrationClaimsPathStep1}/db_claims
+ -pgurl${postgresURL}
+ -pguser${postgresUser}
+ -pgpasswd${postgresPassword}
+ -aclaims
+
+
+
+
+
+
+
+ ${jobTracker}
+ ${nameNode}
+ eu.dnetlib.dhp.migration.step1.MigrateMongoMdstoresApplication
+ -p${migrationClaimsPathStep1}/odf_claims
+ -mongourl${mongoURL}
+ -mongodb${mongoDb}
+ -fODF
+ -lstore
+ -iclaim
+
+
+
+
+
+
+
+ ${jobTracker}
+ ${nameNode}
+ eu.dnetlib.dhp.migration.step1.MigrateMongoMdstoresApplication
+ -p${migrationClaimsPathStep1}/oaf_claims
+ -mongourl${mongoURL}
+ -mongodb${mongoDb}
+ -fOAF
+ -lstore
+ -iclaim
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+ ${jobTracker}
+ ${nameNode}
+ yarn-cluster
+ cluster
+ GenerateClaimEntities
+ eu.dnetlib.dhp.migration.step2.GenerateEntitiesApplication
+ dhp-aggregation-${projectVersion}.jar
+ --executor-memory ${sparkExecutorMemory} --executor-cores ${sparkExecutorCores} --driver-memory=${sparkDriverMemory} --conf spark.extraListeners="com.cloudera.spark.lineage.NavigatorAppListener" --conf spark.sql.queryExecutionListeners="com.cloudera.spark.lineage.NavigatorQueryListener" --conf spark.sql.warehouse.dir="/user/hive/warehouse"
+ -mt yarn-cluster
+ -s${migrationClaimsPathStep1}/db_claims,${migrationClaimsPathStep1}/oaf_claims,${migrationClaimsPathStep1}/odf_claims
+ -t${migrationClaimsPathStep2}/claim_entities
+ -pgurl${postgresURL}
+ -pguser${postgresUser}
+ -pgpasswd${postgresPassword}
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+ ${jobTracker}
+ ${nameNode}
+ yarn-cluster
+ cluster
+ GenerateClaimGraph
+ eu.dnetlib.dhp.migration.step3.DispatchEntitiesApplication
+ dhp-aggregation-${projectVersion}.jar
+ --executor-memory ${sparkExecutorMemory} --executor-cores ${sparkExecutorCores} --driver-memory=${sparkDriverMemory} --conf spark.extraListeners="com.cloudera.spark.lineage.NavigatorAppListener" --conf spark.sql.queryExecutionListeners="com.cloudera.spark.lineage.NavigatorQueryListener" --conf spark.sql.warehouse.dir="/user/hive/warehouse"
+ -mt yarn-cluster
+ -s${migrationClaimsPathStep2}/claim_entities
+ -g${migrationClaimsPathStep3}
+
+
+
+
+
+
+
\ No newline at end of file
diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/wfs/regular_all_steps/oozie_app/config-default.xml b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/wfs/regular_all_steps/oozie_app/config-default.xml
new file mode 100644
index 0000000000..2e0ed9aeea
--- /dev/null
+++ b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/wfs/regular_all_steps/oozie_app/config-default.xml
@@ -0,0 +1,18 @@
+
+
+ jobTracker
+ yarnRM
+
+
+ nameNode
+ hdfs://nameservice1
+
+
+ oozie.use.system.libpath
+ true
+
+
+ oozie.action.sharelib.for.spark
+ spark2
+
+
\ No newline at end of file
diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/wfs/regular_all_steps/oozie_app/workflow.xml b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/wfs/regular_all_steps/oozie_app/workflow.xml
new file mode 100644
index 0000000000..42ab59822d
--- /dev/null
+++ b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/wfs/regular_all_steps/oozie_app/workflow.xml
@@ -0,0 +1,197 @@
+
+
+
+
+ workingPath
+ /tmp/dhp_migration
+ the base path to store temporary intermediate data
+
+
+ graphBasePath
+ the target path to store raw graph
+
+
+ reuseContent
+ false
+ should import content from the aggregator or reuse a previous version
+
+
+ postgresURL
+ the postgres URL to access to the database
+
+
+ postgresUser
+ the user postgres
+
+
+ postgresPassword
+ the password postgres
+
+
+ mongoURL
+ mongoDB url, example: mongodb://[username:password@]host[:port]
+
+
+ mongoDb
+ mongo database
+
+
+ sparkDriverMemory
+ memory for driver process
+
+
+ sparkExecutorMemory
+ memory for individual executor
+
+
+ sparkExecutorCores
+ number of cores used by single executor
+
+
+
+
+ ${jobTracker}
+ ${nameNode}
+
+
+ mapreduce.job.queuename
+ ${queueName}
+
+
+ oozie.launcher.mapred.job.queue.name
+ ${oozieLauncherQueueName}
+
+
+
+
+
+
+
+ Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}]
+
+
+
+
+ ${wf:conf('reuseContent') eq false}
+ ${wf:conf('reuseContent') eq true}
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+ eu.dnetlib.dhp.migration.step1.MigrateDbEntitiesApplication
+ -p${workingPath}/db_records
+ -pgurl${postgresURL}
+ -pguser${postgresUser}
+ -pgpasswd${postgresPassword}
+
+
+
+
+
+
+
+ eu.dnetlib.dhp.migration.step1.MigrateMongoMdstoresApplication
+ -p${workingPath}/odf_records
+ -mongourl${mongoURL}
+ -mongodb${mongoDb}
+ -fODF
+ -lstore
+ -icleaned
+
+
+
+
+
+
+
+ eu.dnetlib.dhp.migration.step1.MigrateMongoMdstoresApplication
+ -p${workingPath}/oaf_records
+ -mongourl${mongoURL}
+ -mongodb${mongoDb}
+ -fOAF
+ -lstore
+ -icleaned
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+ yarn
+ cluster
+ GenerateEntities
+ eu.dnetlib.dhp.migration.step2.GenerateEntitiesApplication
+ dhp-aggregation-${projectVersion}.jar
+
+ --executor-memory ${sparkExecutorMemory}
+ --executor-cores ${sparkExecutorCores}
+ --driver-memory=${sparkDriverMemory}
+ --conf spark.extraListeners="com.cloudera.spark.lineage.NavigatorAppListener"
+ --conf spark.sql.queryExecutionListeners="com.cloudera.spark.lineage.NavigatorQueryListener"
+ --conf spark.sql.warehouse.dir="/user/hive/warehouse"
+
+ -mt yarn-cluster
+ -s${workingPath}/db_records,${workingPath}/oaf_records,${workingPath}/odf_records
+ -t${workingPath}/all_entities
+ -pgurl${postgresURL}
+ -pguser${postgresUser}
+ -pgpasswd${postgresPassword}
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+ yarn
+ cluster
+ GenerateGraph
+ eu.dnetlib.dhp.migration.step3.DispatchEntitiesApplication
+ dhp-aggregation-${projectVersion}.jar
+
+ --executor-memory ${sparkExecutorMemory}
+ --executor-cores ${sparkExecutorCores}
+ --driver-memory=${sparkDriverMemory}
+ --conf spark.extraListeners="com.cloudera.spark.lineage.NavigatorAppListener"
+ --conf spark.sql.queryExecutionListeners="com.cloudera.spark.lineage.NavigatorQueryListener"
+ --conf spark.sql.warehouse.dir="/user/hive/warehouse"
+
+ -mt yarn-cluster
+ -s${workingPath}/all_entities
+ -g${graphBasePath}/graph_raw
+
+
+
+
+
+
+
\ No newline at end of file
diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/wfs/regular_step1/oozie_app/config-default.xml b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/wfs/regular_step1/oozie_app/config-default.xml
new file mode 100644
index 0000000000..2e0ed9aeea
--- /dev/null
+++ b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/wfs/regular_step1/oozie_app/config-default.xml
@@ -0,0 +1,18 @@
+
+
+ jobTracker
+ yarnRM
+
+
+ nameNode
+ hdfs://nameservice1
+
+
+ oozie.use.system.libpath
+ true
+
+
+ oozie.action.sharelib.for.spark
+ spark2
+
+
\ No newline at end of file
diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/wfs/regular_step1/oozie_app/workflow.xml b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/wfs/regular_step1/oozie_app/workflow.xml
new file mode 100644
index 0000000000..f16e22f957
--- /dev/null
+++ b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/wfs/regular_step1/oozie_app/workflow.xml
@@ -0,0 +1,103 @@
+
+
+
+ migrationPathStep1
+ the base path to store hdfs file
+
+
+ postgresURL
+ the postgres URL to access to the database
+
+
+ postgresUser
+ the user postgres
+
+
+ postgresPassword
+ the password postgres
+
+
+ mongoURL
+ mongoDB url, example: mongodb://[username:password@]host[:port]
+
+
+ mongoDb
+ mongo database
+
+
+ sparkDriverMemory
+ memory for driver process
+
+
+ sparkExecutorMemory
+ memory for individual executor
+
+
+ sparkExecutorCores
+ number of cores used by single executor
+
+
+
+
+
+
+ Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}]
+
+
+
+
+
+
+
+
+
+
+
+
+
+ ${jobTracker}
+ ${nameNode}
+ eu.dnetlib.dhp.migration.step1.MigrateDbEntitiesApplication
+ -p${migrationPathStep1}/db_records
+ -pgurl${postgresURL}
+ -pguser${postgresUser}
+ -pgpasswd${postgresPassword}
+
+
+
+
+
+
+
+ ${jobTracker}
+ ${nameNode}
+ eu.dnetlib.dhp.migration.step1.MigrateMongoMdstoresApplication
+ -p${migrationPathStep1}/odf_records
+ -mongourl${mongoURL}
+ -mongodb${mongoDb}
+ -fODF
+ -lstore
+ -icleaned
+
+
+
+
+
+
+
+ ${jobTracker}
+ ${nameNode}
+ eu.dnetlib.dhp.migration.step1.MigrateMongoMdstoresApplication
+ -p${migrationPathStep1}/oaf_records
+ -mongourl${mongoURL}
+ -mongodb${mongoDb}
+ -fOAF
+ -lstore
+ -icleaned
+
+
+
+
+
+
+
\ No newline at end of file
diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/wfs/regular_step2/oozie_app/config-default.xml b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/wfs/regular_step2/oozie_app/config-default.xml
new file mode 100644
index 0000000000..2e0ed9aeea
--- /dev/null
+++ b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/wfs/regular_step2/oozie_app/config-default.xml
@@ -0,0 +1,18 @@
+
+
+ jobTracker
+ yarnRM
+
+
+ nameNode
+ hdfs://nameservice1
+
+
+ oozie.use.system.libpath
+ true
+
+
+ oozie.action.sharelib.for.spark
+ spark2
+
+
\ No newline at end of file
diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/wfs/regular_step2/oozie_app/workflow.xml b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/wfs/regular_step2/oozie_app/workflow.xml
new file mode 100644
index 0000000000..cd0a4025e8
--- /dev/null
+++ b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/wfs/regular_step2/oozie_app/workflow.xml
@@ -0,0 +1,74 @@
+
+
+
+ migrationPathStep1
+ the base path to store hdfs file
+
+
+ migrationPathStep2
+ the temporary path to store entities before dispatching
+
+
+ postgresURL
+ the postgres URL to access to the database
+
+
+ postgresUser
+ the user postgres
+
+
+ postgresPassword
+ the password postgres
+
+
+ sparkDriverMemory
+ memory for driver process
+
+
+ sparkExecutorMemory
+ memory for individual executor
+
+
+ sparkExecutorCores
+ number of cores used by single executor
+
+
+
+
+
+
+ Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}]
+
+
+
+
+
+
+
+
+
+
+
+
+
+ ${jobTracker}
+ ${nameNode}
+ yarn-cluster
+ cluster
+ GenerateEntities
+ eu.dnetlib.dhp.migration.step2.GenerateEntitiesApplication
+ dhp-aggregation-${projectVersion}.jar
+ --executor-memory ${sparkExecutorMemory} --executor-cores ${sparkExecutorCores} --driver-memory=${sparkDriverMemory} --conf spark.extraListeners="com.cloudera.spark.lineage.NavigatorAppListener" --conf spark.sql.queryExecutionListeners="com.cloudera.spark.lineage.NavigatorQueryListener" --conf spark.sql.warehouse.dir="/user/hive/warehouse"
+ -mt yarn-cluster
+ -s${migrationPathStep1}/db_records,${migrationPathStep1}/oaf_records,${migrationPathStep1}/odf_records
+ -t${migrationPathStep2}/all_entities
+ -pgurl${postgresURL}
+ -pguser${postgresUser}
+ -pgpasswd${postgresPassword}
+
+
+
+
+
+
+
\ No newline at end of file
diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/wfs/regular_step3/oozie_app/config-default.xml b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/wfs/regular_step3/oozie_app/config-default.xml
new file mode 100644
index 0000000000..2e0ed9aeea
--- /dev/null
+++ b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/wfs/regular_step3/oozie_app/config-default.xml
@@ -0,0 +1,18 @@
+
+
+ jobTracker
+ yarnRM
+
+
+ nameNode
+ hdfs://nameservice1
+
+
+ oozie.use.system.libpath
+ true
+
+
+ oozie.action.sharelib.for.spark
+ spark2
+
+
\ No newline at end of file
diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/wfs/regular_step3/oozie_app/workflow.xml b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/wfs/regular_step3/oozie_app/workflow.xml
new file mode 100644
index 0000000000..8688f09d18
--- /dev/null
+++ b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/migration/wfs/regular_step3/oozie_app/workflow.xml
@@ -0,0 +1,60 @@
+
+
+
+
+ migrationPathStep2
+ the temporary path to store entities before dispatching
+
+
+ migrationPathStep3
+ the graph Raw base path
+
+
+ sparkDriverMemory
+ memory for driver process
+
+
+ sparkExecutorMemory
+ memory for individual executor
+
+
+ sparkExecutorCores
+ number of cores used by single executor
+
+
+
+
+
+
+ Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}]
+
+
+
+
+
+
+
+
+
+
+
+
+
+ ${jobTracker}
+ ${nameNode}
+ yarn-cluster
+ cluster
+ GenerateGraph
+ eu.dnetlib.dhp.migration.step3.DispatchEntitiesApplication
+ dhp-aggregation-${projectVersion}.jar
+ --executor-memory ${sparkExecutorMemory} --executor-cores ${sparkExecutorCores} --driver-memory=${sparkDriverMemory} --conf spark.extraListeners="com.cloudera.spark.lineage.NavigatorAppListener" --conf spark.sql.queryExecutionListeners="com.cloudera.spark.lineage.NavigatorQueryListener" --conf spark.sql.warehouse.dir="/user/hive/warehouse"
+ -mt yarn-cluster
+ -s${migrationPathStep2}/all_entities
+ -g${migrationPathStep3}
+
+
+
+
+
+
+
\ No newline at end of file
diff --git a/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/collection/CollectionJobTest.java b/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/collection/CollectionJobTest.java
index 848fbe17da..b367491e52 100644
--- a/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/collection/CollectionJobTest.java
+++ b/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/collection/CollectionJobTest.java
@@ -1,79 +1,87 @@
package eu.dnetlib.dhp.collection;
-import com.fasterxml.jackson.databind.ObjectMapper;
-import eu.dnetlib.dhp.model.mdstore.MetadataRecord;
-import eu.dnetlib.dhp.model.mdstore.Provenance;
-import org.apache.commons.io.FileUtils;
-import org.apache.commons.io.IOUtils;
-import org.junit.*;
-
import java.io.IOException;
import java.nio.file.Files;
import java.nio.file.Path;
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.io.IOUtils;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+
+import eu.dnetlib.dhp.model.mdstore.MetadataRecord;
+import eu.dnetlib.dhp.model.mdstore.Provenance;
+
public class CollectionJobTest {
- private Path testDir;
- @Before
- public void setup() throws IOException {
- testDir = Files.createTempDirectory("dhp-collection");
- }
+ private Path testDir;
- @After
- public void teadDown() throws IOException {
- FileUtils.deleteDirectory(testDir.toFile());
- }
+ @Before
+ public void setup() throws IOException {
+ testDir = Files.createTempDirectory("dhp-collection");
+ }
- @Test
- public void tesCollection() throws Exception {
- Provenance provenance = new Provenance("pippo", "puppa", "ns_prefix");
- GenerateNativeStoreSparkJob.main(new String[] {
- "-mt", "local",
- "-w", "wid",
- "-e", "XML",
- "-d", ""+System.currentTimeMillis(),
- "-p", new ObjectMapper().writeValueAsString(provenance),
- "-x", "./*[local-name()='record']/*[local-name()='header']/*[local-name()='identifier']",
- "-i", this.getClass().getResource("/eu/dnetlib/dhp/collection/native.seq").toString(),
- "-o", testDir.toString()+"/store",
- "-t", "true",
- "-ru", "",
- "-rp", "",
- "-rh", "",
- "-ro", "",
- "-rr", ""});
- System.out.println(new ObjectMapper().writeValueAsString(provenance));
- }
+ @After
+ public void teadDown() throws IOException {
+ FileUtils.deleteDirectory(testDir.toFile());
+ }
+ @Test
+ public void tesCollection() throws Exception {
+ final Provenance provenance = new Provenance("pippo", "puppa", "ns_prefix");
+ GenerateNativeStoreSparkJob.main(new String[] {
+ "-mt", "local",
+ "-w", "wid",
+ "-e", "XML",
+ "-d", "" + System.currentTimeMillis(),
+ "-p", new ObjectMapper().writeValueAsString(provenance),
+ "-x", "./*[local-name()='record']/*[local-name()='header']/*[local-name()='identifier']",
+ "-i", this.getClass().getResource("/eu/dnetlib/dhp/collection/native.seq").toString(),
+ "-o", testDir.toString() + "/store",
+ "-t", "true",
+ "-ru", "",
+ "-rp", "",
+ "-rh", "",
+ "-ro", "",
+ "-rr", "" });
+ System.out.println(new ObjectMapper().writeValueAsString(provenance));
+ }
+ @Test
+ public void testGenerationMetadataRecord() throws Exception {
- @Test
- public void testGenerationMetadataRecord() throws Exception {
+ final String xml = IOUtils.toString(this.getClass().getResourceAsStream("./record.xml"));
- final String xml = IOUtils.toString(this.getClass().getResourceAsStream("./record.xml"));
+ final MetadataRecord record = GenerateNativeStoreSparkJob
+ .parseRecord(xml, "./*[local-name()='record']/*[local-name()='header']/*[local-name()='identifier']", "XML", new Provenance("foo", "bar",
+ "ns_prefix"), System.currentTimeMillis(), null, null);
- MetadataRecord record = GenerateNativeStoreSparkJob.parseRecord(xml, "./*[local-name()='record']/*[local-name()='header']/*[local-name()='identifier']", "XML", new Provenance("foo", "bar", "ns_prefix"), System.currentTimeMillis(), null,null);
+ assert record != null;
+ System.out.println(record.getId());
+ System.out.println(record.getOriginalId());
- assert record != null;
- System.out.println(record.getId());
- System.out.println(record.getOriginalId());
+ }
+ @Test
+ public void TestEquals() throws IOException {
- }
+ final String xml = IOUtils.toString(this.getClass().getResourceAsStream("./record.xml"));
+ final MetadataRecord record = GenerateNativeStoreSparkJob
+ .parseRecord(xml, "./*[local-name()='record']/*[local-name()='header']/*[local-name()='identifier']", "XML", new Provenance("foo", "bar",
+ "ns_prefix"), System.currentTimeMillis(), null, null);
+ final MetadataRecord record1 = GenerateNativeStoreSparkJob
+ .parseRecord(xml, "./*[local-name()='record']/*[local-name()='header']/*[local-name()='identifier']", "XML", new Provenance("foo", "bar",
+ "ns_prefix"), System.currentTimeMillis(), null, null);
+ assert record != null;
+ record.setBody("ciao");
+ assert record1 != null;
+ record1.setBody("mondo");
+ Assert.assertEquals(record, record1);
-
- @Test
- public void TestEquals () throws IOException {
-
- final String xml = IOUtils.toString(this.getClass().getResourceAsStream("./record.xml"));
- MetadataRecord record = GenerateNativeStoreSparkJob.parseRecord(xml, "./*[local-name()='record']/*[local-name()='header']/*[local-name()='identifier']", "XML", new Provenance("foo", "bar", "ns_prefix"), System.currentTimeMillis(), null,null);
- MetadataRecord record1 = GenerateNativeStoreSparkJob.parseRecord(xml, "./*[local-name()='record']/*[local-name()='header']/*[local-name()='identifier']", "XML", new Provenance("foo", "bar", "ns_prefix"), System.currentTimeMillis(), null,null);
- assert record != null;
- record.setBody("ciao");
- assert record1 != null;
- record1.setBody("mondo");
- Assert.assertEquals(record, record1);
-
- }
+ }
}
diff --git a/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/SparkCreateConnectedComponent.java b/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/SparkCreateConnectedComponent.java
index 16e112c252..3a92a15580 100644
--- a/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/SparkCreateConnectedComponent.java
+++ b/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/SparkCreateConnectedComponent.java
@@ -74,6 +74,6 @@ public class SparkCreateConnectedComponent {
}
public static long getHashcode(final String id) {
- return Hashing.murmur3_128().hashUnencodedChars(id).asLong();
+ return Hashing.murmur3_128().hashString(id).asLong();
}
}
diff --git a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/graph/GraphMappingUtils.java b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/graph/GraphMappingUtils.java
index 7c0967b2e7..0291be47ef 100644
--- a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/graph/GraphMappingUtils.java
+++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/graph/GraphMappingUtils.java
@@ -18,13 +18,13 @@ public class GraphMappingUtils {
public final static Map types = Maps.newHashMap();
static {
- types.put("datasource", Datasource.class);
- types.put("organization", Organization.class);
+ types.put("datasource", Datasource.class);
+ types.put("organization", Organization.class);
types.put("project", Project.class);
- types.put("dataset", Dataset.class);
- types.put("otherresearchproduct", OtherResearchProduct.class);
- types.put("software", Software.class);
- types.put("publication", Publication.class);
+ types.put("dataset", Dataset.class);
+ types.put("otherresearchproduct", OtherResearchProduct.class);
+ types.put("software", Software.class);
+ types.put("publication", Publication.class);
types.put("relation", Relation.class);
}
diff --git a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/graph/SparkGraphImporterJob.java b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/graph/SparkGraphImporterJob.java
index c5223c1f68..95c3cd4800 100644
--- a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/graph/SparkGraphImporterJob.java
+++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/graph/SparkGraphImporterJob.java
@@ -3,7 +3,7 @@ package eu.dnetlib.dhp.graph;
import com.fasterxml.jackson.databind.ObjectMapper;
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
import org.apache.commons.io.IOUtils;
-import org.apache.hadoop.io.Text;
+import org.apache.spark.SparkConf;
import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.sql.Encoders;
import org.apache.spark.sql.SaveMode;
@@ -13,31 +13,40 @@ public class SparkGraphImporterJob {
public static void main(String[] args) throws Exception {
- final ArgumentApplicationParser parser = new ArgumentApplicationParser(IOUtils.toString(SparkGraphImporterJob.class.getResourceAsStream("/eu/dnetlib/dhp/graph/input_graph_parameters.json")));
+ final ArgumentApplicationParser parser = new ArgumentApplicationParser(
+ IOUtils.toString(SparkGraphImporterJob.class.getResourceAsStream(
+ "/eu/dnetlib/dhp/graph/input_graph_parameters.json")));
parser.parseArgument(args);
- final SparkSession spark = SparkSession
+
+ try(SparkSession spark = getSparkSession(parser)) {
+
+ final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext());
+ final String inputPath = parser.get("sourcePath");
+ final String hiveDbName = parser.get("hive_db_name");
+
+ spark.sql(String.format("DROP DATABASE IF EXISTS %s CASCADE", hiveDbName));
+ spark.sql(String.format("CREATE DATABASE IF NOT EXISTS %s", hiveDbName));
+
+ // Read the input file and convert it into RDD of serializable object
+ GraphMappingUtils.types.forEach((name, clazz) -> spark.createDataset(sc.textFile(inputPath + "/" + name)
+ .map(s -> new ObjectMapper().readValue(s, clazz))
+ .rdd(), Encoders.bean(clazz))
+ .write()
+ .mode(SaveMode.Overwrite)
+ .saveAsTable(hiveDbName + "." + name));
+ }
+ }
+
+ private static SparkSession getSparkSession(ArgumentApplicationParser parser) {
+ SparkConf conf = new SparkConf();
+ conf.set("hive.metastore.uris", parser.get("hive_metastore_uris"));
+
+ return SparkSession
.builder()
.appName(SparkGraphImporterJob.class.getSimpleName())
.master(parser.get("master"))
- .config("hive.metastore.uris", parser.get("hive_metastore_uris"))
+ .config(conf)
.enableHiveSupport()
.getOrCreate();
-
- final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext());
- final String inputPath = parser.get("sourcePath");
- final String hiveDbName = parser.get("hive_db_name");
-
- spark.sql(String.format("CREATE DATABASE IF NOT EXISTS %s", hiveDbName));
-
- // Read the input file and convert it into RDD of serializable object
- GraphMappingUtils.types.forEach((name, clazz) -> {
- spark.createDataset(sc.sequenceFile(inputPath + "/" + name, Text.class, Text.class)
- .map(s -> new ObjectMapper().readValue(s._2().toString(), clazz))
- .rdd(), Encoders.bean(clazz))
- .write()
- .mode(SaveMode.Overwrite)
- .saveAsTable(hiveDbName + "." + name);
- });
-
}
}
diff --git a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/graph/oozie_app/lib/scripts/postprocessing.sql b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/graph/oozie_app/lib/scripts/postprocessing.sql
new file mode 100644
index 0000000000..26fcbacf5f
--- /dev/null
+++ b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/graph/oozie_app/lib/scripts/postprocessing.sql
@@ -0,0 +1,8 @@
+CREATE view result as
+ select id, dateofcollection, bestaccessright, datainfo, collectedfrom, pid, author, resulttype, language, country, subject, description, dateofacceptance, embargoenddate, resourcetype, context, instance from ${hive_db_name}.publication p
+ union all
+ select id, dateofcollection, bestaccessright, datainfo, collectedfrom, pid, author, resulttype, language, country, subject, description, dateofacceptance, embargoenddate, resourcetype, context, instance from ${hive_db_name}.dataset d
+ union all
+ select id, dateofcollection, bestaccessright, datainfo, collectedfrom, pid, author, resulttype, language, country, subject, description, dateofacceptance, embargoenddate, resourcetype, context, instance from ${hive_db_name}.software s
+ union all
+ select id, dateofcollection, bestaccessright, datainfo, collectedfrom, pid, author, resulttype, language, country, subject, description, dateofacceptance, embargoenddate, resourcetype, context, instance from ${hive_db_name}.otherresearchproduct o;
diff --git a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/graph/oozie_app/workflow.xml b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/graph/oozie_app/workflow.xml
index 24090a2458..481cc70b40 100644
--- a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/graph/oozie_app/workflow.xml
+++ b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/graph/oozie_app/workflow.xml
@@ -1,4 +1,5 @@
+
sourcePath
@@ -22,6 +23,21 @@
+
+ ${jobTracker}
+ ${nameNode}
+
+
+ mapreduce.job.queuename
+ ${queueName}
+
+
+ oozie.launcher.mapred.job.queue.name
+ ${oozieLauncherQueueName}
+
+
+
+
@@ -30,19 +46,39 @@
- ${jobTracker}
- ${nameNode}
- yarn-cluster
+ yarn
cluster
MapGraphIntoDataFrame
eu.dnetlib.dhp.graph.SparkGraphImporterJob
dhp-graph-mapper-${projectVersion}.jar
- --executor-memory ${sparkExecutorMemory} --executor-cores ${sparkExecutorCores} --driver-memory=${sparkDriverMemory} --conf spark.extraListeners="com.cloudera.spark.lineage.NavigatorAppListener" --conf spark.sql.queryExecutionListeners="com.cloudera.spark.lineage.NavigatorQueryListener" --conf spark.sql.warehouse.dir="/user/hive/warehouse"
+
+ --executor-memory ${sparkExecutorMemory}
+ --executor-cores ${sparkExecutorCores}
+ --driver-memory=${sparkDriverMemory}
+ --conf spark.extraListeners="com.cloudera.spark.lineage.NavigatorAppListener"
+ --conf spark.sql.queryExecutionListeners="com.cloudera.spark.lineage.NavigatorQueryListener"
+ --conf spark.sql.warehouse.dir="/user/hive/warehouse"
+
-mt yarn-cluster
--sourcePath${sourcePath}
--hive_db_name${hive_db_name}
--hive_metastore_uris${hive_metastore_uris}
+
+
+
+
+
+
+
+
+ oozie.hive.defaults
+ hive-site.xml
+
+
+
+ hive_db_name=${hive_db_name}
+
diff --git a/dhp-workflows/dhp-graph-provision/job-override.properties b/dhp-workflows/dhp-graph-provision/job-override.properties
index b5ab079825..68816c2240 100644
--- a/dhp-workflows/dhp-graph-provision/job-override.properties
+++ b/dhp-workflows/dhp-graph-provision/job-override.properties
@@ -1,10 +1,12 @@
-sparkDriverMemory=8G
-sparkExecutorMemory=8G
+sparkDriverMemory=10G
+sparkExecutorMemory=15G
#isLookupUrl=http://services.openaire.eu:8280/is/services/isLookUp
isLookupUrl=http://beta.services.openaire.eu:8280/is/services/isLookUp?wsdl
sourcePath=/tmp/db_openaireplus_services.export_dhp.2020.02.03
outputPath=/tmp/openaire_provision
format=TMF
batchSize=2000
+sparkExecutorCoresForJoining=128
sparkExecutorCoresForIndexing=64
-reuseRecords=true
\ No newline at end of file
+reuseRecords=false
+otherDsTypeId=scholarcomminfra, infospace, pubsrepository::mock, entityregistry, entityregistry::projects, entityregistry::repositories, websource
\ No newline at end of file
diff --git a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/graph/GraphJoiner.java b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/graph/GraphJoiner.java
index 062c8886b8..d260e05512 100644
--- a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/graph/GraphJoiner.java
+++ b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/graph/GraphJoiner.java
@@ -1,31 +1,32 @@
package eu.dnetlib.dhp.graph;
-import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.databind.DeserializationFeature;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.collect.Iterables;
-import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
import com.jayway.jsonpath.DocumentContext;
import com.jayway.jsonpath.JsonPath;
import eu.dnetlib.dhp.graph.model.*;
import eu.dnetlib.dhp.graph.utils.ContextMapper;
import eu.dnetlib.dhp.graph.utils.GraphMappingUtils;
+import eu.dnetlib.dhp.graph.utils.RelationPartitioner;
import eu.dnetlib.dhp.graph.utils.XmlRecordFactory;
import eu.dnetlib.dhp.schema.oaf.*;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.io.compress.GzipCodec;
import org.apache.hadoop.mapred.SequenceFileOutputFormat;
+import org.apache.spark.SparkContext;
import org.apache.spark.api.java.JavaPairRDD;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.api.java.function.PairFunction;
import org.apache.spark.sql.SparkSession;
+import org.apache.spark.util.LongAccumulator;
import scala.Tuple2;
import java.io.IOException;
import java.io.Serializable;
-import java.util.HashSet;
-import java.util.List;
-import java.util.stream.Collectors;
+import java.util.Map;
import static eu.dnetlib.dhp.graph.utils.GraphMappingUtils.asRelatedEntity;
@@ -45,10 +46,12 @@ import static eu.dnetlib.dhp.graph.utils.GraphMappingUtils.asRelatedEntity;
* 3) we only need a subset of fields from the related entities, so we introduce a distinction between E_source = S
* and E_target = T. Objects in T are heavily pruned by all the unnecessary information
*
- * 4) perform the join as (((T join R) union S) groupby S.id) yield S -> [ ]
+ * 4) perform the join as (((T.id join R.target) union S) groupby S.id) yield S -> [ ]
*/
public class GraphJoiner implements Serializable {
+ private Map accumulators = Maps.newHashMap();
+
public static final int MAX_RELS = 100;
public static final String schemaLocation = "https://www.openaire.eu/schema/1.0/oaf-1.0.xsd";
@@ -61,24 +64,30 @@ public class GraphJoiner implements Serializable {
private String outPath;
- public GraphJoiner(SparkSession spark, ContextMapper contextMapper, String inputPath, String outPath) {
+ private String otherDsTypeId;
+
+ public GraphJoiner(SparkSession spark, ContextMapper contextMapper, String otherDsTypeId, String inputPath, String outPath) {
this.spark = spark;
this.contextMapper = contextMapper;
+ this.otherDsTypeId = otherDsTypeId;
this.inputPath = inputPath;
this.outPath = outPath;
+
+ final SparkContext sc = spark.sparkContext();
+ prepareAccumulators(sc);
}
public GraphJoiner adjacencyLists() {
- final JavaSparkContext sc = new JavaSparkContext(getSpark().sparkContext());
+ final JavaSparkContext jsc = new JavaSparkContext(getSpark().sparkContext());
// read each entity
- JavaPairRDD datasource = readPathEntity(sc, getInputPath(), "datasource");
- JavaPairRDD organization = readPathEntity(sc, getInputPath(), "organization");
- JavaPairRDD project = readPathEntity(sc, getInputPath(), "project");
- JavaPairRDD dataset = readPathEntity(sc, getInputPath(), "dataset");
- JavaPairRDD otherresearchproduct = readPathEntity(sc, getInputPath(), "otherresearchproduct");
- JavaPairRDD software = readPathEntity(sc, getInputPath(), "software");
- JavaPairRDD publication = readPathEntity(sc, getInputPath(), "publication");
+ JavaPairRDD datasource = readPathEntity(jsc, getInputPath(), "datasource");
+ JavaPairRDD organization = readPathEntity(jsc, getInputPath(), "organization");
+ JavaPairRDD project = readPathEntity(jsc, getInputPath(), "project");
+ JavaPairRDD dataset = readPathEntity(jsc, getInputPath(), "dataset");
+ JavaPairRDD otherresearchproduct = readPathEntity(jsc, getInputPath(), "otherresearchproduct");
+ JavaPairRDD software = readPathEntity(jsc, getInputPath(), "software");
+ JavaPairRDD publication = readPathEntity(jsc, getInputPath(), "publication");
// create the union between all the entities
final String entitiesPath = getOutPath() + "/entities";
@@ -93,31 +102,43 @@ public class GraphJoiner implements Serializable {
.map(GraphMappingUtils::serialize)
.saveAsTextFile(entitiesPath, GzipCodec.class);
- JavaPairRDD entities = sc.textFile(entitiesPath)
+ JavaPairRDD entities = jsc.textFile(entitiesPath)
.map(t -> new ObjectMapper().readValue(t, EntityRelEntity.class))
.mapToPair(t -> new Tuple2<>(t.getSource().getSourceId(), t));
+ final String relationPath = getOutPath() + "/relation";
// reads the relationships
- final JavaPairRDD relation = readPathRelation(sc, getInputPath())
- .filter(r -> !r.getDeleted()) //only consider those that are not virtually deleted
+ final JavaPairRDD rels = readPathRelation(jsc, getInputPath())
+ .filter(rel -> !rel.getDeleted()) //only consider those that are not virtually deleted
.map(p -> new EntityRelEntity().setRelation(p))
- .mapToPair(p -> new Tuple2<>(p.getRelation().getSourceId(), p))
- .groupByKey()
+ .mapToPair(p -> new Tuple2<>(SortableRelationKey.from(p), p));
+ rels
+ .groupByKey(new RelationPartitioner(rels.getNumPartitions()))
.map(p -> Iterables.limit(p._2(), MAX_RELS))
.flatMap(p -> p.iterator())
+ .map(s -> new ObjectMapper().writeValueAsString(s))
+ .saveAsTextFile(relationPath, GzipCodec.class);
+
+ final JavaPairRDD relation = jsc.textFile(relationPath)
+ .map(s -> new ObjectMapper().readValue(s, EntityRelEntity.class))
.mapToPair(p -> new Tuple2<>(p.getRelation().getTargetId(), p));
- //final String bySource = getOutPath() + "/1_join_by_target";
- JavaPairRDD bySource = relation
+ final String bySourcePath = getOutPath() + "/join_by_source";
+ relation
.join(entities
.filter(e -> !e._2().getSource().getDeleted())
.mapToPair(e -> new Tuple2<>(e._1(), asRelatedEntity(e._2()))))
.map(s -> new EntityRelEntity()
.setRelation(s._2()._1().getRelation())
.setTarget(s._2()._2().getSource()))
+ .map(j -> new ObjectMapper().writeValueAsString(j))
+ .saveAsTextFile(bySourcePath, GzipCodec.class);
+
+ JavaPairRDD bySource = jsc.textFile(bySourcePath)
+ .map(e -> getObjectMapper().readValue(e, EntityRelEntity.class))
.mapToPair(t -> new Tuple2<>(t.getRelation().getSourceId(), t));
- final XmlRecordFactory recordFactory = new XmlRecordFactory(contextMapper, false, schemaLocation, new HashSet<>());
+ final XmlRecordFactory recordFactory = new XmlRecordFactory(accumulators, contextMapper, false, schemaLocation, otherDsTypeId);
entities
.union(bySource)
.groupByKey() // by source id
@@ -130,20 +151,6 @@ public class GraphJoiner implements Serializable {
return this;
}
- public GraphJoiner asXML() {
- final JavaSparkContext sc = new JavaSparkContext(getSpark().sparkContext());
- final XmlRecordFactory recordFactory = new XmlRecordFactory(contextMapper, true, "", new HashSet<>());
- final ObjectMapper mapper = new ObjectMapper();
-
- final String joinedEntitiesPath = getOutPath() + "/1_joined_entities";
- sc.textFile(joinedEntitiesPath)
- .map(s -> mapper.readValue(s, JoinedEntity.class))
- .mapToPair(je -> new Tuple2<>(new Text(je.getEntity().getId()), new Text(recordFactory.build(je))))
- .saveAsHadoopFile(getOutPath() + "/2_xml", Text.class, Text.class, SequenceFileOutputFormat.class, GzipCodec.class);
-
- return this;
- }
-
public SparkSession getSpark() {
return spark;
}
@@ -158,24 +165,23 @@ public class GraphJoiner implements Serializable {
// HELPERS
- private OafEntity parseOaf(final String json, final String type) {
- final ObjectMapper o = new ObjectMapper();
+ private OafEntity parseOaf(final String json, final String type, final ObjectMapper mapper) {
try {
switch (GraphMappingUtils.EntityType.valueOf(type)) {
case publication:
- return o.readValue(json, Publication.class);
+ return mapper.readValue(json, Publication.class);
case dataset:
- return o.readValue(json, Dataset.class);
+ return mapper.readValue(json, Dataset.class);
case otherresearchproduct:
- return o.readValue(json, OtherResearchProduct.class);
+ return mapper.readValue(json, OtherResearchProduct.class);
case software:
- return o.readValue(json, Software.class);
+ return mapper.readValue(json, Software.class);
case datasource:
- return o.readValue(json, Datasource.class);
+ return mapper.readValue(json, Datasource.class);
case organization:
- return o.readValue(json, Organization.class);
+ return mapper.readValue(json, Organization.class);
case project:
- return o.readValue(json, Project.class);
+ return mapper.readValue(json, Project.class);
default:
throw new IllegalArgumentException("invalid type: " + type);
}
@@ -185,26 +191,26 @@ public class GraphJoiner implements Serializable {
}
private JoinedEntity toJoinedEntity(Tuple2> p) {
- final ObjectMapper o = new ObjectMapper();
+ final ObjectMapper mapper = getObjectMapper();
final JoinedEntity j = new JoinedEntity();
- final Links links2 = new Links();
+ final Links links = new Links();
for(EntityRelEntity rel : p._2()) {
if (rel.hasMainEntity() & j.getEntity() == null) {
j.setType(rel.getSource().getType());
- j.setEntity(parseOaf(rel.getSource().getOaf(), rel.getSource().getType()));
+ j.setEntity(parseOaf(rel.getSource().getOaf(), rel.getSource().getType(), mapper));
}
if (rel.hasRelatedEntity()) {
try {
- links2.add(
+ links.add(
new eu.dnetlib.dhp.graph.model.Tuple2()
- .setRelation(o.readValue(rel.getRelation().getOaf(), Relation.class))
- .setRelatedEntity(o.readValue(rel.getTarget().getOaf(), RelatedEntity.class)));
+ .setRelation(mapper.readValue(rel.getRelation().getOaf(), Relation.class))
+ .setRelatedEntity(mapper.readValue(rel.getTarget().getOaf(), RelatedEntity.class)));
} catch (IOException e) {
throw new IllegalArgumentException(e);
}
}
}
- j.setLinks(links2);
+ j.setLinks(links);
if (j.getEntity() == null) {
throw new IllegalStateException("missing main entity on '" + p._1() + "'");
}
@@ -220,9 +226,8 @@ public class GraphJoiner implements Serializable {
* @return the JavaPairRDD indexed by entity identifier
*/
private JavaPairRDD readPathEntity(final JavaSparkContext sc, final String inputPath, final String type) {
- return sc.sequenceFile(inputPath + "/" + type, Text.class, Text.class)
- .mapToPair((PairFunction, String, TypedRow>) item -> {
- final String s = item._2().toString();
+ return sc.textFile(inputPath + "/" + type)
+ .mapToPair((PairFunction) s -> {
final DocumentContext json = JsonPath.parse(s);
final String id = json.read("$.id");
return new Tuple2<>(id, new TypedRow()
@@ -241,17 +246,46 @@ public class GraphJoiner implements Serializable {
* @return the JavaRDD containing all the relationships
*/
private JavaRDD readPathRelation(final JavaSparkContext sc, final String inputPath) {
- return sc.sequenceFile(inputPath + "/relation", Text.class, Text.class)
- .map(item -> {
- final String s = item._2().toString();
+ return sc.textFile(inputPath + "/relation")
+ .map(s -> {
final DocumentContext json = JsonPath.parse(s);
return new TypedRow()
.setSourceId(json.read("$.source"))
.setTargetId(json.read("$.target"))
.setDeleted(json.read("$.dataInfo.deletedbyinference"))
.setType("relation")
+ .setRelType("$.relType")
+ .setSubRelType("$.subRelType")
+ .setRelClass("$.relClass")
.setOaf(s);
});
}
+ private ObjectMapper getObjectMapper() {
+ return new ObjectMapper().configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
+ }
+
+ private void prepareAccumulators(SparkContext sc) {
+ accumulators.put("resultResult_similarity_isAmongTopNSimilarDocuments", sc.longAccumulator("resultResult_similarity_isAmongTopNSimilarDocuments"));
+ accumulators.put("resultResult_similarity_hasAmongTopNSimilarDocuments", sc.longAccumulator("resultResult_similarity_hasAmongTopNSimilarDocuments"));
+ accumulators.put("resultResult_supplement_isSupplementTo", sc.longAccumulator("resultResult_supplement_isSupplementTo"));
+ accumulators.put("resultResult_supplement_isSupplementedBy", sc.longAccumulator("resultResult_supplement_isSupplementedBy"));
+ accumulators.put("resultResult_dedup_isMergedIn", sc.longAccumulator("resultResult_dedup_isMergedIn"));
+ accumulators.put("resultResult_dedup_merges", sc.longAccumulator("resultResult_dedup_merges"));
+
+ accumulators.put("resultResult_publicationDataset_isRelatedTo", sc.longAccumulator("resultResult_publicationDataset_isRelatedTo"));
+ accumulators.put("resultResult_relationship_isRelatedTo", sc.longAccumulator("resultResult_relationship_isRelatedTo"));
+ accumulators.put("resultProject_outcome_isProducedBy", sc.longAccumulator("resultProject_outcome_isProducedBy"));
+ accumulators.put("resultProject_outcome_produces", sc.longAccumulator("resultProject_outcome_produces"));
+ accumulators.put("resultOrganization_affiliation_isAuthorInstitutionOf", sc.longAccumulator("resultOrganization_affiliation_isAuthorInstitutionOf"));
+
+ accumulators.put("resultOrganization_affiliation_hasAuthorInstitution", sc.longAccumulator("resultOrganization_affiliation_hasAuthorInstitution"));
+ accumulators.put("projectOrganization_participation_hasParticipant", sc.longAccumulator("projectOrganization_participation_hasParticipant"));
+ accumulators.put("projectOrganization_participation_isParticipant", sc.longAccumulator("projectOrganization_participation_isParticipant"));
+ accumulators.put("organizationOrganization_dedup_isMergedIn", sc.longAccumulator("organizationOrganization_dedup_isMergedIn"));
+ accumulators.put("organizationOrganization_dedup_merges", sc.longAccumulator("resultProject_outcome_produces"));
+ accumulators.put("datasourceOrganization_provision_isProvidedBy", sc.longAccumulator("datasourceOrganization_provision_isProvidedBy"));
+ accumulators.put("datasourceOrganization_provision_provides", sc.longAccumulator("datasourceOrganization_provision_provides"));
+ }
+
}
diff --git a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/graph/SparkXmlRecordBuilderJob.java b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/graph/SparkXmlRecordBuilderJob.java
index 0b2180f190..5fa3e63850 100644
--- a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/graph/SparkXmlRecordBuilderJob.java
+++ b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/graph/SparkXmlRecordBuilderJob.java
@@ -24,6 +24,7 @@ public class SparkXmlRecordBuilderJob {
final String inputPath = parser.get("sourcePath");
final String outputPath = parser.get("outputPath");
final String isLookupUrl = parser.get("isLookupUrl");
+ final String otherDsTypeId = parser.get("otherDsTypeId");
final FileSystem fs = FileSystem.get(spark.sparkContext().hadoopConfiguration());
if (fs.exists(new Path(outputPath))) {
@@ -31,8 +32,9 @@ public class SparkXmlRecordBuilderJob {
fs.mkdirs(new Path(outputPath));
}
- new GraphJoiner(spark, ContextMapper.fromIS(isLookupUrl), inputPath, outputPath)
+ new GraphJoiner(spark, ContextMapper.fromIS(isLookupUrl), otherDsTypeId, inputPath, outputPath)
.adjacencyLists();
+ //.asXML();
}
}
diff --git a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/graph/model/SortableRelationKey.java b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/graph/model/SortableRelationKey.java
new file mode 100644
index 0000000000..6bfbab5471
--- /dev/null
+++ b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/graph/model/SortableRelationKey.java
@@ -0,0 +1,99 @@
+package eu.dnetlib.dhp.graph.model;
+
+import com.google.common.collect.ComparisonChain;
+import com.google.common.collect.Maps;
+
+import java.io.Serializable;
+import java.util.Map;
+
+/**
+ * Allows to sort relationships according to the priority defined in weights map.
+ */
+public class SortableRelationKey implements Comparable, Serializable {
+
+ private String sourceId;
+ private String targetId;
+
+ private String relType;
+ private String subRelType;
+ private String relClass;
+
+ private final static Map weights = Maps.newHashMap();
+
+ static {
+ weights.put("outcome", 0);
+ weights.put("supplement", 1);
+ weights.put("publicationDataset", 2);
+ weights.put("relationship", 3);
+ weights.put("similarity", 4);
+ weights.put("affiliation", 5);
+
+ weights.put("provision", 6);
+ weights.put("participation", 7);
+ weights.put("dedup", 8);
+ }
+
+ public static SortableRelationKey from(final EntityRelEntity e) {
+ return new SortableRelationKey()
+ .setSourceId(e.getRelation().getSourceId())
+ .setTargetId(e.getRelation().getTargetId())
+ .setRelType(e.getRelation().getRelType())
+ .setSubRelType(e.getRelation().getSubRelType())
+ .setRelClass(e.getRelation().getRelClass());
+ }
+
+ public String getSourceId() {
+ return sourceId;
+ }
+
+ public SortableRelationKey setSourceId(String sourceId) {
+ this.sourceId = sourceId;
+ return this;
+ }
+
+ public String getTargetId() {
+ return targetId;
+ }
+
+ public SortableRelationKey setTargetId(String targetId) {
+ this.targetId = targetId;
+ return this;
+ }
+
+ public String getRelType() {
+ return relType;
+ }
+
+ public SortableRelationKey setRelType(String relType) {
+ this.relType = relType;
+ return this;
+ }
+
+ public String getSubRelType() {
+ return subRelType;
+ }
+
+ public SortableRelationKey setSubRelType(String subRelType) {
+ this.subRelType = subRelType;
+ return this;
+ }
+
+ public String getRelClass() {
+ return relClass;
+ }
+
+ public SortableRelationKey setRelClass(String relClass) {
+ this.relClass = relClass;
+ return this;
+ }
+
+ @Override
+ public int compareTo(SortableRelationKey o) {
+ return ComparisonChain.start()
+ .compare(weights.get(getSubRelType()), weights.get(o.getSubRelType()))
+ .compare(getSourceId(), o.getSourceId())
+ .compare(getTargetId(), o.getTargetId())
+ .result();
+ }
+
+}
diff --git a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/graph/model/TypedRow.java b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/graph/model/TypedRow.java
index 3651e28c9d..8205c38ef7 100644
--- a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/graph/model/TypedRow.java
+++ b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/graph/model/TypedRow.java
@@ -12,6 +12,10 @@ public class TypedRow implements Serializable {
private String type;
+ private String relType;
+ private String subRelType;
+ private String relClass;
+
private String oaf;
public String getSourceId() {
@@ -50,6 +54,33 @@ public class TypedRow implements Serializable {
return this;
}
+ public String getRelType() {
+ return relType;
+ }
+
+ public TypedRow setRelType(String relType) {
+ this.relType = relType;
+ return this;
+ }
+
+ public String getSubRelType() {
+ return subRelType;
+ }
+
+ public TypedRow setSubRelType(String subRelType) {
+ this.subRelType = subRelType;
+ return this;
+ }
+
+ public String getRelClass() {
+ return relClass;
+ }
+
+ public TypedRow setRelClass(String relClass) {
+ this.relClass = relClass;
+ return this;
+ }
+
public String getOaf() {
return oaf;
}
diff --git a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/graph/utils/GraphMappingUtils.java b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/graph/utils/GraphMappingUtils.java
index 0921fe1050..3d8cde703e 100644
--- a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/graph/utils/GraphMappingUtils.java
+++ b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/graph/utils/GraphMappingUtils.java
@@ -26,6 +26,8 @@ import static org.apache.commons.lang3.StringUtils.*;
public class GraphMappingUtils {
+ public static final String SEPARATOR = "_";
+
public enum EntityType {
publication, dataset, otherresearchproduct, software, datasource, organization, project
}
@@ -38,34 +40,6 @@ public class GraphMappingUtils {
public static Set instanceFieldFilter = Sets.newHashSet("instancetype", "hostedby", "license", "accessright", "collectedfrom", "dateofacceptance", "distributionlocation");
- private static BiMap relClassMapping = HashBiMap.create();
-
- static {
- relClassMapping.put("isAuthorInstitutionOf", "hasAuthorInstitution");
- relClassMapping.put("isMergedIn", "merges");
- relClassMapping.put("isProducedBy", "produces");
- relClassMapping.put("hasParticipant", "isParticipant");
- relClassMapping.put("isProvidedBy", "provides");
- relClassMapping.put("isRelatedTo", "isRelatedTo");
- relClassMapping.put("isAmongTopNSimilarDocuments", "hasAmongTopNSimilarDocuments");
- relClassMapping.put("isRelatedTo", "isRelatedTo");
- relClassMapping.put("isSupplementTo", "isSupplementedBy");
- }
-
- public static String getInverseRelClass(final String relClass) {
- String res = relClassMapping.get(relClass);
- if (isNotBlank(res)) {
- return res;
- }
- res = relClassMapping.inverse().get(relClass);
-
- if (isNotBlank(res)) {
- return res;
- }
-
- throw new IllegalArgumentException("unable to find an inverse relationship class for term: " + relClass);
- }
-
private static final String schemeTemplate = "dnet:%s_%s_relations";
private static Map entityMapping = Maps.newHashMap();
@@ -158,7 +132,7 @@ public class GraphMappingUtils {
re.setLegalname(j.read("$.legalname.value"));
re.setLegalshortname(j.read("$.legalshortname.value"));
re.setCountry(asQualifier(j.read("$.country")));
-
+ re.setWebsiteurl(j.read("$.websiteurl.value"));
break;
case project:
re.setProjectTitle(j.read("$.title.value"));
@@ -250,5 +224,8 @@ public class GraphMappingUtils {
return s;
}
+ public static String getRelDescriptor(String relType, String subRelType, String relClass) {
+ return relType + SEPARATOR + subRelType + SEPARATOR + relClass;
+ }
}
diff --git a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/graph/utils/RelationPartitioner.java b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/graph/utils/RelationPartitioner.java
new file mode 100644
index 0000000000..f4b1514d0e
--- /dev/null
+++ b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/graph/utils/RelationPartitioner.java
@@ -0,0 +1,29 @@
+package eu.dnetlib.dhp.graph.utils;
+
+import eu.dnetlib.dhp.graph.model.SortableRelationKey;
+import org.apache.spark.Partitioner;
+import org.apache.spark.util.Utils;
+
+/**
+ * Used in combination with SortableRelationKey, allows to partition the records by source id, therefore
+ * allowing to sort relations sharing the same source id by the ordering defined in SortableRelationKey.
+ */
+public class RelationPartitioner extends Partitioner {
+
+ private int numPartitions;
+
+ public RelationPartitioner(int numPartitions) {
+ this.numPartitions = numPartitions;
+ }
+
+ @Override
+ public int numPartitions() {
+ return numPartitions;
+ }
+
+ @Override
+ public int getPartition(Object key) {
+ return Utils.nonNegativeMod(((SortableRelationKey) key).getSourceId().hashCode(), numPartitions());
+ }
+
+}
diff --git a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/graph/utils/XmlRecordFactory.java b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/graph/utils/XmlRecordFactory.java
index df34b08d35..74e36a818c 100644
--- a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/graph/utils/XmlRecordFactory.java
+++ b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/graph/utils/XmlRecordFactory.java
@@ -3,6 +3,7 @@ package eu.dnetlib.dhp.graph.utils;
import com.google.common.base.Joiner;
import com.google.common.base.Splitter;
import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
import com.google.common.collect.Sets;
import com.mycila.xmltool.XMLDoc;
import com.mycila.xmltool.XMLTag;
@@ -11,6 +12,8 @@ import eu.dnetlib.dhp.graph.model.RelatedEntity;
import eu.dnetlib.dhp.graph.model.Tuple2;
import eu.dnetlib.dhp.schema.oaf.Result;
import eu.dnetlib.dhp.schema.oaf.*;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.spark.util.LongAccumulator;
import org.dom4j.Document;
import org.dom4j.DocumentException;
import org.dom4j.Element;
@@ -27,6 +30,7 @@ import java.io.Serializable;
import java.io.StringReader;
import java.io.StringWriter;
import java.util.List;
+import java.util.Map;
import java.util.Set;
import java.util.stream.Collectors;
@@ -37,6 +41,8 @@ import static org.apache.commons.lang3.StringUtils.substringBefore;
public class XmlRecordFactory implements Serializable {
+ private Map accumulators;
+
private Set specialDatasourceTypes;
private ContextMapper contextMapper;
@@ -47,11 +53,20 @@ public class XmlRecordFactory implements Serializable {
public XmlRecordFactory(
final ContextMapper contextMapper, final boolean indent,
- final String schemaLocation, final Set otherDatasourceTypesUForUI) {
+ final String schemaLocation, final String otherDatasourceTypesUForUI) {
+ this(Maps.newHashMap(), contextMapper, indent, schemaLocation, otherDatasourceTypesUForUI);
+ }
+
+ public XmlRecordFactory(
+ final Map accumulators,
+ final ContextMapper contextMapper, final boolean indent,
+ final String schemaLocation, final String otherDatasourceTypesUForUI) {
+
+ this.accumulators = accumulators;
this.contextMapper = contextMapper;
this.schemaLocation = schemaLocation;
- this.specialDatasourceTypes = otherDatasourceTypesUForUI;
+ this.specialDatasourceTypes = Sets.newHashSet(Splitter.on(",").trimResults().split(otherDatasourceTypesUForUI));
this.indent = indent;
}
@@ -448,7 +463,7 @@ public class XmlRecordFactory implements Serializable {
if (ds.getSubjects() != null) {
metadata.addAll(ds.getSubjects()
.stream()
- .map(sp -> mapStructuredProperty("subject", sp))
+ .map(sp -> mapStructuredProperty("subjects", sp))
.collect(Collectors.toList()));
}
@@ -583,7 +598,7 @@ public class XmlRecordFactory implements Serializable {
if (p.getFundingtree() != null) {
metadata.addAll(p.getFundingtree()
.stream()
- .map(ft -> asXmlElement("fundingtree", ft.getValue()))
+ .map(ft -> ft.getValue())
.collect(Collectors.toList()));
}
@@ -712,13 +727,23 @@ public class XmlRecordFactory implements Serializable {
}
final DataInfo info = rel.getDataInfo();
+ final String scheme = getScheme(re.getType(), targetType);
+
+ if (StringUtils.isBlank(scheme)) {
+ throw new IllegalArgumentException(String.format("missing scheme for: <%s - %s>", re.getType(), targetType));
+ }
+
+ final String accumulatorName = getRelDescriptor(rel.getRelType(), rel.getSubRelType(), rel.getRelClass());
+ if (accumulators.containsKey(accumulatorName)) {
+ accumulators.get(accumulatorName).add(1);
+ }
rels.add(templateFactory.getRel(
targetType,
rel.getTarget(),
Sets.newHashSet(metadata),
- getInverseRelClass(rel.getRelClass()),
- getScheme(targetType, re.getType()),
+ rel.getRelClass(),
+ scheme,
info));
}
return rels;
@@ -931,7 +956,7 @@ public class XmlRecordFactory implements Serializable {
@SuppressWarnings("unchecked")
- private String getRelFundingTree(final String xmlTree) {
+ protected static String getRelFundingTree(final String xmlTree) {
String funding = "";
try {
final Document ftree = new SAXReader().read(new StringReader(xmlTree));
@@ -952,11 +977,11 @@ public class XmlRecordFactory implements Serializable {
return funding;
}
- private String getFunderElement(final Document ftree) {
- final String funderId = ftree.valueOf("//fundingtree/funder/id/text()");
- final String funderShortName = ftree.valueOf("//fundingtree/funder/shortname/text()");
- final String funderName = ftree.valueOf("//fundingtree/funder/name/text()");
- final String funderJurisdiction = ftree.valueOf("//fundingtree/funder/jurisdiction/text()");
+ private static String getFunderElement(final Document ftree) {
+ final String funderId = ftree.valueOf("//fundingtree/funder/id");
+ final String funderShortName = ftree.valueOf("//fundingtree/funder/shortname");
+ final String funderName = ftree.valueOf("//fundingtree/funder/name");
+ final String funderJurisdiction = ftree.valueOf("//fundingtree/funder/jurisdiction");
return "";
diff --git a/dhp-workflows/dhp-graph-provision/src/main/resources/eu/dnetlib/dhp/graph/input_params_build_adjacency_lists.json b/dhp-workflows/dhp-graph-provision/src/main/resources/eu/dnetlib/dhp/graph/input_params_build_adjacency_lists.json
index e63322028c..a5d20a55f4 100644
--- a/dhp-workflows/dhp-graph-provision/src/main/resources/eu/dnetlib/dhp/graph/input_params_build_adjacency_lists.json
+++ b/dhp-workflows/dhp-graph-provision/src/main/resources/eu/dnetlib/dhp/graph/input_params_build_adjacency_lists.json
@@ -1,6 +1,7 @@
[
- {"paramName":"mt", "paramLongName":"master", "paramDescription": "should be local or yarn", "paramRequired": true},
- {"paramName":"is", "paramLongName":"isLookupUrl", "paramDescription": "URL of the isLookUp Service", "paramRequired": true},
- {"paramName":"o", "paramLongName":"outputPath", "paramDescription": "the path used to store temporary output files", "paramRequired": true},
- {"paramName":"s", "paramLongName":"sourcePath", "paramDescription": "the path of the sequence file to read", "paramRequired": true}
+ {"paramName":"mt", "paramLongName":"master", "paramDescription": "should be local or yarn", "paramRequired": true},
+ {"paramName":"is", "paramLongName":"isLookupUrl", "paramDescription": "URL of the isLookUp Service", "paramRequired": true},
+ {"paramName":"o", "paramLongName":"outputPath", "paramDescription": "the path used to store temporary output files", "paramRequired": true},
+ {"paramName":"s", "paramLongName":"sourcePath", "paramDescription": "the path of the sequence file to read", "paramRequired": true},
+ {"paramName":"t", "paramLongName":"otherDsTypeId", "paramDescription": "list of datasource types to populate field datasourcetypeui", "paramRequired": true}
]
\ No newline at end of file
diff --git a/dhp-workflows/dhp-graph-provision/src/main/resources/eu/dnetlib/dhp/graph/oozie_app/workflow.xml b/dhp-workflows/dhp-graph-provision/src/main/resources/eu/dnetlib/dhp/graph/oozie_app/workflow.xml
index 3503589441..b154b61e11 100644
--- a/dhp-workflows/dhp-graph-provision/src/main/resources/eu/dnetlib/dhp/graph/oozie_app/workflow.xml
+++ b/dhp-workflows/dhp-graph-provision/src/main/resources/eu/dnetlib/dhp/graph/oozie_app/workflow.xml
@@ -1,4 +1,5 @@
+
hive_db_name
@@ -26,6 +27,21 @@
+
+ ${jobTracker}
+ ${nameNode}
+
+
+ mapreduce.job.queuename
+ ${queueName}
+
+
+ oozie.launcher.mapred.job.queue.name
+ ${oozieLauncherQueueName}
+
+
+
+
@@ -42,17 +58,16 @@
- ${jobTracker}
- ${nameNode}
yarn
cluster
build_adjacency_lists
eu.dnetlib.dhp.graph.SparkXmlRecordBuilderJob
dhp-graph-provision-${projectVersion}.jar
- --executor-memory ${sparkExecutorMemory}
- --executor-cores ${sparkExecutorCores}
- --driver-memory=${sparkDriverMemory}
+ --executor-cores ${sparkExecutorCoresForJoining}
+ --executor-memory ${sparkExecutorMemoryForJoining}
+ --driver-memory=${sparkDriverMemoryForJoining}
+ --conf spark.dynamicAllocation.maxExecutors=${sparkExecutorCoresForJoining}
--conf spark.extraListeners="com.cloudera.spark.lineage.NavigatorAppListener"
--conf spark.sql.queryExecutionListeners="com.cloudera.spark.lineage.NavigatorQueryListener"
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
@@ -60,6 +75,7 @@
-mt yarn
-is ${isLookupUrl}
+ -t ${otherDsTypeId}
--sourcePath${sourcePath}
--outputPath${outputPath}
@@ -69,16 +85,15 @@
- ${jobTracker}
- ${nameNode}
yarn
cluster
to_solr_index
eu.dnetlib.dhp.graph.SparkXmlIndexingJob
dhp-graph-provision-${projectVersion}.jar
- --executor-memory ${sparkExecutorMemory}
- --driver-memory=${sparkDriverMemory}
+ --executor-cores ${sparkExecutorCoresForIndexing}
+ --executor-memory ${sparkExecutorMemoryForIndexing}
+ --driver-memory=${sparkDriverMemoryForIndexing}
--conf spark.dynamicAllocation.maxExecutors=${sparkExecutorCoresForIndexing}
--conf spark.extraListeners="com.cloudera.spark.lineage.NavigatorAppListener"
--conf spark.sql.queryExecutionListeners="com.cloudera.spark.lineage.NavigatorQueryListener"
diff --git a/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/graph/GraphJoinerTest.java b/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/graph/GraphJoinerTest.java
new file mode 100644
index 0000000000..147ac801cd
--- /dev/null
+++ b/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/graph/GraphJoinerTest.java
@@ -0,0 +1,38 @@
+package eu.dnetlib.dhp.graph;
+
+import org.junit.Before;
+
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+
+public class GraphJoinerTest {
+
+ private ClassLoader cl = getClass().getClassLoader();
+ private Path workingDir;
+ private Path inputDir;
+ private Path outputDir;
+
+ @Before
+ public void before() throws IOException {
+ workingDir = Files.createTempDirectory("promote_action_set");
+ inputDir = workingDir.resolve("input");
+ outputDir = workingDir.resolve("output");
+ }
+
+ private static void copyFiles(Path source, Path target) throws IOException {
+ Files.list(source).forEach(f -> {
+ try {
+ if (Files.isDirectory(f)) {
+ Path subTarget = Files.createDirectories(target.resolve(f.getFileName()));
+ copyFiles(f, subTarget);
+ } else {
+ Files.copy(f, target.resolve(f.getFileName()));
+ }
+ } catch (IOException e) {
+ e.printStackTrace();
+ throw new RuntimeException(e);
+ }
+ });
+ }
+}
diff --git a/pom.xml b/pom.xml
index 74003a4078..f47d49ea7f 100644
--- a/pom.xml
+++ b/pom.xml
@@ -110,6 +110,12 @@
${dhp.hadoop.version}
provided
+
+ org.apache.hadoop
+ hadoop-distcp
+ ${dhp.hadoop.version}
+ provided
+
org.apache.spark
spark-core_2.11
@@ -142,6 +148,13 @@
${dhp.commons.lang.version}
+
+ com.google.guava
+ guava
+ ${dhp.guava.version}
+
+
+
commons-codec
commons-codec
@@ -262,6 +275,16 @@
provided
+
+ eu.dnetlib
+ dnet-actionmanager-common
+ 6.0.5
+
+
+ eu.dnetlib
+ dnet-openaire-data-protos
+ 3.9.8-proto250
+
eu.dnetlib
dnet-pace-core
@@ -480,6 +503,7 @@
2.4.0.cloudera2
2.9.6
3.5
+ 11.0.2
2.11.12
4.12
3.4.2