refactoring code

This commit is contained in:
Sandro La Bruzzo 2020-02-19 10:07:08 +01:00
parent ad4387dd38
commit 2b8675462f
38 changed files with 1332 additions and 169 deletions

View File

@ -46,6 +46,10 @@
<groupId>com.ximpleware</groupId>
<artifactId>vtd-xml</artifactId>
</dependency>
<dependency>
<groupId>com.jayway.jsonpath</groupId>
<artifactId>json-path</artifactId>
</dependency>
</dependencies>
</project>

View File

@ -1,5 +1,7 @@
package eu.dnetlib.dhp.utils;
import com.jayway.jsonpath.JsonPath;
import net.minidev.json.JSONArray;
import org.apache.commons.codec.binary.Base64;
import org.apache.commons.codec.binary.Base64OutputStream;
import org.apache.commons.codec.binary.Hex;
@ -56,4 +58,17 @@ public class DHPUtils {
}
public static String getJPathString(final String jsonPath, final String json) {
try {
Object o = JsonPath.read(json, jsonPath);
if (o instanceof String)
return (String) o;
if (o instanceof JSONArray && ((JSONArray) o).size() > 0)
return (String) ((JSONArray) o).get(0);
return "";
} catch (Exception e) {
return "";
}
}
}

View File

@ -45,6 +45,7 @@
<artifactId>jaxen</artifactId>
</dependency>
<dependency>
<groupId>org.mockito</groupId>
<artifactId>mockito-core</artifactId>

View File

