forked from antonis.lempesis/dnet-hadoop
refactoring code
This commit is contained in:
parent
ad4387dd38
commit
2b8675462f
|
@ -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>
|
||||
|
|
|
@ -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 "";
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -45,6 +45,7 @@
|
|||
<artifactId>jaxen</artifactId>
|
||||
</dependency>
|
||||
|
||||
|
||||
<dependency>
|
||||
<groupId>org.mockito</groupId>
|
||||
<artifactId>mockito-core</artifactId>
|
||||
|
|
|
@ -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();
|
||||
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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");
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -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));
|
||||
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
|
||||
|
||||
}
|
||||
|
||||
|
||||
}
|
|
@ -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
|
||||
}
|
||||
]
|
|
@ -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
|
||||
}
|
||||
]
|
|
@ -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>
|
|
@ -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>
|
|
@ -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>
|
|
@ -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>
|
|
@ -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 {
|
||||
|
|
|
@ -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$",
|
||||
"^(à l’exception 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": {}
|
||||
}
|
||||
}
|
|
@ -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>
|
||||
|
||||
|
|
|
@ -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");
|
||||
}
|
||||
|
||||
}
|
|
@ -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;
|
||||
|
|
|
@ -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()));
|
||||
}
|
||||
|
||||
|
||||
|
|
|
@ -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']");
|
||||
|
|
|
@ -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;
|
||||
|
||||
|
|
|
@ -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>
|
|
@ -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
|
||||
<!-- --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"-->
|
||||
--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>
|
|
@ -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>
|
|
@ -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>
|
|
@ -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
|
||||
<!-- --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"-->
|
||||
--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"/>
|
|
@ -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}
|
||||
|
||||
]
|
|
@ -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>
|
|
@ -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"
|
||||
});
|
||||
}
|
||||
|
||||
}
|
|
@ -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) {
|
||||
|
||||
}
|
||||
});
|
||||
|
||||
}
|
||||
}
|
|
@ -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 & 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
10
pom.xml
|
@ -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>
|
||||
|
|
Loading…
Reference in New Issue