@ -1,24 +1,20 @@
package eu.dnetlib.dedup;
import com.fasterxml.jackson.databind.DeserializationFeature;
import com.google.common.collect.Lists;
import eu.dnetlib.dhp.schema.oaf.*;
import eu.dnetlib.pace.config.DedupConfig;
import eu.dnetlib.pace.util.MapDocumentUtil;
import org.apache.commons.lang.NotImplementedException;
import org.apache.commons.lang.StringUtils;
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.Encoders;
import org.apache.spark.sql.SparkSession;
import org.codehaus.jackson.map.ObjectMapper;
import com.fasterxml.jackson.databind.ObjectMapper;
import scala.Tuple2;
import java.util.Collection;
import java.util.Random;
import static java.util.stream.Collectors.toMap;
public class DedupRecordFactory {
@ -73,6 +69,8 @@ public class DedupRecordFactory {
p.setId(e._1());
final ObjectMapper mapper = new ObjectMapper();
mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
final Collection<String> dateofacceptance = Lists.newArrayList();
@ -105,6 +103,7 @@ public class DedupRecordFactory {
d.setId(e._1());
final ObjectMapper mapper = new ObjectMapper();
mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
final Collection<String> dateofacceptance = Lists.newArrayList();
@ -137,6 +136,7 @@ public class DedupRecordFactory {
p.setId(e._1());
final ObjectMapper mapper = new ObjectMapper();
mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
if (e._2() != null)
e._2().forEach(proj -> {
try {
@ -160,6 +160,7 @@ public class DedupRecordFactory {
s.setId(e._1());
final ObjectMapper mapper = new ObjectMapper();
mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
final Collection<String> dateofacceptance = Lists.newArrayList();
if (e._2() != null)
e._2().forEach(soft -> {
@ -187,6 +188,7 @@ public class DedupRecordFactory {
Datasource d = new Datasource(); //the result of the merge, to be returned at the end
d.setId(e._1());
final ObjectMapper mapper = new ObjectMapper();
mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
if (e._2() != null)
e._2().forEach(dat -> {
try {
@ -211,6 +213,7 @@ public class DedupRecordFactory {
o.setId(e._1());
final ObjectMapper mapper = new ObjectMapper();
mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
StringBuilder trust = new StringBuilder("0.0");
@ -251,6 +254,7 @@ public class DedupRecordFactory {
o.setId(e._1());
final ObjectMapper mapper = new ObjectMapper();
mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
final Collection<String> dateofacceptance = Lists.newArrayList();

View File

@ -151,11 +151,11 @@ public class DedupUtility {
}
public static String createSimRelPath(final String basePath, final String entityType) {
return String.format("%s/%s_simRel", basePath, entityType);
return String.format("%s/%s/simRel", basePath, entityType);
}
public static String createMergeRelPath(final String basePath, final String entityType) {
return String.format("%s/%s_mergeRel", basePath, entityType);
return String.format("%s/%s/mergeRel", basePath, entityType);
}
private static Double sim(Author a, Author b) {

View File

@ -10,7 +10,6 @@ import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.sql.SparkSession;
public class SparkCreateDedupRecord {
public static void main(String[] args) throws Exception {
final ArgumentApplicationParser parser = new ArgumentApplicationParser(IOUtils.toString(SparkCreateDedupRecord.class.getResourceAsStream("/eu/dnetlib/dhp/dedup/dedupRecord_parameters.json")));
parser.parseArgument(args);
@ -24,16 +23,12 @@ public class SparkCreateDedupRecord {
final String sourcePath = parser.get("sourcePath");
final String entity = parser.get("entity");
final String dedupPath = parser.get("dedupPath");
// final DedupConfig dedupConf = DedupConfig.load(IOUtils.toString(SparkCreateDedupRecord.class.getResourceAsStream("/eu/dnetlib/dhp/dedup/conf/org.curr.conf2.json")));
final DedupConfig dedupConf = DedupConfig.load(parser.get("dedupConf"));
final JavaRDD<OafEntity> dedupRecord = DedupRecordFactory.createDedupRecord(sc, spark, DedupUtility.createMergeRelPath(dedupPath,entity), DedupUtility.createEntityPath(sourcePath,entity), OafEntityType.valueOf(entity), dedupConf);
dedupRecord.map(r-> {
ObjectMapper mapper = new ObjectMapper();
return mapper.writeValueAsString(r);
}).saveAsTextFile(dedupPath+"/"+entity+"_dedup_record_json");
}).saveAsTextFile(dedupPath+"/"+entity+"/dedup_records");
}
}

View File

@ -44,7 +44,7 @@ public class SparkCreateSimRels {
// final DedupConfig dedupConf = DedupConfig.load(IOUtils.toString(SparkCreateSimRels.class.getResourceAsStream("/eu/dnetlib/dhp/dedup/conf/org.curr.conf.json")));
final DedupConfig dedupConf = DedupConfig.load(parser.get("dedupConf"));
final long total = sc.textFile(inputPath + "/" + entity).count();
JavaPairRDD<String, MapDocument> mapDocument = sc.textFile(inputPath + "/" + entity)
.mapToPair(s->{
@ -70,4 +70,4 @@ public class SparkCreateSimRels {
spark.createDataset(isSimilarToRDD.rdd(), Encoders.bean(Relation.class)).write().mode("overwrite").save( DedupUtility.createSimRelPath(targetPath,entity));
}
}
}

View File

@ -0,0 +1,117 @@
package eu.dnetlib.dedup;
import com.fasterxml.jackson.databind.DeserializationFeature;
import com.fasterxml.jackson.databind.ObjectMapper;
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
import eu.dnetlib.dhp.schema.oaf.DataInfo;
import eu.dnetlib.dhp.schema.oaf.Relation;
import eu.dnetlib.dhp.utils.DHPUtils;
import org.apache.commons.io.IOUtils;
import org.apache.hadoop.io.compress.GzipCodec;
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.Optional;
import org.apache.spark.api.java.function.Function;
import org.apache.spark.api.java.function.PairFunction;
import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Encoders;
import org.apache.spark.sql.Row;
import org.apache.spark.sql.SparkSession;
import scala.Tuple2;
import java.io.IOException;
public class SparkPropagateRelationsJob {
enum FieldType {
SOURCE,
TARGET
}
final static String IDJSONPATH = "$.id";
final static String SOURCEJSONPATH = "$.source";
final static String TARGETJSONPATH = "$.target";
public static void main(String[] args) throws Exception {
final ArgumentApplicationParser parser = new ArgumentApplicationParser(IOUtils.toString(SparkPropagateRelationsJob.class.getResourceAsStream("/eu/dnetlib/dhp/dedup/dedup_propagate_relation_parameters.json")));
parser.parseArgument(args);
final SparkSession spark = SparkSession
.builder()
.appName(SparkUpdateEntityJob.class.getSimpleName())
.master(parser.get("master"))
.getOrCreate();
final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext());
final String relationPath = parser.get("relationPath");
final String mergeRelPath = parser.get("mergeRelPath");
final String targetRelPath = parser.get("targetRelPath");
final Dataset<Relation> df = spark.read().load(mergeRelPath).as(Encoders.bean(Relation.class));
final JavaPairRDD<String, String> mergedIds = df
.where("relClass == 'merges'")
.select(df.col("source"),df.col("target"))
.distinct()
.toJavaRDD()
.mapToPair((PairFunction<Row, String, String>) r -> new Tuple2<>(r.getString(1), r.getString(0)));
final JavaRDD<String> sourceEntity = sc.textFile(relationPath);
JavaRDD<String> newRels = sourceEntity.mapToPair(
(PairFunction<String, String, String>) s ->
new Tuple2<>(DHPUtils.getJPathString(SOURCEJSONPATH, s), s))
.leftOuterJoin(mergedIds)
.map((Function<Tuple2<String, Tuple2<String, Optional<String>>>, String>) v1 -> {
if (v1._2()._2().isPresent()) {
return replaceField(v1._2()._1(), v1._2()._2().get(), FieldType.SOURCE);
}
return v1._2()._1();
})
.mapToPair(
(PairFunction<String, String, String>) s ->
new Tuple2<>(DHPUtils.getJPathString(TARGETJSONPATH, s), s))
.leftOuterJoin(mergedIds)
.map((Function<Tuple2<String, Tuple2<String, Optional<String>>>, String>) v1 -> {
if (v1._2()._2().isPresent()) {
return replaceField(v1._2()._1(), v1._2()._2().get(), FieldType.TARGET);
}
return v1._2()._1();
}).filter(SparkPropagateRelationsJob::containsDedup)
.repartition(500);
newRels.union(sourceEntity).repartition(1000).saveAsTextFile(targetRelPath, GzipCodec.class);
}
private static boolean containsDedup(final String json) {
final String source = DHPUtils.getJPathString(SOURCEJSONPATH, json);
final String target = DHPUtils.getJPathString(TARGETJSONPATH, json);
return source.toLowerCase().contains("dedup") || target.toLowerCase().contains("dedup");
}
private static String replaceField(final String json, final String id, final FieldType type) {
ObjectMapper mapper = new ObjectMapper();
mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
try {
Relation relation = mapper.readValue(json, Relation.class);
if (relation.getDataInfo() == null)
relation.setDataInfo(new DataInfo());
relation.getDataInfo().setDeletedbyinference(false);
switch (type) {
case SOURCE:
relation.setSource(id);
return mapper.writeValueAsString(relation);
case TARGET:
relation.setTarget(id);
return mapper.writeValueAsString(relation);
default:
throw new IllegalArgumentException("");
}
} catch (IOException e) {
throw new RuntimeException("unable to deserialize json relation: " + json, e);
}
}
}

View File

@ -0,0 +1,114 @@
package eu.dnetlib.dedup;
import com.fasterxml.jackson.databind.DeserializationFeature;
import com.fasterxml.jackson.databind.ObjectMapper;
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
import eu.dnetlib.dhp.schema.oaf.DataInfo;
import eu.dnetlib.dhp.schema.oaf.Oaf;
import eu.dnetlib.dhp.schema.oaf.Relation;
import eu.dnetlib.dhp.schema.scholexplorer.DLIDataset;
import eu.dnetlib.dhp.schema.scholexplorer.DLIPublication;
import eu.dnetlib.dhp.schema.scholexplorer.DLIUnknown;
import eu.dnetlib.dhp.utils.DHPUtils;
import org.apache.commons.io.IOUtils;
import org.apache.hadoop.io.compress.GzipCodec;
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.Dataset;
import org.apache.spark.sql.Encoders;
import org.apache.spark.sql.Row;
import org.apache.spark.sql.SparkSession;
import scala.Tuple2;
import java.io.IOException;
public class SparkUpdateEntityJob {
final static String IDJSONPATH = "$.id";
final static String SOURCEJSONPATH = "$.source";
final static String TARGETJSONPATH = "$.target";
public static void main(String[] args) throws Exception {
final ArgumentApplicationParser parser = new ArgumentApplicationParser(IOUtils.toString(SparkUpdateEntityJob.class.getResourceAsStream("/eu/dnetlib/dhp/dedup/dedup_delete_by_inference_parameters.json")));
parser.parseArgument(args);
final SparkSession spark = SparkSession
.builder()
.appName(SparkUpdateEntityJob.class.getSimpleName())
.master(parser.get("master"))
.getOrCreate();
final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext());
final String entityPath = parser.get("entityPath");
final String mergeRelPath = parser.get("mergeRelPath");
final String dedupRecordPath = parser.get("dedupRecordPath");
final String entity = parser.get("entity");
final Dataset<Relation> df = spark.read().load(mergeRelPath).as(Encoders.bean(Relation.class));
final JavaPairRDD<String, String> mergedIds = df
.where("relClass == 'merges'")
.select(df.col("target"))
.distinct()
.toJavaRDD()
.mapToPair((PairFunction<Row, String, String>) r -> new Tuple2<>(r.getString(0), "d"));
final JavaRDD<String> sourceEntity = sc.textFile(entityPath);
if ("relation".equalsIgnoreCase(entity)) {
sourceEntity.mapToPair(
(PairFunction<String, String, String>) s ->
new Tuple2<>(DHPUtils.getJPathString(SOURCEJSONPATH, s), s))
.leftOuterJoin(mergedIds)
.map(k -> k._2()._2().isPresent() ? updateDeletedByInference(k._2()._1(), Relation.class) : k._2()._1())
.mapToPair((PairFunction<String, String, String>) s -> new Tuple2<>(DHPUtils.getJPathString(TARGETJSONPATH, s), s))
.leftOuterJoin(mergedIds)
.map(k -> k._2()._2().isPresent() ? updateDeletedByInference(k._2()._1(), Relation.class) : k._2()._1())
.saveAsTextFile(entityPath + "_new", GzipCodec.class);
} else {
final JavaRDD<String> dedupEntity = sc.textFile(dedupRecordPath);
JavaPairRDD<String, String> entitiesWithId = sourceEntity.mapToPair((PairFunction<String, String, String>) s -> new Tuple2<>(DHPUtils.getJPathString(IDJSONPATH, s), s));
Class<? extends Oaf> mainClass;
switch (entity) {
case "publication":
mainClass = DLIPublication.class;
break;
case "dataset":
mainClass = DLIDataset.class;
break;
case "unknown":
mainClass = DLIUnknown.class;
break;
default:
throw new IllegalArgumentException("Illegal type " + entity);
}
JavaRDD<String> map = entitiesWithId.leftOuterJoin(mergedIds).map(k -> k._2()._2().isPresent() ? updateDeletedByInference(k._2()._1(), mainClass) : k._2()._1());
map.union(dedupEntity).saveAsTextFile(entityPath + "_new", GzipCodec.class);
}
}
private static <T extends Oaf> String updateDeletedByInference(final String json, final Class<T> clazz) {
final ObjectMapper mapper = new ObjectMapper();
mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
try {
Oaf entity = mapper.readValue(json, clazz);
if (entity.getDataInfo()== null)
entity.setDataInfo(new DataInfo());
entity.getDataInfo().setDeletedbyinference(true);
return mapper.writeValueAsString(entity);
} catch (IOException e) {
throw new RuntimeException("Unable to convert json", e);
}
}
}

View File

@ -0,0 +1,31 @@
[
{
"paramName": "mt",
"paramLongName": "master",
"paramDescription": "should be local or yarn",
"paramRequired": true
},
{
"paramName": "ep",
"paramLongName": "entityPath",
"paramDescription": "the input entity path",
"paramRequired": true
},
{
"paramName": "mr",
"paramLongName": "mergeRelPath",
"paramDescription": "the input path of merge Rel",
"paramRequired": true
},
{
"paramName": "dr",
"paramLongName": "dedupRecordPath",
"paramDescription": "the inputPath of dedup record",
"paramRequired": true
}, {
"paramName": "e",
"paramLongName": "entity",
"paramDescription": "the type of entity",
"paramRequired": true
}
]

View File

@ -0,0 +1,26 @@
[
{
"paramName": "mt",
"paramLongName": "master",
"paramDescription": "should be local or yarn",
"paramRequired": true
},
{
"paramName": "ep",
"paramLongName": "relationPath",
"paramDescription": "the input relation path",
"paramRequired": true
},
{
"paramName": "mr",
"paramLongName": "mergeRelPath",
"paramDescription": "the input path of merge Rel",
"paramRequired": true
},
{
"paramName": "t",
"paramLongName": "targetRelPath",
"paramDescription": "the output Rel Path",
"paramRequired": true
}
]

View File

@ -24,27 +24,24 @@
<name>sparkExecutorMemory</name>
<description>memory for individual executor</description>
</property>
<property>
<name>sparkExecutorCores</name>
<description>number of cores used by single executor</description>
</property>
</parameters>
<start to="CreateSimRels"/>
<start to="DeleteWorkingPath"/>
<kill name="Kill">
<message>Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}]</message>
</kill>
<!-- <action name="DeleteTargetPath">-->
<!-- <fs>-->
<!-- <delete path='${targetPath}/${entity}_simrel'/>-->
<!-- <delete path='${targetPath}/${entity}_mergeRels'/>-->
<!-- </fs>-->
<!-- <ok to="CreateSimRels"/>-->
<!-- <error to="Kill"/>-->
<!-- </action>-->
<action name="DeleteWorkingPath">
<fs>
<delete path='${targetPath}/${entity}'/>
<mkdir path="${targetPath}"/>
<mkdir path="${targetPath}/${entity}"/>
</fs>
<ok to="CreateSimRels"/>
<error to="Kill"/>
</action>
<action name="CreateSimRels">
<spark xmlns="uri:oozie:spark-action:0.2">
@ -55,11 +52,11 @@
<name>Create Similarity Relations</name>
<class>eu.dnetlib.dedup.SparkCreateSimRels</class>
<jar>dhp-dedup-${projectVersion}.jar</jar>
<spark-opts>--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"
<spark-opts>
--executor-memory ${sparkExecutorMemory}
--driver-memory=${sparkDriverMemory}
--num-executors 100
--conf spark.yarn.jars="hdfs://hadoop-rm1.garr-pa1.d4science.org:8020/user/oozie/share/lib/lib_20180405103059/spark2"
</spark-opts>
<arg>-mt</arg><arg>yarn-cluster</arg>
<arg>--sourcePath</arg><arg>${sourcePath}</arg>
@ -71,7 +68,6 @@
<error to="Kill"/>
</action>
<action name="CreateConnectedComponents">
<spark xmlns="uri:oozie:spark-action:0.2">
<job-tracker>${jobTracker}</job-tracker>
@ -81,11 +77,11 @@
<name>Create Connected Components</name>
<class>eu.dnetlib.dedup.SparkCreateConnectedComponent</class>
<jar>dhp-dedup-${projectVersion}.jar</jar>
<spark-opts>--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"
<spark-opts>
--executor-memory ${sparkExecutorMemory}
--driver-memory=${sparkDriverMemory}
--num-executors 100
--conf spark.yarn.jars="hdfs://hadoop-rm1.garr-pa1.d4science.org:8020/user/oozie/share/lib/lib_20180405103059/spark2"
</spark-opts>
<arg>-mt</arg><arg>yarn-cluster</arg>
<arg>--sourcePath</arg><arg>${sourcePath}</arg>
@ -106,21 +102,46 @@
<name>Create Dedup Record</name>
<class>eu.dnetlib.dedup.SparkCreateDedupRecord</class>
<jar>dhp-dedup-${projectVersion}.jar</jar>
<spark-opts>--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"
<spark-opts>
--executor-memory ${sparkExecutorMemory}
--driver-memory=${sparkDriverMemory}
--num-executors 100
--conf spark.yarn.jars="hdfs://hadoop-rm1.garr-pa1.d4science.org:8020/user/oozie/share/lib/lib_20180405103059/spark2"
</spark-opts>
<arg>-mt</arg><arg>yarn-cluster</arg>
<arg>--sourcePath</arg><arg>${sourcePath}</arg>
<arg>--dedupPath</arg><arg>${dedupPath}</arg>
<arg>--dedupPath</arg><arg>${targetPath}</arg>
<arg>--entity</arg><arg>${entity}</arg>
<arg>--dedupConf</arg><arg>${dedupConf}</arg>
</spark>
<ok to="fixRelation"/>
<error to="Kill"/>
</action>
<action name="fixRelation">
<spark xmlns="uri:oozie:spark-action:0.2">
<job-tracker>${jobTracker}</job-tracker>
<name-node>${nameNode}</name-node>
<master>yarn-cluster</master>
<mode>cluster</mode>
<name>Propagate Dedup Relations</name>
<class>eu.dnetlib.dedup.SparkPropagateRelationsJob</class>
<jar>dhp-dedup-${projectVersion}.jar</jar>
<spark-opts>
--executor-memory ${sparkExecutorMemory}
--driver-memory=${sparkDriverMemory}
--num-executors 100
--conf spark.yarn.jars="hdfs://hadoop-rm1.garr-pa1.d4science.org:8020/user/oozie/share/lib/lib_20180405103059/spark2"
</spark-opts>
<arg>-mt</arg><arg>yarn-cluster</arg>
<arg>--mergeRelPath</arg><arg>${targetPath}/${entity}/mergeRel</arg>
<arg>--relationPath</arg><arg>${sourcePath}/relation</arg>
<arg>--targetRelPath</arg><arg>${targetPath}/${entity}/relation_updated</arg>
</spark>
<ok to="End"/>
<error to="Kill"/>
</action>
<end name="End"/>
</workflow-app>

View File

@ -0,0 +1,52 @@
<workflow-app name="Dedup Entities" xmlns="uri:oozie:workflow:0.5">
<parameters>
<property>
<name>relationPath</name>
<description>the source path</description>
</property>
<property>
<name>mergeRelPath</name>
<description>the target path</description>
</property>
<property>
<name>sparkDriverMemory</name>
<description>memory for driver process</description>
</property>
<property>
<name>sparkExecutorMemory</name>
<description>memory for individual executor</description>
</property>
</parameters>
<start to="PropagateRelation"/>
<kill name="Kill">
<message>Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}]</message>
</kill>
<action name="PropagateRelation">
<spark xmlns="uri:oozie:spark-action:0.2">
<job-tracker>${jobTracker}</job-tracker>
<name-node>${nameNode}</name-node>
<master>yarn-cluster</master>
<mode>cluster</mode>
<name>Propagate Dedup Relations</name>
<class>eu.dnetlib.dedup.SparkPropagateRelationsJob</class>
<jar>dhp-dedup-${projectVersion}.jar</jar>
<spark-opts>
--executor-memory ${sparkExecutorMemory}
--driver-memory=${sparkDriverMemory}
--num-executors 100
--conf spark.yarn.jars="hdfs://hadoop-rm1.garr-pa1.d4science.org:8020/user/oozie/share/lib/lib_20180405103059/spark2"
</spark-opts>
<arg>-mt</arg><arg>yarn-cluster</arg>
<arg>--mergeRelPath</arg><arg>${mergeRelPath}</arg>
<arg>--relationPath</arg><arg>${relationPath}</arg>
</spark>
<ok to="End"/>
<error to="Kill"/>
</action>
<end name="End"/>
</workflow-app>

View File

@ -0,0 +1,30 @@
<configuration>
<property>
<name>jobTracker</name>
<value>yarnRM</value>
</property>
<property>
<name>nameNode</name>
<value>hdfs://nameservice1</value>
</property>
<property>
<name>oozie.use.system.libpath</name>
<value>true</value>
</property>
<property>
<name>oozie.action.sharelib.for.spark</name>
<value>spark2</value>
</property>
<property>
<name>hive_metastore_uris</name>
<value>thrift://iis-cdh5-test-m3.ocean.icm.edu.pl:9083</value>
</property>
<property>
<name>hive_db_name</name>
<value>openaire</value>
</property>
<property>
<name>master</name>
<value>yarn</value>
</property>
</configuration>

View File

@ -0,0 +1,65 @@
<workflow-app name="Dedup Entities" xmlns="uri:oozie:workflow:0.5">
<parameters>
<property>
<name>entity</name>
<description>the entity that should be processed</description>
</property>
<property>
<name>entityPath</name>
<description>the source path</description>
</property>
<property>
<name>mergeRelPath</name>
<description>the target path</description>
</property>
<property>
<name>dedupRecordPath</name>
<description>the target path</description>
</property>
<property>
<name>master</name>
<description>the target path</description>
</property>
<property>
<name>sparkDriverMemory</name>
<description>memory for driver process</description>
</property>
<property>
<name>sparkExecutorMemory</name>
<description>memory for individual executor</description>
</property>
</parameters>
<start to="updateEntity"/>
<kill name="Kill">
<message>Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}]</message>
</kill>
<action name="updateEntity">
<spark xmlns="uri:oozie:spark-action:0.2">
<job-tracker>${jobTracker}</job-tracker>
<name-node>${nameNode}</name-node>
<master>${master}</master>
<mode>cluster</mode>
<name>Update ${entity} and add DedupRecord</name>
<class>eu.dnetlib.dedup.SparkUpdateEntityJob</class>
<jar>dhp-dedup-${projectVersion}.jar</jar>
<spark-opts>
--executor-memory ${sparkExecutorMemory}
--driver-memory=${sparkDriverMemory}
--num-executors 100
--conf spark.yarn.jars="hdfs://hadoop-rm1.garr-pa1.d4science.org:8020/user/oozie/share/lib/lib_20180405103059/spark2"
</spark-opts>
<arg>-mt</arg><arg>${master}</arg>
<arg>--entityPath</arg><arg>${entityPath}</arg>
<arg>--mergeRelPath</arg><arg>${mergeRelPath}</arg>
<arg>--entity</arg><arg>${entity}</arg>
<arg>--dedupRecordPath</arg><arg>${dedupRecordPath}</arg>
</spark>
<ok to="End"/>
<error to="Kill"/>
</action>
<end name="End"/>
</workflow-app>

View File

@ -1,19 +1,14 @@
package eu.dnetlib.dedup;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.hash.HashFunction;
import com.google.common.hash.Hashing;
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
import eu.dnetlib.dhp.schema.oaf.Publication;
import org.apache.commons.io.FileUtils;
import org.apache.commons.io.IOUtils;
import org.junit.Before;
import org.junit.Ignore;
import org.junit.Test;
import java.io.File;
import java.io.IOException;
import java.util.List;
public class SparkCreateDedupTest {
@ -22,7 +17,7 @@ public class SparkCreateDedupTest {
@Before
public void setUp() throws IOException {
configuration = IOUtils.toString(getClass().getResourceAsStream("/eu/dnetlib/dedup/conf/org.curr.conf.json"));
configuration = IOUtils.toString(getClass().getResourceAsStream("/eu/dnetlib/dedup/conf/pub_scholix.conf.json"));
}
@ -38,6 +33,14 @@ public class SparkCreateDedupTest {
});
}
@Test
public void createDeletedByInference() throws Exception {
SparkUpdateEntityJob.main(new String[] {
"-mt", "local[*]"
});
}
@Test
@Ignore
public void createCCTest() throws Exception {

View File

@ -0,0 +1,378 @@
{
"wf": {
"threshold": "0.99",
"dedupRun": "001",
"entityType": "result",
"subEntityType": "resulttype",
"subEntityValue": "publication",
"orderField": "title",
"queueMaxSize": "2000",
"groupMaxSize": "100",
"maxChildren": "100",
"slidingWindowSize": "200",
"rootBuilder": [
],
"includeChildren": "true",
"maxIterations": 20,
"idPath": "$.id"
},
"pace": {
"clustering": [
{
"name": "ngrampairs",
"fields": [
"title"
],
"params": {
"max": "1",
"ngramLen": "3"
}
},
{
"name": "suffixprefix",
"fields": [
"title"
],
"params": {
"max": "1",
"len": "3"
}
}
],
"decisionTree": {
"start": {
"fields": [
{
"field": "pid",
"comparator": "jsonListMatch",
"weight": 1.0,
"countIfUndefined": "false",
"params": {
"jpath_value": "$.value",
"jpath_classid": "$.qualifier.classid"
}
}
],
"threshold": 0.5,
"aggregation": "AVG",
"positive": "MATCH",
"negative": "layer2",
"undefined": "layer2",
"ignoreUndefined": "true"
},
"layer2": {
"fields": [
{
"field": "title",
"comparator": "titleVersionMatch",
"weight": 1.0,
"countIfUndefined": "false",
"params": {}
},
{
"field": "authors",
"comparator": "sizeMatch",
"weight": 1.0,
"countIfUndefined": "false",
"params": {}
}
],
"threshold": 1.0,
"aggregation": "AND",
"positive": "layer3",
"negative": "NO_MATCH",
"undefined": "layer3",
"ignoreUndefined": "false"
},
"layer3": {
"fields": [
{
"field": "title",
"comparator": "levensteinTitle",
"weight": 1.0,
"countIfUndefined": "true",
"params": {}
}
],
"threshold": 0.99,
"aggregation": "AVG",
"positive": "MATCH",
"negative": "NO_MATCH",
"undefined": "NO_MATCH",
"ignoreUndefined": "true"
}
},
"model": [
{
"name": "pid",
"type": "JSON",
"path": "$.pid",
"overrideMatch": "true"
},
{
"name": "title",
"type": "String",
"path": "$.title[*].value",
"length": 250,
"size": 5
},
{
"name": "authors",
"type": "List",
"path": "$.author[*].fullname",
"size": 200
},
{
"name": "resulttype",
"type": "String",
"path": "$.resulttype.classid"
}
],
"blacklists": {
"title": [
"^Inside Front Cover$",
"^CORR Insights$",
"^Index des notions$",
"^Department of Error.$",
"^Untitled Item$",
"^Department of Error$",
"^Tome II : 1598 à 1605$",
"^(à lexception de roi, prince, royauté, pouvoir, image… qui sont omniprésents)$",
"^Museen und Ausstellungsinstitute in Nürnberg$",
"^Text/Conference Paper$",
"^Table des illustrations$",
"^An Intimate Insight on Psychopathy and a Novel Hermeneutic Psychological Science$",
"^Index des noms$",
"^Reply by Authors.$",
"^Titelblatt - Inhalt$",
"^Index des œuvres,$",
"(?i)^Poster presentations$",
"^THE ASSOCIATION AND THE GENERAL MEDICAL COUNCIL$",
"^Problems with perinatal pathology\\.?$",
"(?i)^Cases? of Puerperal Convulsions$",
"(?i)^Operative Gyna?ecology$",
"(?i)^Mind the gap\\!?\\:?$",
"^Chronic fatigue syndrome\\.?$",
"^Cartas? ao editor Letters? to the Editor$",
"^Note from the Editor$",
"^Anesthesia Abstract$",
"^Annual report$",
"(?i)^“?THE RADICAL PREVENTION OF VENEREAL DISEASE\\.?”?$",
"(?i)^Graph and Table of Infectious Diseases?$",
"^Presentation$",
"(?i)^Reviews and Information on Publications$",
"(?i)^PUBLIC HEALTH SERVICES?$",
"(?i)^COMBINED TEXT-?BOOK OF OBSTETRICS AND GYN(Æ|ae)COLOGY$",
"(?i)^Adrese autora$",
"(?i)^Systematic Part .*\\. Catalogus Fossilium Austriae, Band 2: Echinoidea neogenica$",
"(?i)^Acknowledgement to Referees$",
"(?i)^Behçet's disease\\.?$",
"(?i)^Isolation and identification of restriction endonuclease.*$",
"(?i)^CEREBROVASCULAR DISEASES?.?$",
"(?i)^Screening for abdominal aortic aneurysms?\\.?$",
"^Event management$",
"(?i)^Breakfast and Crohn's disease.*\\.?$",
"^Cálculo de concentraciones en disoluciones acuosas. Ejercicio interactivo\\..*\\.$",
"(?i)^Genetic and functional analyses of SHANK2 mutations suggest a multiple hit model of Autism spectrum disorders?\\.?$",
"^Gushi hakubutsugaku$",
"^Starobosanski nadpisi u Bosni i Hercegovini \\(.*\\)$",
"^Intestinal spirocha?etosis$",
"^Treatment of Rodent Ulcer$",
"(?i)^\\W*Cloud Computing\\W*$",
"^Compendio mathematico : en que se contienen todas las materias mas principales de las Ciencias que tratan de la cantidad$",
"^Free Communications, Poster Presentations: Session [A-F]$",
"^“The Historical Aspects? of Quackery\\.?”$",
"^A designated centre for people with disabilities operated by St John of God Community Services (Limited|Ltd), Louth$",
"^P(er|re)-Mile Premiums for Auto Insurance\\.?$",
"(?i)^Case Report$",
"^Boletín Informativo$",
"(?i)^Glioblastoma Multiforme$",
"(?i)^Nuevos táxones animales descritos en la península Ibérica y Macaronesia desde 1994 \\(.*\\)$",
"^Zaměstnanecké výhody$",
"(?i)^The Economics of Terrorism and Counter-Terrorism: A Survey \\(Part .*\\)$",
"(?i)^Carotid body tumours?\\.?$",
"(?i)^\\[Españoles en Francia : La condición Emigrante.*\\]$",
"^Avant-propos$",
"(?i)^St\\. Patrick's Cathedral, Dublin, County Dublin - Head(s)? and Capital(s)?$",
"(?i)^St\\. Patrick's Cathedral, Dublin, County Dublin - Bases?$",
"(?i)^PUBLIC HEALTH VERSUS THE STATE$",
"^Viñetas de Cortázar$",
"(?i)^Search for heavy neutrinos and W(\\[|_|\\(|_\\{|-)?R(\\]|\\)|\\})? bosons with right-handed couplings in a left-right symmetric model in pp collisions at.*TeV(\\.)?$",
"(?i)^Measurement of the pseudorapidity and centrality dependence of the transverse energy density in Pb(-?)Pb collisions at.*tev(\\.?)$",
"(?i)^Search for resonances decaying into top-quark pairs using fully hadronic decays in pp collisions with ATLAS at.*TeV$",
"(?i)^Search for neutral minimal supersymmetric standard model Higgs bosons decaying to tau pairs in pp collisions at.*tev$",
"(?i)^Relatório de Estágio (de|em) Angiologia e Cirurgia Vascular$",
"^Aus der AGMB$",
"^Znanstveno-stručni prilozi$",
"(?i)^Zhodnocení finanční situace podniku a návrhy na zlepšení$",
"(?i)^Evaluation of the Financial Situation in the Firm and Proposals to its Improvement$",
"(?i)^Hodnocení finanční situace podniku a návrhy na její zlepšení$",
"^Finanční analýza podniku$",
"^Financial analysis( of business)?$",
"(?i)^Textbook of Gyn(a)?(Æ)?(e)?cology$",
"^Jikken nihon shūshinsho$",
"(?i)^CORONER('|s)(s|') INQUESTS$",
"(?i)^(Μελέτη παραγόντων )?risk management( για ανάπτυξη και εφαρμογή ενός πληροφοριακού συστήματος| και ανάπτυξη συστήματος)?$",
"(?i)^Consultants' contract(s)?$",
"(?i)^Upute autorima$",
"(?i)^Bijdrage tot de Kennis van den Godsdienst der Dajaks van Lan(d|f)ak en Tajan$",
"^Joshi shin kokubun$",
"^Kōtō shōgaku dokuhon nōson'yō$",
"^Jinjō shōgaku shōka$",
"^Shōgaku shūjichō$",
"^Nihon joshi dokuhon$",
"^Joshi shin dokuhon$",
"^Chūtō kanbun dokuhon$",
"^Wabun dokuhon$",
"(?i)^(Analysis of economy selected village or town|Rozbor hospodaření vybrané obce či města)$",
"(?i)^cardiac rehabilitation$",
"(?i)^Analytical summary$",
"^Thesaurus resolutionum Sacrae Congregationis Concilii$",
"(?i)^Sumario analítico(\\s{1})?(Analitic summary)?$",
"^Prikazi i osvrti$",
"^Rodinný dům s provozovnou$",
"^Family house with an establishment$",
"^Shinsei chūtō shin kokugun$",
"^Pulmonary alveolar proteinosis(\\.?)$",
"^Shinshū kanbun$",
"^Viñeta(s?) de Rodríguez$",
"(?i)^RUBRIKA UREDNIKA$",
"^A Matching Model of the Academic Publication Market$",
"^Yōgaku kōyō$",
"^Internetový marketing$",
"^Internet marketing$",
"^Chūtō kokugo dokuhon$",
"^Kokugo dokuhon$",
"^Antibiotic Cover for Dental Extraction(s?)$",
"^Strategie podniku$",
"^Strategy of an Enterprise$",
"(?i)^respiratory disease(s?)(\\.?)$",
"^Award(s?) for Gallantry in Civil Defence$",
"^Podniková kultura$",
"^Corporate Culture$",
"^Severe hyponatraemia in hospital inpatient(s?)(\\.?)$",
"^Pracovní motivace$",
"^Work Motivation$",
"^Kaitei kōtō jogaku dokuhon$",
"^Konsolidovaná účetní závěrka$",
"^Consolidated Financial Statements$",
"(?i)^intracranial tumour(s?)$",
"^Climate Change Mitigation Options and Directed Technical Change: A Decentralized Equilibrium Analysis$",
"^\\[CERVECERIAS MAHOU(\\.|\\:) INTERIOR\\] \\[Material gráfico\\]$",
"^Housing Market Dynamics(\\:|\\.) On the Contribution of Income Shocks and Credit Constraint(s?)$",
"^\\[Funciones auxiliares de la música en Radio París,.*\\]$",
"^Úroveň motivačního procesu jako způsobu vedení lidí$",
"^The level of motivation process as a leadership$",
"^Pay-beds in N(\\.?)H(\\.?)S(\\.?) Hospitals$",
"(?i)^news and events$",
"(?i)^NOVOSTI I DOGAĐAJI$",
"^Sansū no gakushū$",
"^Posouzení informačního systému firmy a návrh změn$",
"^Information System Assessment and Proposal for ICT Modification$",
"^Stresové zatížení pracovníků ve vybrané profesi$",
"^Stress load in a specific job$",
"^Sunday: Poster Sessions, Pt.*$",
"^Monday: Poster Sessions, Pt.*$",
"^Wednesday: Poster Sessions, Pt.*",
"^Tuesday: Poster Sessions, Pt.*$",
"^Analýza reklamy$",
"^Analysis of advertising$",
"^Shōgaku shūshinsho$",
"^Shōgaku sansū$",
"^Shintei joshi kokubun$",
"^Taishō joshi kokubun dokuhon$",
"^Joshi kokubun$",
"^Účetní uzávěrka a účetní závěrka v ČR$",
"(?i)^The \"?Causes\"? of Cancer$",
"^Normas para la publicación de artículos$",
"^Editor('|s)(s|') [Rr]eply$",
"^Editor(|s)(s|) letter$",
"^Redaktoriaus žodis$",
"^DISCUSSION ON THE PRECEDING PAPER$",
"^Kōtō shōgaku shūshinsho jidōyō$",
"^Shōgaku nihon rekishi$",
"^(Theory of the flow of action currents in isolated myelinated nerve fibers).*$",
"^Préface$",
"^Occupational [Hh]ealth [Ss]ervices.$",
"^In Memoriam Professor Toshiyuki TAKESHIMA$",
"^Účetní závěrka ve vybraném podniku.*$",
"^Financial statements in selected company$",
"^Abdominal [Aa]ortic [Aa]neurysms.*$",
"^Pseudomyxoma peritonei$",
"^Kazalo autora$",
"(?i)^uvodna riječ$",
"^Motivace jako způsob vedení lidí$",
"^Motivation as a leadership$",
"^Polyfunkční dům$",
"^Multi\\-funkcional building$",
"^Podnikatelský plán$",
"(?i)^Podnikatelský záměr$",
"(?i)^Business Plan$",
"^Oceňování nemovitostí$",
"^Marketingová komunikace$",
"^Marketing communication$",
"^Sumario Analítico$",
"^Riječ uredništva$",
"^Savjetovanja i priredbe$",
"^Índice$",
"^(Starobosanski nadpisi).*$",
"^Vzdělávání pracovníků v organizaci$",
"^Staff training in organization$",
"^(Life Histories of North American Geometridae).*$",
"^Strategická analýza podniku$",
"^Strategic Analysis of an Enterprise$",
"^Sadržaj$",
"^Upute suradnicima$",
"^Rodinný dům$",
"(?i)^Fami(l)?ly house$",
"^Upute autorima$",
"^Strategic Analysis$",
"^Finanční analýza vybraného podniku$",
"^Finanční analýza$",
"^Riječ urednika$",
"(?i)^Content(s?)$",
"(?i)^Inhalt$",
"^Jinjō shōgaku shūshinsho jidōyō$",
"(?i)^Index$",
"^Chūgaku kokubun kyōkasho$",
"^Retrato de una mujer$",
"^Retrato de un hombre$",
"^Kōtō shōgaku dokuhon$",
"^Shotōka kokugo$",
"^Shōgaku dokuhon$",
"^Jinjō shōgaku kokugo dokuhon$",
"^Shinsei kokugo dokuhon$",
"^Teikoku dokuhon$",
"^Instructions to Authors$",
"^KİTAP TAHLİLİ$",
"^PRZEGLĄD PIŚMIENNICTWA$",
"(?i)^Presentación$",
"^İçindekiler$",
"(?i)^Tabl?e of contents$",
"^(CODICE DEL BEATO DE LOS REYES FERNANDO I Y SANCHA).*$",
"^(\\[MADRID\\. BIBL\\. NAC\\. N.*KING FERDINAND I.*FROM SAN ISIDORO DE LEON\\. FACUNDUS SCRIPSIT DATED.*\\]).*",
"^Editorial( Board)?$",
"(?i)^Editorial \\(English\\)$",
"^Editörden$",
"^(Corpus Oral Dialectal \\(COD\\)\\.).*$",
"^(Kiri Karl Morgensternile).*$",
"^(\\[Eksliibris Aleksandr).*\\]$",
"^(\\[Eksliibris Aleksandr).*$",
"^(Eksliibris Aleksandr).*$",
"^(Kiri A\\. de Vignolles).*$",
"^(2 kirja Karl Morgensternile).*$",
"^(Pirita kloostri idaosa arheoloogilised).*$",
"^(Kiri tundmatule).*$",
"^(Kiri Jenaer Allgemeine Literaturzeitung toimetusele).*$",
"^(Eksliibris Nikolai Birukovile).*$",
"^(Eksliibris Nikolai Issakovile).*$",
"^(WHP Cruise Summary Information of section).*$",
"^(Measurement of the top quark\\-pair production cross section with ATLAS in pp collisions at).*$",
"^(Measurement of the spin\\-dependent structure function).*",
"(?i)^.*authors[']? reply\\.?$",
"(?i)^.*authors[']? response\\.?$"
]
},
"synonyms": {}
}
}

View File

@ -1,5 +1,6 @@
<?xml version="1.0" encoding="UTF-8"?>
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
<parent>
<artifactId>dhp-workflows</artifactId>
<groupId>eu.dnetlib.dhp</groupId>
@ -11,6 +12,11 @@
<dependencies>
<dependency>
<groupId>commons-io</groupId>
<artifactId>commons-io</artifactId>
</dependency>
<dependency>
<groupId>org.apache.spark</groupId>
<artifactId>spark-core_2.11</artifactId>
@ -34,6 +40,10 @@
<groupId>com.jayway.jsonpath</groupId>
<artifactId>json-path</artifactId>
</dependency>
<dependency>
<groupId>org.mongodb</groupId>
<artifactId>mongo-java-driver</artifactId>
</dependency>
</dependencies>

View File

@ -0,0 +1,103 @@
package eu.dnetlib.dhp.graph;
import com.mongodb.*;
import com.mongodb.client.FindIterable;
import com.mongodb.client.MongoCollection;
import com.mongodb.client.MongoDatabase;
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
import eu.dnetlib.message.Message;
import eu.dnetlib.message.MessageType;
import org.apache.commons.io.IOUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.IntWritable;
import org.apache.hadoop.io.SequenceFile;
import org.apache.hadoop.io.Text;
import org.bson.Document;
import org.bson.conversions.Bson;
import java.io.IOException;
import java.net.URI;
import java.util.*;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.function.Consumer;
import java.util.stream.Collectors;
public class ImportDataFromMongo {
public static void main(String[] args) throws Exception {
final ArgumentApplicationParser parser = new ArgumentApplicationParser(IOUtils.toString(SparkGraphImporterJob.class.getResourceAsStream("/eu/dnetlib/dhp/graph/import_from_mongo_parameters.json")));
parser.parseArgument(args);
final int port = Integer.parseInt(parser.get("dbport"));
final String host = parser.get("dbhost");
final String format = parser.get("format");
final String layout = parser.get("layout");
final String interpretation = parser.get("interpretation");
final String dbName = parser.get("dbName");
final MongoClient client = new MongoClient(host, port);
MongoDatabase database = client.getDatabase(dbName);
MongoCollection<Document> metadata = database.getCollection("metadata");
MongoCollection<Document> metadataManager = database.getCollection("metadataManager");
final DBObject query = QueryBuilder.start("format").is(format).and("layout").is(layout).and("interpretation").is(interpretation).get();
final List<String> ids = new ArrayList<>();
metadata.find((Bson) query).forEach((Consumer<Document>) document -> ids.add(document.getString("mdId")));
List<String> databaseId = ids.stream().map(it -> getCurrentId(it, metadataManager)).filter(Objects::nonNull).collect(Collectors.toList());
final String hdfsuri = parser.get("namenode");
// ====== Init HDFS File System Object
Configuration conf = new Configuration();
// Set FileSystem URI
conf.set("fs.defaultFS", hdfsuri);
// Because of Maven
conf.set("fs.hdfs.impl", org.apache.hadoop.hdfs.DistributedFileSystem.class.getName());
conf.set("fs.file.impl", org.apache.hadoop.fs.LocalFileSystem.class.getName());
System.setProperty("HADOOP_USER_NAME", parser.get("user"));
System.setProperty("hadoop.home.dir", "/");
FileSystem.get(URI.create(hdfsuri), conf);
Path hdfswritepath = new Path(parser.get("targetPath"));
final AtomicInteger counter = new AtomicInteger(0);
try (SequenceFile.Writer writer = SequenceFile.createWriter(conf,
SequenceFile.Writer.file(hdfswritepath), SequenceFile.Writer.keyClass(IntWritable.class),
SequenceFile.Writer.valueClass(Text.class))) {
final IntWritable key = new IntWritable(counter.get());
final Text value = new Text();
databaseId.forEach(id -> {
System.out.println("Reading :"+id);
MongoCollection<Document> collection = database.getCollection(id);
collection.find().forEach((Consumer<Document>) document ->
{
key.set(counter.getAndIncrement());
value.set(document.getString("body"));
if (counter.get() % 10000 == 0) {
System.out.println("Added "+counter.get());
}
try {
writer.append(key, value);
} catch (IOException e) {
throw new RuntimeException(e);
}
}
);
});
}
}
private static String getCurrentId(final String mdId, final MongoCollection<Document> metadataManager) {
FindIterable<Document> result = metadataManager.find((Bson) QueryBuilder.start("mdId").is(mdId).get());
final Document item = result.first();
return item == null ? null : item.getString("currentId");
}
}

View File

@ -5,7 +5,6 @@ import com.fasterxml.jackson.databind.ObjectMapper;
import com.jayway.jsonpath.JsonPath;
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
import eu.dnetlib.dhp.graph.SparkGraphImporterJob;
import eu.dnetlib.dhp.schema.oaf.Oaf;
import eu.dnetlib.dhp.schema.oaf.Relation;
import eu.dnetlib.dhp.schema.scholexplorer.DLIDataset;
import eu.dnetlib.dhp.schema.scholexplorer.DLIPublication;
@ -17,10 +16,8 @@ import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.compress.GzipCodec;
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.Function2;
import org.apache.spark.api.java.function.PairFunction;
import org.apache.spark.sql.SparkSession;
import scala.Tuple2;

View File

@ -82,7 +82,7 @@ public abstract class AbstractScholexplorerParser {
}
protected String generateId(final String pid, final String pidType, final String entityType) {
String type = "50|";
String type;
switch (entityType){
case "publication":
type = "50|";
@ -100,7 +100,7 @@ public abstract class AbstractScholexplorerParser {
if ("dnet".equalsIgnoreCase(pidType))
return type+StringUtils.substringAfter(pid, "::");
return type+ DHPUtils.md5(String.format("%s::%s", pid, pidType));
return type+ DHPUtils.md5(String.format("%s::%s", pid.toLowerCase().trim(), pidType.toLowerCase().trim()));
}

View File

@ -11,6 +11,7 @@ import eu.dnetlib.dhp.schema.scholexplorer.ProvenaceInfo;
import eu.dnetlib.dhp.parser.utility.VtdUtilityParser.Node;
import org.apache.commons.lang3.StringUtils;
import org.apache.hadoop.yarn.webapp.hamlet.Hamlet;
import java.util.ArrayList;
import java.util.Arrays;
@ -37,10 +38,6 @@ public class DatasetScholexplorerParser extends AbstractScholexplorerParser {
di.setInvisible(false);
parsedObject.setDataInfo(di);
final String objIdentifier = VtdUtilityParser.getSingleValue(ap, vn, "//*[local-name()='objIdentifier']");
parsedObject.setId("60|" + StringUtils.substringAfter(objIdentifier, "::"));
parsedObject.setOriginalId(Collections.singletonList(VtdUtilityParser.getSingleValue(ap, vn, "//*[local-name()='recordIdentifier']")));
@ -112,12 +109,16 @@ public class DatasetScholexplorerParser extends AbstractScholexplorerParser {
final List<Node> identifierType =
VtdUtilityParser.getTextValuesWithAttributes(ap, vn, "//*[local-name()='resource']/*[local-name()='identifier']", Collections.singletonList("identifierType"));
StructuredProperty currentPid = extractIdentifier(identifierType, "type");
StructuredProperty currentPid = extractIdentifier(identifierType, "identifierType");
if (currentPid == null) return null;
inferPid(currentPid);
parsedObject.setPid(Collections.singletonList(currentPid));
final String sourceId = generateId(currentPid.getValue(), currentPid.getQualifier().getClassid(), "dataset");
parsedObject.setId(sourceId);
List<String> descs = VtdUtilityParser.getTextValue(ap, vn, "//*[local-name()='description']");
if (descs != null && descs.size() > 0)
parsedObject.setDescription(descs.stream()
@ -149,15 +150,20 @@ public class DatasetScholexplorerParser extends AbstractScholexplorerParser {
final String targetId = generateId(relatedPid, relatedPidType, relatedType);
r.setTarget(targetId);
r.setRelType(relationSemantic);
r.setRelClass("datacite");
r.setCollectedFrom(parsedObject.getCollectedfrom());
r.setDataInfo(di);
rels.add(r);
r = new Relation();
r.setDataInfo(di);
r.setSource(targetId);
r.setTarget(parsedObject.getId());
r.setRelType(inverseRelation);
r.setRelClass("datacite");
r.setCollectedFrom(parsedObject.getCollectedfrom());
rels.add(r);
result.add(createUnknownObject(relatedPid, relatedPidType, parsedObject.getCollectedfrom().get(0), di));
if("unknown".equalsIgnoreCase(relatedType))
result.add(createUnknownObject(relatedPid, relatedPidType, parsedObject.getCollectedfrom().get(0), di));
return rels.stream();
}).collect(Collectors.toList()));
}
@ -185,6 +191,13 @@ public class DatasetScholexplorerParser extends AbstractScholexplorerParser {
parsedObject.setSubject(subjects);
Qualifier q = new Qualifier();
q.setClassname("dataset");
q.setClassid("dataset");
q.setSchemename("dataset");
q.setSchemeid("dataset");
parsedObject.setResulttype(q);
parsedObject.setCompletionStatus(completionStatus);
final List<String> creators = VtdUtilityParser.getTextValue(ap, vn, "//*[local-name()='resource']//*[local-name()='creator']/*[local-name()='creatorName']");

View File

@ -36,9 +36,6 @@ public class PublicationScholexplorerParser extends AbstractScholexplorerParser
di.setDeletedbyinference(false);
di.setInvisible(false);
final String objIdentifier = VtdUtilityParser.getSingleValue(ap, vn, "//*[local-name()='objIdentifier']");
parsedObject.setId("50|" + StringUtils.substringAfter(objIdentifier, "::"));
parsedObject.setDateofcollection(VtdUtilityParser.getSingleValue(ap, vn, "//*[local-name()='dateOfCollection']"));
final String resolvedDate = VtdUtilityParser.getSingleValue(ap, vn, "//*[local-name()='resolvedDate']");
@ -63,6 +60,8 @@ public class PublicationScholexplorerParser extends AbstractScholexplorerParser
if (currentPid == null) return null;
inferPid(currentPid);
parsedObject.setPid(Collections.singletonList(currentPid));
final String sourceId = generateId(currentPid.getValue(), currentPid.getQualifier().getClassid(), "publication");
parsedObject.setId(sourceId);
String provisionMode = VtdUtilityParser.getSingleValue(ap, vn, "//*[local-name()='provisionMode']");
@ -136,12 +135,12 @@ public class PublicationScholexplorerParser extends AbstractScholexplorerParser
r.setDataInfo(di);
rels.add(r);
r = new Relation();
r.setDataInfo(di);
r.setSource(targetId);
r.setTarget(parsedObject.getId());
r.setRelType(inverseRelation);
r.setCollectedFrom(parsedObject.getCollectedfrom());
r.setDataInfo(di);
r.setRelClass("datacite");
r.setCollectedFrom(parsedObject.getCollectedfrom());
rels.add(r);
return rels.stream();
@ -217,7 +216,13 @@ public class PublicationScholexplorerParser extends AbstractScholexplorerParser
parsedObject.setDataInfo(di);
parsedObject.setSubject(subjects);
Qualifier q = new Qualifier();
q.setClassname("publication");
q.setClassid("publication");
q.setSchemename("publication");
q.setSchemeid("publication");
parsedObject.setResulttype(q);
result.add(parsedObject);
return result;

View File

@ -8,10 +8,6 @@
<name>targetPath</name>
<description>the source path</description>
</property>
<property>
<name>targetDir</name>
<description>the name of the path</description>
</property>
<property>
<name>sparkDriverMemory</name>
<description>memory for driver process</description>
@ -26,15 +22,22 @@
</property>
<property>
<name>entity</name>
<description>the entity to be merged</description>
<description>the entity type</description>
</property>
</parameters>
<start to="MapGraphIntoDataFrame"/>
<start to="DeleteTargetPath"/>
<kill name="Kill">
<message>Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}]</message>
</kill>
<action name="DeleteTargetPath">
<fs>
<delete path='${targetPath}'/>
</fs>
<ok to="MapGraphIntoDataFrame"/>
<error to="Kill"/>
</action>
<action name="MapGraphIntoDataFrame">
<spark xmlns="uri:oozie:spark-action:0.2">
@ -42,15 +45,10 @@
<name-node>${nameNode}</name-node>
<master>yarn-cluster</master>
<mode>cluster</mode>
<name>Merge ${entity}</name>
<class>eu.dnetlib.dhp.graph.scholexplorer.SparkScholexplorerMergeEntitiesJob</class>
<name>Import ${entity} and related entities</name>
<class>eu.dnetlib.dhp.graph.scholexplorer.SparkScholexplorerGraphImporter</class>
<jar>dhp-graph-mapper-${projectVersion}.jar</jar>
<spark-opts>
--executor-memory ${sparkExecutorMemory}
--driver-memory=${sparkDriverMemory}
--num-executors 100
--conf spark.yarn.jars="hdfs://hadoop-rm1.garr-pa1.d4science.org:8020/user/oozie/share/lib/lib_20180405103059/spark2"
</spark-opts>
<spark-opts>--executor-memory ${sparkExecutorMemory} --driver-memory=${sparkDriverMemory} ${sparkExtraOPT}</spark-opts>
<arg>-mt</arg> <arg>yarn-cluster</arg>
<arg>--sourcePath</arg><arg>${sourcePath}</arg>
<arg>--targetPath</arg><arg>${targetPath}</arg>

View File

@ -20,23 +20,34 @@
<name>sparkExecutorMemory</name>
<description>memory for individual executor</description>
</property>
<property>
<name>sparkExecutorCores</name>
<description>number of cores used by single executor</description>
</property>
<property>
<name>entities</name>
<description>the entities to be extracted</description>
</property>
</parameters>
<start to="MapGraphIntoDataFrame"/>
<start to="DeleteTargetPath"/>
<kill name="Kill">
<message>Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}]</message>
</kill>
<action name="DeleteTargetPath">
<fs>
<mkdir path="${targetPath}"/>
<mkdir path="${targetPath}/dataset"/>
<mkdir path="${targetPath}/publication"/>
<mkdir path="${targetPath}/unknown"/>
<mkdir path="${targetPath}/relation"/>
<delete path='${targetPath}/dataset/${targetDir}'/>
<delete path='${targetPath}/publication/${targetDir}'/>
<delete path='${targetPath}/unknown/${targetDir}'/>
<delete path='${targetPath}/relation/${targetDir}'/>
</fs>
<ok to="ExtractDLIEntities"/>
<error to="Kill"/>
</action>
<action name="MapGraphIntoDataFrame">
<action name="ExtractDLIEntities">
<spark xmlns="uri:oozie:spark-action:0.2">
<job-tracker>${jobTracker}</job-tracker>
<name-node>${nameNode}</name-node>
@ -47,12 +58,8 @@
<jar>dhp-graph-mapper-${projectVersion}.jar</jar>
<spark-opts>
--executor-memory ${sparkExecutorMemory}
--driver-memory=${sparkDriverMemory}
--num-executors 100
<!-- &#45;&#45;conf spark.extraListeners="com.cloudera.spark.lineage.NavigatorAppListener"-->
<!-- &#45;&#45;conf spark.sql.queryExecutionListeners="com.cloudera.spark.lineage.NavigatorQueryListener"-->
<!-- &#45;&#45;conf spark.sql.warehouse.dir="/user/hive/warehouse"-->
--conf spark.yarn.jars="hdfs://hadoop-rm1.garr-pa1.d4science.org:8020/user/oozie/share/lib/lib_20180405103059/spark2"
--driver-memory=${sparkDriverMemory}
${sparkExtraOPT}
</spark-opts>
<arg>-mt</arg> <arg>yarn-cluster</arg>
<arg>--sourcePath</arg><arg>${sourcePath}</arg>

View File

@ -0,0 +1,73 @@
<workflow-app name="import Entities from aggretor to HDFS" xmlns="uri:oozie:workflow:0.5">
<parameters>
<property>
<name>workingPath</name>
<description>the working dir base path</description>
</property>
<property>
<name>targetPath</name>
<description>the graph Raw base path</description>
</property>
<property>
<name>format</name>
<description>the postgres URL to access to the database</description>
</property>
<property>
<name>layout</name>
<description>the user postgres</description>
</property>
<property>
<name>interpretation</name>
<description>the password postgres</description>
</property>
<property>
<name>dbhost</name>
<description>mongoDB url, example: mongodb://[username:password@]host[:port]</description>
</property>
<property>
<name>dbName</name>
<description>mongo database</description>
</property>
<property>
<name>user</name>
<description>HDFS user</description>
</property>
</parameters>
<start to="ResetWorkingPath"/>
<kill name="Kill">
<message>Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}]</message>
</kill>
<action name="ResetWorkingPath">
<fs>
<delete path='${targetPath}'/>
<mkdir path='${workingPath}'/>
</fs>
<ok to="ImportEntitiesFromMongo"/>
<error to="Kill"/>
</action>
<action name="ImportEntitiesFromMongo">
<java>
<job-tracker>${jobTracker}</job-tracker>
<name-node>${nameNode}</name-node>
<main-class>eu.dnetlib.dhp.graph.ImportDataFromMongo</main-class>
<arg>-t</arg><arg>${targetPath}</arg>
<arg>-n</arg><arg>${nameNode}</arg>
<arg>-u</arg><arg>${user}</arg>
<arg>-h</arg><arg>${dbhost}</arg>
<arg>-p</arg><arg>27017</arg>
<arg>-dn</arg><arg>${dbName}</arg>
<arg>-f</arg><arg>${format}</arg>
<arg>-l</arg><arg>${layout}</arg>
<arg>-i</arg><arg>${interpretation}</arg>
</java>
<ok to="End"/>
<error to="Kill"/>
</action>
<end name="End"/>
</workflow-app>

View File

@ -0,0 +1,10 @@
<configuration>
<property>
<name>oozie.use.system.libpath</name>
<value>true</value>
</property>
<property>
<name>oozie.action.sharelib.for.spark</name>
<value>spark2</value>
</property>
</configuration>

View File

@ -16,43 +16,41 @@
<name>sparkExecutorMemory</name>
<description>memory for individual executor</description>
</property>
<property>
<name>sparkExecutorCores</name>
<description>number of cores used by single executor</description>
</property>
<property>
<name>entity</name>
<description>the entity type</description>
<description>the entity to be merged</description>
</property>
</parameters>
<start to="MapGraphIntoDataFrame"/>
<start to="DeleteTargetPath"/>
<kill name="Kill">
<message>Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}]</message>
</kill>
<action name="MapGraphIntoDataFrame">
<action name="DeleteTargetPath">
<fs>
<mkdir path="${targetPath}"/>
<delete path='${targetPath}/${entity}'/>
</fs>
<ok to="MergeDLIEntities"/>
<error to="Kill"/>
</action>
<action name="MergeDLIEntities">
<spark xmlns="uri:oozie:spark-action:0.2">
<job-tracker>${jobTracker}</job-tracker>
<name-node>${nameNode}</name-node>
<master>yarn-cluster</master>
<mode>cluster</mode>
<name>Import ${entity} and related entities</name>
<class>eu.dnetlib.dhp.graph.scholexplorer.SparkScholexplorerGraphImporter</class>
<name>Merge ${entity}</name>
<class>eu.dnetlib.dhp.graph.scholexplorer.SparkScholexplorerMergeEntitiesJob</class>
<jar>dhp-graph-mapper-${projectVersion}.jar</jar>
<spark-opts>
--executor-memory ${sparkExecutorMemory}
--driver-memory=${sparkDriverMemory}
--num-executors 100
<!-- &#45;&#45;conf spark.extraListeners="com.cloudera.spark.lineage.NavigatorAppListener"-->
<!-- &#45;&#45;conf spark.sql.queryExecutionListeners="com.cloudera.spark.lineage.NavigatorQueryListener"-->
<!-- &#45;&#45;conf spark.sql.warehouse.dir="/user/hive/warehouse"-->
--conf spark.yarn.jars="hdfs://hadoop-rm1.garr-pa1.d4science.org:8020/user/oozie/share/lib/lib_20180405103059/spark2"
</spark-opts>
<spark-opts> --executor-memory ${sparkExecutorMemory} --driver-memory=${sparkDriverMemory} ${sparkExtraOPT}</spark-opts>
<arg>-mt</arg> <arg>yarn-cluster</arg>
<arg>--sourcePath</arg><arg>${sourcePath}</arg>
<arg>--targetPath</arg><arg>${targetPath}</arg>
<arg>--sourcePath</arg><arg>${sourcePath}/${entity}</arg>
<arg>--targetPath</arg><arg>${targetPath}/${entity}</arg>
<arg>--entity</arg><arg>${entity}</arg>
</spark>
<ok to="End"/>

View File

@ -0,0 +1,12 @@
[
{"paramName":"n", "paramLongName":"namenode", "paramDescription": "the name node", "paramRequired": true},
{"paramName":"u", "paramLongName":"user", "paramDescription": "the name node", "paramRequired": true},
{"paramName":"t", "paramLongName":"targetPath", "paramDescription": "the name node", "paramRequired": true},
{"paramName":"h", "paramLongName":"dbhost", "paramDescription": "the mongo host", "paramRequired": true},
{"paramName":"p", "paramLongName":"dbport", "paramDescription": "the mongo port", "paramRequired": true},
{"paramName":"f", "paramLongName":"format", "paramDescription": "the metadata format to import", "paramRequired": true},
{"paramName":"l", "paramLongName":"layout", "paramDescription": "the metadata layout to import", "paramRequired": true},
{"paramName":"i", "paramLongName":"interpretation", "paramDescription": "the metadata interpretation to import", "paramRequired": true},
{"paramName":"dn", "paramLongName":"dbName", "paramDescription": "the database Name", "paramRequired": true}
]

View File

@ -1,51 +0,0 @@
<workflow-app name="import_infospace_graph" xmlns="uri:oozie:workflow:0.5">
<parameters>
<property>
<name>sourcePath</name>
<description>the source path</description>
</property>
<property>
<name>hive_db_name</name>
<description>the target hive database name</description>
</property>
<property>
<name>sparkDriverMemory</name>
<description>memory for driver process</description>
</property>
<property>
<name>sparkExecutorMemory</name>
<description>memory for individual executor</description>
</property>
<property>
<name>sparkExecutorCores</name>
<description>number of cores used by single executor</description>
</property>
</parameters>
<start to="MapGraphIntoDataFrame"/>
<kill name="Kill">
<message>Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}]</message>
</kill>
<action name="MapGraphIntoDataFrame">
<spark xmlns="uri:oozie:spark-action:0.2">
<job-tracker>${jobTracker}</job-tracker>
<name-node>${nameNode}</name-node>
<master>yarn-cluster</master>
<mode>cluster</mode>
<name>MapGraphIntoDataFrame</name>
<class>eu.dnetlib.dhp.graph.SparkGraphImporterJob</class>
<jar>dhp-graph-mapper-${projectVersion}.jar</jar>
<spark-opts>--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"</spark-opts>
<arg>-mt</arg> <arg>yarn-cluster</arg>
<arg>--sourcePath</arg><arg>${sourcePath}</arg>
<arg>--hive_db_name</arg><arg>${hive_db_name}</arg>
<arg>--hive_metastore_uris</arg><arg>${hive_metastore_uris}</arg>
</spark>
<ok to="End"/>
<error to="Kill"/>
</action>
<end name="End"/>
</workflow-app>

View File

@ -0,0 +1,22 @@
package eu.dnetlib.dhp.graph;
import org.junit.Test;
public class ImportDataFromMongoTest {
@Test
public void doTest() throws Exception {
ImportDataFromMongo.main(new String[] {
"-h", "localhost",
"-p", "2800",
"-f", "PMF",
"-l", "store",
"-i", "cleaned",
"-dn", "mdstore_dli",
"-n", "file:///home/sandro/test.seq",
"-u", "sandro",
"-t", "file:///home/sandro/test.seq"
});
}
}

View File

@ -0,0 +1,38 @@
package eu.dnetlib.dhp.graph.scholexplorer;
import com.fasterxml.jackson.core.JsonProcessingException;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.databind.SerializationFeature;
import eu.dnetlib.dhp.graph.scholexplorer.parser.DatasetScholexplorerParser;
import eu.dnetlib.dhp.schema.oaf.Oaf;
import org.apache.commons.io.IOUtils;
import org.junit.Test;
import java.io.IOException;
import java.util.List;
public class ScholexplorerParserTest {
@Test
public void testDataciteParser() throws IOException {
String xml = IOUtils.toString(this.getClass().getResourceAsStream("dmf.xml"));
DatasetScholexplorerParser p = new DatasetScholexplorerParser();
List<Oaf> oaves = p.parseObject(xml);
ObjectMapper m = new ObjectMapper();
m.enable(SerializationFeature.INDENT_OUTPUT);
oaves.forEach(oaf -> {
try {
System.out.println(m.writeValueAsString(oaf));
System.out.println("----------------------------");
} catch (JsonProcessingException e) {
}
});
}
}

View File

@ -0,0 +1,66 @@
<?xml version="1.0" encoding="UTF-8"?>
<oai:record xmlns:oai="http://www.openarchives.org/OAI/2.0/"
xmlns:oaf="http://namespace.openaire.eu/oaf"
xmlns:dri="http://www.driver-repository.eu/namespace/dri"
xmlns:dc="http://purl.org/dc/elements/1.1/">
<oai:header>
<dri:repositoryId>aaadf8b3-01a8-4cc2-9964-63cfb19df3b4_UmVwb3NpdG9yeVNlcnZpY2VSZXNvdXJjZXMvUmVwb3NpdG9yeVNlcnZpY2VSZXNvdXJjZVR5cGU=</dri:repositoryId>
<dri:recordIdentifier>oai:pangaea.de:doi:10.1594/PANGAEA.821876</dri:recordIdentifier>
<dri:datasourceprefix>r3d100010134</dri:datasourceprefix>
<dri:objIdentifier>r3d100010134::000083be706192d2d839915694ecfd47</dri:objIdentifier>
<dri:resolvedDate>2020-01-08T04:12:12.287</dri:resolvedDate>
<dri:dateOfCollection>2020-01-08T03:24:10.865Z</dri:dateOfCollection>
<oaf:datasourceprefix/>
<identifier>oai:pangaea.de:doi:10.1594/PANGAEA.821876</identifier>
<setSpec>citable</setSpec>
</oai:header>
<metadata>
<resource xmlns="http://datacite.org/schema/kernel-3" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://datacite.org/schema/kernel-3 http://schema.datacite.org/meta/kernel-3/metadata.xsd">
<identifier identifierType="doi">10.1594/pangaea.821876</identifier>
<creators> <creator><creatorName>Macke, Andreas</creatorName></creator><creator><creatorName>Kalisch, John</creatorName></creator> </creators>
<titles> <title>Total Sky Imager observations during POLARSTERN cruise ANT-XXVI/4 on 2010-05-14 with links to images</title> </titles>
<publisher>PANGAEA - Data Publisher for Earth &amp; Environmental Science</publisher>
<dates>
<date dateType="Collected">2010-05-14T00:13:47/2010-05-14T23:55:47</date>
</dates>
<subjects>
<subject subjectScheme="Parameter">DATE/TIME</subject>
<subject subjectScheme="Parameter">LATITUDE</subject>
<subject subjectScheme="Parameter">LONGITUDE</subject>
<subject subjectScheme="Parameter">Uniform resource locator/link to image</subject>
<subject subjectScheme="Method">Total Sky Imager</subject>
<subject subjectScheme="Campaign">ANT-XXVI/4</subject>
<subject subjectScheme="Basis">Polarstern</subject>
</subjects>
<resourceType resourceTypeGeneral="dataset">dataset</resourceType>
<relatedIdentifiers>
<relatedIdentifier relatedIdentifierType="dnet" relationType="isPartOf" inverseRelationType="hasPart" entityType="dataset">dli_resolver::cf447a378b0b6603593f8b0e57242695</relatedIdentifier>
<relatedIdentifier relatedIdentifierType="URL" relationType="references" inverseRelationType="isReferencedBy" entityType="unknown">http://hs.pangaea.de/images/airphoto/ps/ps75/2010-05-14/ant-xxvi_4_2010-05-14_tsi-images-links.zip</relatedIdentifier>
<relatedIdentifier relatedIdentifierType="dnet" relationType="references" inverseRelationType="isReferencedBy" entityType="publication">dli_resolver::f0f5975d20991cffd222c6002ddd5821</relatedIdentifier>
</relatedIdentifiers>
</resource>
</metadata>
<oaf:about xmlns:oaf="http://namespace.dnet.eu/oaf">
<oaf:datainfo >
<oaf:completionStatus>complete</oaf:completionStatus>
<oaf:collectedFrom id="dli_________::r3d100010134" name="Pangaea" completionStatus="complete"/>
</oaf:datainfo>
</oaf:about>
</oai:record>

10
pom.xml
View File

@ -138,6 +138,12 @@
<artifactId>commons-io</artifactId>
<version>2.4</version>
</dependency>
<dependency>
<groupId>org.mongodb</groupId>
<artifactId>mongo-java-driver</artifactId>
<version>3.4.2</version>
</dependency>
<dependency>
<groupId>commons-cli</groupId>
@ -200,7 +206,7 @@
<dependency>
<groupId>eu.dnetlib</groupId>
<artifactId>dnet-pace-core</artifactId>
<version>4.0.0-SNAPSHOT</version>
<version>4.0.0</version>
</dependency>
@ -418,7 +424,7 @@
<project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
<project.reporting.outputEncoding>UTF-8</project.reporting.outputEncoding>
<maven.compiler.plugin.version>3.6.0</maven.compiler.plugin.version>
<maven.failsave.plugin.version>2.22.2</maven.failsave.plugin.version>
<maven.failsave.plugin.version>2.22.2</maven.failsave.plugin.version>
<dhp.cdh.version>cdh5.9.2</dhp.cdh.version>
<dhp.hadoop.version>2.6.0-${dhp.cdh.version}</dhp.hadoop.version>
<dhp.oozie.version>4.1.0-${dhp.cdh.version}</dhp.oozie.version>