forked from D-Net/dnet-hadoop
Compare commits
38 Commits
master
...
SWH_integr
Author | SHA1 | Date |
---|---|---|
Claudio Atzori | 3c23d5f9bc | |
Claudio Atzori | 858931ccb6 | |
Claudio Atzori | f759b18bca | |
Claudio Atzori | eed9fe0902 | |
Claudio Atzori | 7f27111b1f | |
Claudio Atzori | 73c49b8d26 | |
Sandro La Bruzzo | 13f332ce77 | |
Serafeim Chatzopoulos | 1bb83b9188 | |
Claudio Atzori | ee8a39e7d2 | |
Serafeim Chatzopoulos | e9f24df21c | |
Serafeim Chatzopoulos | cae75fc75d | |
Serafeim Chatzopoulos | b49a3ac9b2 | |
Serafeim Chatzopoulos | 24c43e0c60 | |
Serafeim Chatzopoulos | 9f73d93e62 | |
Claudio Atzori | f344ad76d0 | |
Claudio Atzori | 5919e488dd | |
Serafeim Chatzopoulos | 839a8524e7 | |
Miriam Baglioni | d7fccdc64b | |
Miriam Baglioni | 9898470b0e | |
Giambattista Bloisi | c412dc162b | |
Claudio Atzori | 5d09b7db8b | |
Claudio Atzori | 7b403a920f | |
Claudio Atzori | dc86018a5f | |
Giambattista Bloisi | 3c47920c78 | |
Claudio Atzori | 7f244d9a7a | |
Giambattista Bloisi | e239b81740 | |
Miriam Baglioni | e84f5b5e64 | |
Serafeim Chatzopoulos | ab0d70691c | |
Serafeim Chatzopoulos | ed9c81a0b7 | |
Alessia Bardi | 0935d7757c | |
Alessia Bardi | cc7204a089 | |
Sandro La Bruzzo | 76476cdfb6 | |
Serafeim Chatzopoulos | 9d44418d38 | |
Serafeim Chatzopoulos | 395a4af020 | |
Claudio Atzori | 8a6892cc63 | |
Claudio Atzori | 4786aa0e09 | |
Giambattista Bloisi | 2caaaec42d | |
Giambattista Bloisi | 6cc7d8ca7b |
|
@ -26,3 +26,4 @@ spark-warehouse
|
|||
/**/*.log
|
||||
/**/.factorypath
|
||||
/**/.scalafmt.conf
|
||||
/.java-version
|
||||
|
|
|
@ -51,6 +51,7 @@ public class Constants {
|
|||
public static final String RETRY_DELAY = "retryDelay";
|
||||
public static final String CONNECT_TIMEOUT = "connectTimeOut";
|
||||
public static final String READ_TIMEOUT = "readTimeOut";
|
||||
public static final String REQUEST_METHOD = "requestMethod";
|
||||
public static final String FROM_DATE_OVERRIDE = "fromDateOverride";
|
||||
public static final String UNTIL_DATE_OVERRIDE = "untilDateOverride";
|
||||
|
||||
|
|
|
@ -1,6 +1,9 @@
|
|||
|
||||
package eu.dnetlib.dhp.common.collection;
|
||||
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
* Bundles the http connection parameters driving the client behaviour.
|
||||
*/
|
||||
|
@ -13,6 +16,8 @@ public class HttpClientParams {
|
|||
public static int _connectTimeOut = 10; // seconds
|
||||
public static int _readTimeOut = 30; // seconds
|
||||
|
||||
public static String _requestMethod = "GET";
|
||||
|
||||
/**
|
||||
* Maximum number of allowed retires before failing
|
||||
*/
|
||||
|
@ -38,17 +43,30 @@ public class HttpClientParams {
|
|||
*/
|
||||
private int readTimeOut;
|
||||
|
||||
/**
|
||||
* Custom http headers
|
||||
*/
|
||||
private Map<String, String> headers;
|
||||
|
||||
/**
|
||||
* Request method (i.e., GET, POST etc)
|
||||
*/
|
||||
private String requestMethod;
|
||||
|
||||
public HttpClientParams() {
|
||||
this(_maxNumberOfRetry, _requestDelay, _retryDelay, _connectTimeOut, _readTimeOut);
|
||||
this(_maxNumberOfRetry, _requestDelay, _retryDelay, _connectTimeOut, _readTimeOut, new HashMap<>(),
|
||||
_requestMethod);
|
||||
}
|
||||
|
||||
public HttpClientParams(int maxNumberOfRetry, int requestDelay, int retryDelay, int connectTimeOut,
|
||||
int readTimeOut) {
|
||||
int readTimeOut, Map<String, String> headers, String requestMethod) {
|
||||
this.maxNumberOfRetry = maxNumberOfRetry;
|
||||
this.requestDelay = requestDelay;
|
||||
this.retryDelay = retryDelay;
|
||||
this.connectTimeOut = connectTimeOut;
|
||||
this.readTimeOut = readTimeOut;
|
||||
this.headers = headers;
|
||||
this.requestMethod = requestMethod;
|
||||
}
|
||||
|
||||
public int getMaxNumberOfRetry() {
|
||||
|
@ -91,4 +109,19 @@ public class HttpClientParams {
|
|||
this.readTimeOut = readTimeOut;
|
||||
}
|
||||
|
||||
public Map<String, String> getHeaders() {
|
||||
return headers;
|
||||
}
|
||||
|
||||
public void setHeaders(Map<String, String> headers) {
|
||||
this.headers = headers;
|
||||
}
|
||||
|
||||
public String getRequestMethod() {
|
||||
return requestMethod;
|
||||
}
|
||||
|
||||
public void setRequestMethod(String requestMethod) {
|
||||
this.requestMethod = requestMethod;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -107,7 +107,14 @@ public class HttpConnector2 {
|
|||
urlConn.setReadTimeout(getClientParams().getReadTimeOut() * 1000);
|
||||
urlConn.setConnectTimeout(getClientParams().getConnectTimeOut() * 1000);
|
||||
urlConn.addRequestProperty(HttpHeaders.USER_AGENT, userAgent);
|
||||
urlConn.setRequestMethod(getClientParams().getRequestMethod());
|
||||
|
||||
// if provided, add custom headers
|
||||
if (!getClientParams().getHeaders().isEmpty()) {
|
||||
for (Map.Entry<String, String> headerEntry : getClientParams().getHeaders().entrySet()) {
|
||||
urlConn.addRequestProperty(headerEntry.getKey(), headerEntry.getValue());
|
||||
}
|
||||
}
|
||||
if (log.isDebugEnabled()) {
|
||||
logHeaderFields(urlConn);
|
||||
}
|
||||
|
|
|
@ -1,98 +0,0 @@
|
|||
|
||||
package eu.dnetlib.dhp.oa.merge;
|
||||
|
||||
import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession;
|
||||
|
||||
import java.util.Objects;
|
||||
import java.util.Optional;
|
||||
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.commons.lang3.StringUtils;
|
||||
import org.apache.spark.SparkConf;
|
||||
import org.apache.spark.api.java.function.FilterFunction;
|
||||
import org.apache.spark.api.java.function.MapFunction;
|
||||
import org.apache.spark.sql.*;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
|
||||
import eu.dnetlib.dhp.common.HdfsSupport;
|
||||
import eu.dnetlib.dhp.schema.common.ModelSupport;
|
||||
|
||||
public class DispatchEntitiesSparkJob {
|
||||
|
||||
private static final Logger log = LoggerFactory.getLogger(DispatchEntitiesSparkJob.class);
|
||||
|
||||
public static void main(String[] args) throws Exception {
|
||||
|
||||
String jsonConfiguration = IOUtils
|
||||
.toString(
|
||||
Objects
|
||||
.requireNonNull(
|
||||
DispatchEntitiesSparkJob.class
|
||||
.getResourceAsStream(
|
||||
"/eu/dnetlib/dhp/oa/merge/dispatch_entities_parameters.json")));
|
||||
final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration);
|
||||
parser.parseArgument(args);
|
||||
|
||||
Boolean isSparkSessionManaged = Optional
|
||||
.ofNullable(parser.get("isSparkSessionManaged"))
|
||||
.map(Boolean::valueOf)
|
||||
.orElse(Boolean.TRUE);
|
||||
log.info("isSparkSessionManaged: {}", isSparkSessionManaged);
|
||||
|
||||
String inputPath = parser.get("inputPath");
|
||||
log.info("inputPath: {}", inputPath);
|
||||
|
||||
String outputPath = parser.get("outputPath");
|
||||
log.info("outputPath: {}", outputPath);
|
||||
|
||||
boolean filterInvisible = Boolean.valueOf(parser.get("filterInvisible"));
|
||||
log.info("filterInvisible: {}", filterInvisible);
|
||||
|
||||
SparkConf conf = new SparkConf();
|
||||
runWithSparkSession(
|
||||
conf,
|
||||
isSparkSessionManaged,
|
||||
spark -> {
|
||||
HdfsSupport.remove(outputPath, spark.sparkContext().hadoopConfiguration());
|
||||
dispatchEntities(spark, inputPath, outputPath, filterInvisible);
|
||||
});
|
||||
}
|
||||
|
||||
private static void dispatchEntities(
|
||||
SparkSession spark,
|
||||
String inputPath,
|
||||
String outputPath,
|
||||
boolean filterInvisible) {
|
||||
|
||||
Dataset<String> df = spark.read().textFile(inputPath);
|
||||
|
||||
ModelSupport.oafTypes.entrySet().parallelStream().forEach(entry -> {
|
||||
String entityType = entry.getKey();
|
||||
Class<?> clazz = entry.getValue();
|
||||
|
||||
if (!entityType.equalsIgnoreCase("relation")) {
|
||||
Dataset<Row> entityDF = spark
|
||||
.read()
|
||||
.schema(Encoders.bean(clazz).schema())
|
||||
.json(
|
||||
df
|
||||
.filter((FilterFunction<String>) s -> s.startsWith(clazz.getName()))
|
||||
.map(
|
||||
(MapFunction<String, String>) s -> StringUtils.substringAfter(s, "|"),
|
||||
Encoders.STRING()));
|
||||
|
||||
if (filterInvisible) {
|
||||
entityDF = entityDF.filter("dataInfo.invisible != true");
|
||||
}
|
||||
|
||||
entityDF
|
||||
.write()
|
||||
.mode(SaveMode.Overwrite)
|
||||
.option("compression", "gzip")
|
||||
.json(outputPath + "/" + entityType);
|
||||
}
|
||||
});
|
||||
}
|
||||
}
|
|
@ -2,36 +2,28 @@
|
|||
package eu.dnetlib.dhp.oa.merge;
|
||||
|
||||
import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession;
|
||||
import static eu.dnetlib.dhp.utils.DHPUtils.toSeq;
|
||||
import static org.apache.spark.sql.functions.col;
|
||||
import static org.apache.spark.sql.functions.when;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
import java.util.Objects;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
import java.util.concurrent.ForkJoinPool;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.commons.lang3.StringUtils;
|
||||
import org.apache.spark.SparkConf;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.apache.spark.api.java.function.FilterFunction;
|
||||
import org.apache.spark.api.java.function.MapFunction;
|
||||
import org.apache.spark.api.java.function.ReduceFunction;
|
||||
import org.apache.spark.sql.*;
|
||||
import org.apache.spark.sql.expressions.Aggregator;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import com.fasterxml.jackson.databind.DeserializationFeature;
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.jayway.jsonpath.Configuration;
|
||||
import com.jayway.jsonpath.DocumentContext;
|
||||
import com.jayway.jsonpath.JsonPath;
|
||||
import com.jayway.jsonpath.Option;
|
||||
|
||||
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
|
||||
import eu.dnetlib.dhp.common.HdfsSupport;
|
||||
import eu.dnetlib.dhp.schema.common.EntityType;
|
||||
import eu.dnetlib.dhp.schema.common.ModelSupport;
|
||||
import eu.dnetlib.dhp.schema.oaf.*;
|
||||
import eu.dnetlib.dhp.schema.oaf.OafEntity;
|
||||
import eu.dnetlib.dhp.schema.oaf.utils.OafMapperUtils;
|
||||
import scala.Tuple2;
|
||||
|
||||
|
@ -39,13 +31,9 @@ import scala.Tuple2;
|
|||
* Groups the graph content by entity identifier to ensure ID uniqueness
|
||||
*/
|
||||
public class GroupEntitiesSparkJob {
|
||||
|
||||
private static final Logger log = LoggerFactory.getLogger(GroupEntitiesSparkJob.class);
|
||||
|
||||
private static final String ID_JPATH = "$.id";
|
||||
|
||||
private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper()
|
||||
.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
|
||||
private static final Encoder<OafEntity> OAFENTITY_KRYO_ENC = Encoders.kryo(OafEntity.class);
|
||||
|
||||
public static void main(String[] args) throws Exception {
|
||||
|
||||
|
@ -66,9 +54,15 @@ public class GroupEntitiesSparkJob {
|
|||
String graphInputPath = parser.get("graphInputPath");
|
||||
log.info("graphInputPath: {}", graphInputPath);
|
||||
|
||||
String checkpointPath = parser.get("checkpointPath");
|
||||
log.info("checkpointPath: {}", checkpointPath);
|
||||
|
||||
String outputPath = parser.get("outputPath");
|
||||
log.info("outputPath: {}", outputPath);
|
||||
|
||||
boolean filterInvisible = Boolean.valueOf(parser.get("filterInvisible"));
|
||||
log.info("filterInvisible: {}", filterInvisible);
|
||||
|
||||
SparkConf conf = new SparkConf();
|
||||
conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer");
|
||||
conf.registerKryoClasses(ModelSupport.getOafModelClasses());
|
||||
|
@ -78,126 +72,95 @@ public class GroupEntitiesSparkJob {
|
|||
isSparkSessionManaged,
|
||||
spark -> {
|
||||
HdfsSupport.remove(outputPath, spark.sparkContext().hadoopConfiguration());
|
||||
groupEntities(spark, graphInputPath, outputPath);
|
||||
groupEntities(spark, graphInputPath, checkpointPath, outputPath, filterInvisible);
|
||||
});
|
||||
}
|
||||
|
||||
private static void groupEntities(
|
||||
SparkSession spark,
|
||||
String inputPath,
|
||||
String outputPath) {
|
||||
String checkpointPath,
|
||||
String outputPath,
|
||||
boolean filterInvisible) {
|
||||
|
||||
final TypedColumn<OafEntity, OafEntity> aggregator = new GroupingAggregator().toColumn();
|
||||
final JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext());
|
||||
spark
|
||||
.read()
|
||||
.textFile(toSeq(listEntityPaths(inputPath, sc)))
|
||||
.map((MapFunction<String, OafEntity>) GroupEntitiesSparkJob::parseOaf, Encoders.kryo(OafEntity.class))
|
||||
.filter((FilterFunction<OafEntity>) e -> StringUtils.isNotBlank(ModelSupport.idFn().apply(e)))
|
||||
.groupByKey((MapFunction<OafEntity, String>) oaf -> ModelSupport.idFn().apply(oaf), Encoders.STRING())
|
||||
.agg(aggregator)
|
||||
Dataset<OafEntity> allEntities = spark.emptyDataset(OAFENTITY_KRYO_ENC);
|
||||
|
||||
for (Map.Entry<EntityType, Class> e : ModelSupport.entityTypes.entrySet()) {
|
||||
String entity = e.getKey().name();
|
||||
Class<? extends OafEntity> entityClass = e.getValue();
|
||||
String entityInputPath = inputPath + "/" + entity;
|
||||
|
||||
if (!HdfsSupport.exists(entityInputPath, spark.sparkContext().hadoopConfiguration())) {
|
||||
continue;
|
||||
}
|
||||
|
||||
allEntities = allEntities
|
||||
.union(
|
||||
((Dataset<OafEntity>) spark
|
||||
.read()
|
||||
.schema(Encoders.bean(entityClass).schema())
|
||||
.json(entityInputPath)
|
||||
.filter("length(id) > 0")
|
||||
.as(Encoders.bean(entityClass)))
|
||||
.map((MapFunction<OafEntity, OafEntity>) r -> r, OAFENTITY_KRYO_ENC));
|
||||
}
|
||||
|
||||
Dataset<?> groupedEntities = allEntities
|
||||
.groupByKey((MapFunction<OafEntity, String>) OafEntity::getId, Encoders.STRING())
|
||||
.reduceGroups((ReduceFunction<OafEntity>) (b, a) -> OafMapperUtils.mergeEntities(b, a))
|
||||
.map(
|
||||
(MapFunction<Tuple2<String, OafEntity>, String>) t -> t._2().getClass().getName() +
|
||||
"|" + OBJECT_MAPPER.writeValueAsString(t._2()),
|
||||
Encoders.STRING())
|
||||
(MapFunction<Tuple2<String, OafEntity>, Tuple2<String, OafEntity>>) t -> new Tuple2(
|
||||
t._2().getClass().getName(), t._2()),
|
||||
Encoders.tuple(Encoders.STRING(), OAFENTITY_KRYO_ENC));
|
||||
|
||||
// pivot on "_1" (classname of the entity)
|
||||
// created columns containing only entities of the same class
|
||||
for (Map.Entry<EntityType, Class> e : ModelSupport.entityTypes.entrySet()) {
|
||||
String entity = e.getKey().name();
|
||||
Class<? extends OafEntity> entityClass = e.getValue();
|
||||
|
||||
groupedEntities = groupedEntities
|
||||
.withColumn(
|
||||
entity,
|
||||
when(col("_1").equalTo(entityClass.getName()), col("_2")));
|
||||
}
|
||||
|
||||
groupedEntities
|
||||
.drop("_1", "_2")
|
||||
.write()
|
||||
.option("compression", "gzip")
|
||||
.mode(SaveMode.Overwrite)
|
||||
.text(outputPath);
|
||||
}
|
||||
.option("compression", "gzip")
|
||||
.save(checkpointPath);
|
||||
|
||||
public static class GroupingAggregator extends Aggregator<OafEntity, OafEntity, OafEntity> {
|
||||
ForkJoinPool parPool = new ForkJoinPool(ModelSupport.entityTypes.size());
|
||||
|
||||
@Override
|
||||
public OafEntity zero() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public OafEntity reduce(OafEntity b, OafEntity a) {
|
||||
return mergeAndGet(b, a);
|
||||
}
|
||||
|
||||
private OafEntity mergeAndGet(OafEntity b, OafEntity a) {
|
||||
if (Objects.nonNull(a) && Objects.nonNull(b)) {
|
||||
return OafMapperUtils.mergeEntities(b, a);
|
||||
}
|
||||
return Objects.isNull(a) ? b : a;
|
||||
}
|
||||
|
||||
@Override
|
||||
public OafEntity merge(OafEntity b, OafEntity a) {
|
||||
return mergeAndGet(b, a);
|
||||
}
|
||||
|
||||
@Override
|
||||
public OafEntity finish(OafEntity j) {
|
||||
return j;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Encoder<OafEntity> bufferEncoder() {
|
||||
return Encoders.kryo(OafEntity.class);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Encoder<OafEntity> outputEncoder() {
|
||||
return Encoders.kryo(OafEntity.class);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
private static OafEntity parseOaf(String s) {
|
||||
|
||||
DocumentContext dc = JsonPath
|
||||
.parse(s, Configuration.defaultConfiguration().addOptions(Option.SUPPRESS_EXCEPTIONS));
|
||||
final String id = dc.read(ID_JPATH);
|
||||
if (StringUtils.isNotBlank(id)) {
|
||||
|
||||
String prefix = StringUtils.substringBefore(id, "|");
|
||||
switch (prefix) {
|
||||
case "10":
|
||||
return parse(s, Datasource.class);
|
||||
case "20":
|
||||
return parse(s, Organization.class);
|
||||
case "40":
|
||||
return parse(s, Project.class);
|
||||
case "50":
|
||||
String resultType = dc.read("$.resulttype.classid");
|
||||
switch (resultType) {
|
||||
case "publication":
|
||||
return parse(s, Publication.class);
|
||||
case "dataset":
|
||||
return parse(s, eu.dnetlib.dhp.schema.oaf.Dataset.class);
|
||||
case "software":
|
||||
return parse(s, Software.class);
|
||||
case "other":
|
||||
return parse(s, OtherResearchProduct.class);
|
||||
default:
|
||||
throw new IllegalArgumentException(String.format("invalid resultType: '%s'", resultType));
|
||||
}
|
||||
default:
|
||||
throw new IllegalArgumentException(String.format("invalid id prefix: '%s'", prefix));
|
||||
}
|
||||
} else {
|
||||
throw new IllegalArgumentException(String.format("invalid oaf: '%s'", s));
|
||||
}
|
||||
}
|
||||
|
||||
private static <T extends OafEntity> OafEntity parse(String s, Class<T> clazz) {
|
||||
try {
|
||||
return OBJECT_MAPPER.readValue(s, clazz);
|
||||
} catch (IOException e) {
|
||||
throw new IllegalArgumentException(e);
|
||||
}
|
||||
}
|
||||
|
||||
private static List<String> listEntityPaths(String inputPath, JavaSparkContext sc) {
|
||||
return HdfsSupport
|
||||
.listFiles(inputPath, sc.hadoopConfiguration())
|
||||
ModelSupport.entityTypes
|
||||
.entrySet()
|
||||
.stream()
|
||||
.filter(f -> !f.toLowerCase().contains("relation"))
|
||||
.collect(Collectors.toList());
|
||||
}
|
||||
.map(e -> parPool.submit(() -> {
|
||||
String entity = e.getKey().name();
|
||||
Class<? extends OafEntity> entityClass = e.getValue();
|
||||
|
||||
spark
|
||||
.read()
|
||||
.load(checkpointPath)
|
||||
.select(col(entity).as("value"))
|
||||
.filter("value IS NOT NULL")
|
||||
.as(OAFENTITY_KRYO_ENC)
|
||||
.map((MapFunction<OafEntity, OafEntity>) r -> r, (Encoder<OafEntity>) Encoders.bean(entityClass))
|
||||
.filter(filterInvisible ? "dataInfo.invisible != TRUE" : "TRUE")
|
||||
.write()
|
||||
.mode(SaveMode.Overwrite)
|
||||
.option("compression", "gzip")
|
||||
.json(outputPath + "/" + entity);
|
||||
}))
|
||||
.collect(Collectors.toList())
|
||||
.forEach(t -> {
|
||||
try {
|
||||
t.get();
|
||||
} catch (InterruptedException | ExecutionException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
});
|
||||
}
|
||||
}
|
||||
|
|
|
@ -1,26 +0,0 @@
|
|||
[
|
||||
{
|
||||
"paramName": "issm",
|
||||
"paramLongName": "isSparkSessionManaged",
|
||||
"paramDescription": "when true will stop SparkSession after job execution",
|
||||
"paramRequired": false
|
||||
},
|
||||
{
|
||||
"paramName": "i",
|
||||
"paramLongName": "inputPath",
|
||||
"paramDescription": "the source path",
|
||||
"paramRequired": true
|
||||
},
|
||||
{
|
||||
"paramName": "o",
|
||||
"paramLongName": "outputPath",
|
||||
"paramDescription": "path of the output graph",
|
||||
"paramRequired": true
|
||||
},
|
||||
{
|
||||
"paramName": "fi",
|
||||
"paramLongName": "filterInvisible",
|
||||
"paramDescription": "if true filters out invisible entities",
|
||||
"paramRequired": true
|
||||
}
|
||||
]
|
|
@ -8,13 +8,25 @@
|
|||
{
|
||||
"paramName": "gin",
|
||||
"paramLongName": "graphInputPath",
|
||||
"paramDescription": "the graph root path",
|
||||
"paramDescription": "the input graph root path",
|
||||
"paramRequired": true
|
||||
},
|
||||
{
|
||||
"paramName": "cp",
|
||||
"paramLongName": "checkpointPath",
|
||||
"paramDescription": "checkpoint directory",
|
||||
"paramRequired": true
|
||||
},
|
||||
{
|
||||
"paramName": "out",
|
||||
"paramLongName": "outputPath",
|
||||
"paramDescription": "the output merged graph root path",
|
||||
"paramDescription": "the output graph root path",
|
||||
"paramRequired": true
|
||||
},
|
||||
{
|
||||
"paramName": "fi",
|
||||
"paramLongName": "filterInvisible",
|
||||
"paramDescription": "if true filters out invisible entities",
|
||||
"paramRequired": true
|
||||
}
|
||||
]
|
|
@ -81,7 +81,7 @@ case class SparkModel(conf: DedupConfig) {
|
|||
MapDocumentUtil.truncateList(
|
||||
MapDocumentUtil.getJPathList(fdef.getPath, documentContext, fdef.getType),
|
||||
fdef.getSize
|
||||
).toArray
|
||||
).asScala
|
||||
|
||||
case Type.StringConcat =>
|
||||
val jpaths = CONCAT_REGEX.split(fdef.getPath)
|
||||
|
|
|
@ -1,6 +1,23 @@
|
|||
|
||||
package eu.dnetlib.pace.util;
|
||||
|
||||
/*
|
||||
* Diff Match and Patch
|
||||
* Copyright 2018 The diff-match-patch Authors.
|
||||
* https://github.com/google/diff-match-patch
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
/*
|
||||
* Diff Match and Patch
|
||||
* Copyright 2018 The diff-match-patch Authors.
|
||||
|
|
|
@ -117,6 +117,11 @@ public class MapDocumentUtil {
|
|||
return result;
|
||||
}
|
||||
|
||||
if (type == Type.List && jresult instanceof List) {
|
||||
((List<?>) jresult).forEach(x -> result.add(x.toString()));
|
||||
return result;
|
||||
}
|
||||
|
||||
if (jresult instanceof JSONArray) {
|
||||
((JSONArray) jresult).forEach(it -> {
|
||||
try {
|
||||
|
|
|
@ -12,6 +12,7 @@ import org.apache.commons.io.IOUtils;
|
|||
import org.apache.hadoop.io.Text;
|
||||
import org.apache.hadoop.mapred.SequenceFileOutputFormat;
|
||||
import org.apache.spark.SparkConf;
|
||||
import org.apache.spark.api.java.JavaPairRDD;
|
||||
import org.apache.spark.api.java.function.FilterFunction;
|
||||
import org.apache.spark.api.java.function.FlatMapFunction;
|
||||
import org.apache.spark.api.java.function.MapFunction;
|
||||
|
@ -30,15 +31,29 @@ import eu.dnetlib.dhp.schema.common.ModelSupport;
|
|||
import eu.dnetlib.dhp.schema.oaf.*;
|
||||
import eu.dnetlib.dhp.schema.oaf.utils.CleaningFunctions;
|
||||
import eu.dnetlib.dhp.schema.oaf.utils.IdentifierFactory;
|
||||
import eu.dnetlib.dhp.schema.oaf.utils.OafMapperUtils;
|
||||
import eu.dnetlib.dhp.schema.oaf.utils.PidType;
|
||||
import eu.dnetlib.dhp.utils.DHPUtils;
|
||||
import scala.Tuple2;
|
||||
|
||||
public class CreateActionSetSparkJob implements Serializable {
|
||||
public static final String OPENCITATIONS_CLASSID = "sysimport:crosswalk:opencitations";
|
||||
public static final String OPENCITATIONS_CLASSNAME = "Imported from OpenCitations";
|
||||
private static final String ID_PREFIX = "50|doi_________::";
|
||||
|
||||
// DOI-to-DOI citations
|
||||
public static final String COCI = "COCI";
|
||||
|
||||
// PMID-to-PMID citations
|
||||
public static final String POCI = "POCI";
|
||||
|
||||
private static final String DOI_PREFIX = "50|doi_________::";
|
||||
|
||||
private static final String PMID_PREFIX = "50|pmid________::";
|
||||
|
||||
private static final String TRUST = "0.91";
|
||||
|
||||
private static final Logger log = LoggerFactory.getLogger(CreateActionSetSparkJob.class);
|
||||
|
||||
private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
|
||||
|
||||
public static void main(final String[] args) throws IOException, ParseException {
|
||||
|
@ -62,7 +77,7 @@ public class CreateActionSetSparkJob implements Serializable {
|
|||
log.info("isSparkSessionManaged: {}", isSparkSessionManaged);
|
||||
|
||||
final String inputPath = parser.get("inputPath");
|
||||
log.info("inputPath {}", inputPath.toString());
|
||||
log.info("inputPath {}", inputPath);
|
||||
|
||||
final String outputPath = parser.get("outputPath");
|
||||
log.info("outputPath {}", outputPath);
|
||||
|
@ -76,41 +91,68 @@ public class CreateActionSetSparkJob implements Serializable {
|
|||
runWithSparkSession(
|
||||
conf,
|
||||
isSparkSessionManaged,
|
||||
spark -> {
|
||||
extractContent(spark, inputPath, outputPath, shouldDuplicateRels);
|
||||
});
|
||||
spark -> extractContent(spark, inputPath, outputPath, shouldDuplicateRels));
|
||||
|
||||
}
|
||||
|
||||
private static void extractContent(SparkSession spark, String inputPath, String outputPath,
|
||||
boolean shouldDuplicateRels) {
|
||||
spark
|
||||
|
||||
getTextTextJavaPairRDD(spark, inputPath, shouldDuplicateRels, COCI)
|
||||
.union(getTextTextJavaPairRDD(spark, inputPath, shouldDuplicateRels, POCI))
|
||||
.saveAsHadoopFile(outputPath, Text.class, Text.class, SequenceFileOutputFormat.class);
|
||||
}
|
||||
|
||||
private static JavaPairRDD<Text, Text> getTextTextJavaPairRDD(SparkSession spark, String inputPath,
|
||||
boolean shouldDuplicateRels, String prefix) {
|
||||
return spark
|
||||
.read()
|
||||
.textFile(inputPath + "/*")
|
||||
.textFile(inputPath + "/" + prefix + "/" + prefix + "_JSON/*")
|
||||
.map(
|
||||
(MapFunction<String, COCI>) value -> OBJECT_MAPPER.readValue(value, COCI.class),
|
||||
Encoders.bean(COCI.class))
|
||||
.flatMap(
|
||||
(FlatMapFunction<COCI, Relation>) value -> createRelation(value, shouldDuplicateRels).iterator(),
|
||||
(FlatMapFunction<COCI, Relation>) value -> createRelation(
|
||||
value, shouldDuplicateRels, prefix)
|
||||
.iterator(),
|
||||
Encoders.bean(Relation.class))
|
||||
.filter((FilterFunction<Relation>) value -> value != null)
|
||||
.filter((FilterFunction<Relation>) Objects::nonNull)
|
||||
.toJavaRDD()
|
||||
.map(p -> new AtomicAction(p.getClass(), p))
|
||||
.mapToPair(
|
||||
aa -> new Tuple2<>(new Text(aa.getClazz().getCanonicalName()),
|
||||
new Text(OBJECT_MAPPER.writeValueAsString(aa))))
|
||||
.saveAsHadoopFile(outputPath, Text.class, Text.class, SequenceFileOutputFormat.class);
|
||||
|
||||
new Text(OBJECT_MAPPER.writeValueAsString(aa))));
|
||||
}
|
||||
|
||||
private static List<Relation> createRelation(COCI value, boolean duplicate) {
|
||||
private static List<Relation> createRelation(COCI value, boolean duplicate, String p) {
|
||||
|
||||
List<Relation> relationList = new ArrayList<>();
|
||||
String prefix;
|
||||
String citing;
|
||||
String cited;
|
||||
|
||||
String citing = ID_PREFIX
|
||||
+ IdentifierFactory.md5(CleaningFunctions.normalizePidValue("doi", value.getCiting()));
|
||||
final String cited = ID_PREFIX
|
||||
+ IdentifierFactory.md5(CleaningFunctions.normalizePidValue("doi", value.getCited()));
|
||||
switch (p) {
|
||||
case COCI:
|
||||
prefix = DOI_PREFIX;
|
||||
citing = prefix
|
||||
+ IdentifierFactory
|
||||
.md5(CleaningFunctions.normalizePidValue(PidType.doi.toString(), value.getCiting()));
|
||||
cited = prefix
|
||||
+ IdentifierFactory
|
||||
.md5(CleaningFunctions.normalizePidValue(PidType.doi.toString(), value.getCited()));
|
||||
break;
|
||||
case POCI:
|
||||
prefix = PMID_PREFIX;
|
||||
citing = prefix
|
||||
+ IdentifierFactory
|
||||
.md5(CleaningFunctions.normalizePidValue(PidType.pmid.toString(), value.getCiting()));
|
||||
cited = prefix
|
||||
+ IdentifierFactory
|
||||
.md5(CleaningFunctions.normalizePidValue(PidType.pmid.toString(), value.getCited()));
|
||||
break;
|
||||
default:
|
||||
throw new IllegalStateException("Invalid prefix: " + p);
|
||||
}
|
||||
|
||||
if (!citing.equals(cited)) {
|
||||
relationList
|
||||
|
@ -120,7 +162,7 @@ public class CreateActionSetSparkJob implements Serializable {
|
|||
cited, ModelConstants.CITES));
|
||||
|
||||
if (duplicate && value.getCiting().endsWith(".refs")) {
|
||||
citing = ID_PREFIX + IdentifierFactory
|
||||
citing = prefix + IdentifierFactory
|
||||
.md5(
|
||||
CleaningFunctions
|
||||
.normalizePidValue(
|
||||
|
@ -132,59 +174,30 @@ public class CreateActionSetSparkJob implements Serializable {
|
|||
return relationList;
|
||||
}
|
||||
|
||||
private static Collection<Relation> getRelations(String citing, String cited) {
|
||||
|
||||
return Arrays
|
||||
.asList(
|
||||
getRelation(citing, cited, ModelConstants.CITES),
|
||||
getRelation(cited, citing, ModelConstants.IS_CITED_BY));
|
||||
}
|
||||
|
||||
public static Relation getRelation(
|
||||
String source,
|
||||
String target,
|
||||
String relclass) {
|
||||
Relation r = new Relation();
|
||||
r.setCollectedfrom(getCollectedFrom());
|
||||
r.setSource(source);
|
||||
r.setTarget(target);
|
||||
r.setRelClass(relclass);
|
||||
r.setRelType(ModelConstants.RESULT_RESULT);
|
||||
r.setSubRelType(ModelConstants.CITATION);
|
||||
r
|
||||
.setDataInfo(
|
||||
getDataInfo());
|
||||
return r;
|
||||
}
|
||||
String relClass) {
|
||||
|
||||
public static List<KeyValue> getCollectedFrom() {
|
||||
KeyValue kv = new KeyValue();
|
||||
kv.setKey(ModelConstants.OPENOCITATIONS_ID);
|
||||
kv.setValue(ModelConstants.OPENOCITATIONS_NAME);
|
||||
|
||||
return Arrays.asList(kv);
|
||||
}
|
||||
|
||||
public static DataInfo getDataInfo() {
|
||||
DataInfo di = new DataInfo();
|
||||
di.setInferred(false);
|
||||
di.setDeletedbyinference(false);
|
||||
di.setTrust(TRUST);
|
||||
|
||||
di
|
||||
.setProvenanceaction(
|
||||
getQualifier(OPENCITATIONS_CLASSID, OPENCITATIONS_CLASSNAME, ModelConstants.DNET_PROVENANCE_ACTIONS));
|
||||
return di;
|
||||
}
|
||||
|
||||
public static Qualifier getQualifier(String class_id, String class_name,
|
||||
String qualifierSchema) {
|
||||
Qualifier pa = new Qualifier();
|
||||
pa.setClassid(class_id);
|
||||
pa.setClassname(class_name);
|
||||
pa.setSchemeid(qualifierSchema);
|
||||
pa.setSchemename(qualifierSchema);
|
||||
return pa;
|
||||
return OafMapperUtils
|
||||
.getRelation(
|
||||
source,
|
||||
target,
|
||||
ModelConstants.RESULT_RESULT,
|
||||
ModelConstants.CITATION,
|
||||
relClass,
|
||||
Arrays
|
||||
.asList(
|
||||
OafMapperUtils.keyValue(ModelConstants.OPENOCITATIONS_ID, ModelConstants.OPENOCITATIONS_NAME)),
|
||||
OafMapperUtils
|
||||
.dataInfo(
|
||||
false, null, false, false,
|
||||
OafMapperUtils
|
||||
.qualifier(
|
||||
OPENCITATIONS_CLASSID, OPENCITATIONS_CLASSNAME,
|
||||
ModelConstants.DNET_PROVENANCE_ACTIONS, ModelConstants.DNET_PROVENANCE_ACTIONS),
|
||||
TRUST),
|
||||
null);
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -3,6 +3,7 @@ package eu.dnetlib.dhp.actionmanager.opencitations;
|
|||
|
||||
import java.io.*;
|
||||
import java.io.Serializable;
|
||||
import java.util.Arrays;
|
||||
import java.util.Objects;
|
||||
import java.util.zip.GZIPOutputStream;
|
||||
import java.util.zip.ZipEntry;
|
||||
|
@ -37,7 +38,7 @@ public class GetOpenCitationsRefs implements Serializable {
|
|||
parser.parseArgument(args);
|
||||
|
||||
final String[] inputFile = parser.get("inputFile").split(";");
|
||||
log.info("inputFile {}", inputFile.toString());
|
||||
log.info("inputFile {}", Arrays.asList(inputFile));
|
||||
|
||||
final String workingPath = parser.get("workingPath");
|
||||
log.info("workingPath {}", workingPath);
|
||||
|
@ -45,6 +46,9 @@ public class GetOpenCitationsRefs implements Serializable {
|
|||
final String hdfsNameNode = parser.get("hdfsNameNode");
|
||||
log.info("hdfsNameNode {}", hdfsNameNode);
|
||||
|
||||
final String prefix = parser.get("prefix");
|
||||
log.info("prefix {}", prefix);
|
||||
|
||||
Configuration conf = new Configuration();
|
||||
conf.set("fs.defaultFS", hdfsNameNode);
|
||||
|
||||
|
@ -53,30 +57,31 @@ public class GetOpenCitationsRefs implements Serializable {
|
|||
GetOpenCitationsRefs ocr = new GetOpenCitationsRefs();
|
||||
|
||||
for (String file : inputFile) {
|
||||
ocr.doExtract(workingPath + "/Original/" + file, workingPath, fileSystem);
|
||||
ocr.doExtract(workingPath + "/Original/" + file, workingPath, fileSystem, prefix);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
private void doExtract(String inputFile, String workingPath, FileSystem fileSystem)
|
||||
private void doExtract(String inputFile, String workingPath, FileSystem fileSystem, String prefix)
|
||||
throws IOException {
|
||||
|
||||
final Path path = new Path(inputFile);
|
||||
|
||||
FSDataInputStream oc_zip = fileSystem.open(path);
|
||||
|
||||
int count = 1;
|
||||
// int count = 1;
|
||||
try (ZipInputStream zis = new ZipInputStream(oc_zip)) {
|
||||
ZipEntry entry = null;
|
||||
while ((entry = zis.getNextEntry()) != null) {
|
||||
|
||||
if (!entry.isDirectory()) {
|
||||
String fileName = entry.getName();
|
||||
fileName = fileName.substring(0, fileName.indexOf("T")) + "_" + count;
|
||||
count++;
|
||||
// fileName = fileName.substring(0, fileName.indexOf("T")) + "_" + count;
|
||||
fileName = fileName.substring(0, fileName.lastIndexOf("."));
|
||||
// count++;
|
||||
try (
|
||||
FSDataOutputStream out = fileSystem
|
||||
.create(new Path(workingPath + "/COCI/" + fileName + ".gz"));
|
||||
.create(new Path(workingPath + "/" + prefix + "/" + fileName + ".gz"));
|
||||
GZIPOutputStream gzipOs = new GZIPOutputStream(new BufferedOutputStream(out))) {
|
||||
|
||||
IOUtils.copy(zis, gzipOs);
|
||||
|
|
|
@ -7,6 +7,7 @@ import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession;
|
|||
|
||||
import java.io.IOException;
|
||||
import java.io.Serializable;
|
||||
import java.util.Arrays;
|
||||
import java.util.Optional;
|
||||
|
||||
import org.apache.commons.io.IOUtils;
|
||||
|
@ -42,13 +43,16 @@ public class ReadCOCI implements Serializable {
|
|||
log.info("outputPath: {}", outputPath);
|
||||
|
||||
final String[] inputFile = parser.get("inputFile").split(";");
|
||||
log.info("inputFile {}", inputFile.toString());
|
||||
log.info("inputFile {}", Arrays.asList(inputFile));
|
||||
Boolean isSparkSessionManaged = isSparkSessionManaged(parser);
|
||||
log.info("isSparkSessionManaged: {}", isSparkSessionManaged);
|
||||
|
||||
final String workingPath = parser.get("workingPath");
|
||||
log.info("workingPath {}", workingPath);
|
||||
|
||||
final String format = parser.get("format");
|
||||
log.info("format {}", format);
|
||||
|
||||
SparkConf sconf = new SparkConf();
|
||||
|
||||
final String delimiter = Optional
|
||||
|
@ -64,16 +68,17 @@ public class ReadCOCI implements Serializable {
|
|||
workingPath,
|
||||
inputFile,
|
||||
outputPath,
|
||||
delimiter);
|
||||
delimiter,
|
||||
format);
|
||||
});
|
||||
}
|
||||
|
||||
private static void doRead(SparkSession spark, String workingPath, String[] inputFiles,
|
||||
String outputPath,
|
||||
String delimiter) throws IOException {
|
||||
String delimiter, String format) {
|
||||
|
||||
for (String inputFile : inputFiles) {
|
||||
String p_string = workingPath + "/" + inputFile + ".gz";
|
||||
String pString = workingPath + "/" + inputFile + ".gz";
|
||||
|
||||
Dataset<Row> cociData = spark
|
||||
.read()
|
||||
|
@ -82,14 +87,20 @@ public class ReadCOCI implements Serializable {
|
|||
.option("inferSchema", "true")
|
||||
.option("header", "true")
|
||||
.option("quotes", "\"")
|
||||
.load(p_string)
|
||||
.load(pString)
|
||||
.repartition(100);
|
||||
|
||||
cociData.map((MapFunction<Row, COCI>) row -> {
|
||||
COCI coci = new COCI();
|
||||
if (format.equals("COCI")) {
|
||||
coci.setCiting(row.getString(1));
|
||||
coci.setCited(row.getString(2));
|
||||
} else {
|
||||
coci.setCiting(String.valueOf(row.getInt(1)));
|
||||
coci.setCited(String.valueOf(row.getInt(2)));
|
||||
}
|
||||
coci.setOci(row.getString(0));
|
||||
coci.setCiting(row.getString(1));
|
||||
coci.setCited(row.getString(2));
|
||||
|
||||
return coci;
|
||||
}, Encoders.bean(COCI.class))
|
||||
.write()
|
||||
|
|
|
@ -16,10 +16,11 @@
|
|||
"paramLongName": "isSparkSessionManaged",
|
||||
"paramDescription": "the hdfs name node",
|
||||
"paramRequired": false
|
||||
}, {
|
||||
"paramName": "sdr",
|
||||
"paramLongName": "shouldDuplicateRels",
|
||||
"paramDescription": "the hdfs name node",
|
||||
"paramRequired": false
|
||||
}
|
||||
},
|
||||
{
|
||||
"paramName": "sdr",
|
||||
"paramLongName": "shouldDuplicateRels",
|
||||
"paramDescription": "activates/deactivates the construction of bidirectional relations Cites/IsCitedBy",
|
||||
"paramRequired": false
|
||||
}
|
||||
]
|
||||
|
|
|
@ -16,5 +16,11 @@
|
|||
"paramLongName": "hdfsNameNode",
|
||||
"paramDescription": "the hdfs name node",
|
||||
"paramRequired": true
|
||||
},
|
||||
{
|
||||
"paramName": "p",
|
||||
"paramLongName": "prefix",
|
||||
"paramDescription": "COCI or POCI",
|
||||
"paramRequired": true
|
||||
}
|
||||
]
|
||||
|
|
|
@ -30,7 +30,12 @@
|
|||
"paramLongName": "inputFile",
|
||||
"paramDescription": "the hdfs name node",
|
||||
"paramRequired": true
|
||||
}
|
||||
}, {
|
||||
"paramName": "f",
|
||||
"paramLongName": "format",
|
||||
"paramDescription": "the hdfs name node",
|
||||
"paramRequired": true
|
||||
}
|
||||
]
|
||||
|
||||
|
||||
|
|
|
@ -34,6 +34,7 @@
|
|||
<kill name="Kill">
|
||||
<message>Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}]</message>
|
||||
</kill>
|
||||
|
||||
<action name="download">
|
||||
<shell xmlns="uri:oozie:shell-action:0.2">
|
||||
<job-tracker>${jobTracker}</job-tracker>
|
||||
|
@ -46,7 +47,7 @@
|
|||
</configuration>
|
||||
<exec>download.sh</exec>
|
||||
<argument>${filelist}</argument>
|
||||
<argument>${workingPath}/Original</argument>
|
||||
<argument>${workingPath}/${prefix}/Original</argument>
|
||||
<env-var>HADOOP_USER_NAME=${wf:user()}</env-var>
|
||||
<file>download.sh</file>
|
||||
<capture-output/>
|
||||
|
@ -54,12 +55,14 @@
|
|||
<ok to="extract"/>
|
||||
<error to="Kill"/>
|
||||
</action>
|
||||
|
||||
<action name="extract">
|
||||
<java>
|
||||
<main-class>eu.dnetlib.dhp.actionmanager.opencitations.GetOpenCitationsRefs</main-class>
|
||||
<arg>--hdfsNameNode</arg><arg>${nameNode}</arg>
|
||||
<arg>--inputFile</arg><arg>${inputFile}</arg>
|
||||
<arg>--workingPath</arg><arg>${workingPath}</arg>
|
||||
<arg>--workingPath</arg><arg>${workingPath}/${prefix}</arg>
|
||||
<arg>--prefix</arg><arg>${prefix}</arg>
|
||||
</java>
|
||||
<ok to="read"/>
|
||||
<error to="Kill"/>
|
||||
|
@ -82,10 +85,11 @@
|
|||
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
|
||||
--conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir}
|
||||
</spark-opts>
|
||||
<arg>--workingPath</arg><arg>${workingPath}/COCI</arg>
|
||||
<arg>--outputPath</arg><arg>${workingPath}/COCI_JSON/</arg>
|
||||
<arg>--workingPath</arg><arg>${workingPath}/${prefix}/${prefix}</arg>
|
||||
<arg>--outputPath</arg><arg>${workingPath}/${prefix}/${prefix}_JSON/</arg>
|
||||
<arg>--delimiter</arg><arg>${delimiter}</arg>
|
||||
<arg>--inputFile</arg><arg>${inputFileCoci}</arg>
|
||||
<arg>--format</arg><arg>${prefix}</arg>
|
||||
</spark>
|
||||
<ok to="create_actionset"/>
|
||||
<error to="Kill"/>
|
||||
|
@ -108,7 +112,7 @@
|
|||
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
|
||||
--conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir}
|
||||
</spark-opts>
|
||||
<arg>--inputPath</arg><arg>${workingPath}/COCI_JSON</arg>
|
||||
<arg>--inputPath</arg><arg>${workingPath}</arg>
|
||||
<arg>--outputPath</arg><arg>${outputPath}</arg>
|
||||
</spark>
|
||||
<ok to="End"/>
|
||||
|
|
|
@ -1,4 +1,9 @@
|
|||
{
|
||||
"ETHZ.UNIGENF": {
|
||||
"openaire_id": "opendoar____::1400",
|
||||
"datacite_name": "Uni Genf",
|
||||
"official_name": "Archive ouverte UNIGE"
|
||||
},
|
||||
"GESIS.RKI": {
|
||||
"openaire_id": "re3data_____::r3d100010436",
|
||||
"datacite_name": "Forschungsdatenzentrum am Robert Koch Institut",
|
||||
|
|
|
@ -2,7 +2,9 @@
|
|||
package eu.dnetlib.dhp.broker.oa.util;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.spark.sql.Row;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
@ -27,10 +29,14 @@ public class TrustUtils {
|
|||
static {
|
||||
mapper = new ObjectMapper();
|
||||
try {
|
||||
dedupConfig = mapper
|
||||
.readValue(
|
||||
DedupConfig.class.getResourceAsStream("/eu/dnetlib/dhp/broker/oa/dedupConfig/dedupConfig.json"),
|
||||
DedupConfig.class);
|
||||
dedupConfig = DedupConfig
|
||||
.load(
|
||||
IOUtils
|
||||
.toString(
|
||||
DedupConfig.class
|
||||
.getResourceAsStream("/eu/dnetlib/dhp/broker/oa/dedupConfig/dedupConfig.json"),
|
||||
StandardCharsets.UTF_8));
|
||||
|
||||
deduper = new SparkDeduper(dedupConfig);
|
||||
} catch (final IOException e) {
|
||||
log.error("Error loading dedupConfig, e");
|
||||
|
@ -57,7 +63,7 @@ public class TrustUtils {
|
|||
return TrustUtils.rescale(score, threshold);
|
||||
} catch (final Exception e) {
|
||||
log.error("Error computing score between results", e);
|
||||
return BrokerConstants.MIN_TRUST;
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -1,57 +0,0 @@
|
|||
|
||||
package eu.dnetlib.dhp.oa.dedup;
|
||||
|
||||
import java.util.Objects;
|
||||
|
||||
import org.apache.spark.sql.Encoder;
|
||||
import org.apache.spark.sql.Encoders;
|
||||
import org.apache.spark.sql.expressions.Aggregator;
|
||||
|
||||
import eu.dnetlib.dhp.schema.oaf.Relation;
|
||||
|
||||
public class RelationAggregator extends Aggregator<Relation, Relation, Relation> {
|
||||
|
||||
private static final Relation ZERO = new Relation();
|
||||
|
||||
@Override
|
||||
public Relation zero() {
|
||||
return ZERO;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Relation reduce(Relation b, Relation a) {
|
||||
return mergeRel(b, a);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Relation merge(Relation b, Relation a) {
|
||||
return mergeRel(b, a);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Relation finish(Relation r) {
|
||||
return r;
|
||||
}
|
||||
|
||||
private Relation mergeRel(Relation b, Relation a) {
|
||||
if (Objects.equals(b, ZERO)) {
|
||||
return a;
|
||||
}
|
||||
if (Objects.equals(a, ZERO)) {
|
||||
return b;
|
||||
}
|
||||
|
||||
b.mergeFrom(a);
|
||||
return b;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Encoder<Relation> bufferEncoder() {
|
||||
return Encoders.kryo(Relation.class);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Encoder<Relation> outputEncoder() {
|
||||
return Encoders.kryo(Relation.class);
|
||||
}
|
||||
}
|
|
@ -1,78 +0,0 @@
|
|||
package eu.dnetlib.dhp.oa.dedup
|
||||
|
||||
import eu.dnetlib.dhp.application.ArgumentApplicationParser
|
||||
import eu.dnetlib.dhp.common.HdfsSupport
|
||||
import eu.dnetlib.dhp.schema.oaf.Relation
|
||||
import eu.dnetlib.dhp.utils.ISLookupClientFactory
|
||||
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService
|
||||
import org.apache.commons.io.IOUtils
|
||||
import org.apache.spark.SparkConf
|
||||
import org.apache.spark.sql._
|
||||
import org.apache.spark.sql.functions.col
|
||||
import org.apache.spark.sql.types.{DataTypes, StructField, StructType}
|
||||
import org.slf4j.LoggerFactory
|
||||
|
||||
object SparkCleanRelation {
|
||||
private val log = LoggerFactory.getLogger(classOf[SparkCleanRelation])
|
||||
|
||||
@throws[Exception]
|
||||
def main(args: Array[String]): Unit = {
|
||||
val parser = new ArgumentApplicationParser(
|
||||
IOUtils.toString(
|
||||
classOf[SparkCleanRelation].getResourceAsStream("/eu/dnetlib/dhp/oa/dedup/cleanRelation_parameters.json")
|
||||
)
|
||||
)
|
||||
parser.parseArgument(args)
|
||||
val conf = new SparkConf
|
||||
|
||||
new SparkCleanRelation(parser, AbstractSparkAction.getSparkSession(conf))
|
||||
.run(ISLookupClientFactory.getLookUpService(parser.get("isLookUpUrl")))
|
||||
}
|
||||
}
|
||||
|
||||
class SparkCleanRelation(parser: ArgumentApplicationParser, spark: SparkSession)
|
||||
extends AbstractSparkAction(parser, spark) {
|
||||
override def run(isLookUpService: ISLookUpService): Unit = {
|
||||
val graphBasePath = parser.get("graphBasePath")
|
||||
val inputPath = parser.get("inputPath")
|
||||
val outputPath = parser.get("outputPath")
|
||||
|
||||
SparkCleanRelation.log.info("graphBasePath: '{}'", graphBasePath)
|
||||
SparkCleanRelation.log.info("inputPath: '{}'", inputPath)
|
||||
SparkCleanRelation.log.info("outputPath: '{}'", outputPath)
|
||||
|
||||
AbstractSparkAction.removeOutputDir(spark, outputPath)
|
||||
|
||||
val entities =
|
||||
Seq("datasource", "project", "organization", "publication", "dataset", "software", "otherresearchproduct")
|
||||
|
||||
val idsSchema = StructType.fromDDL("`id` STRING, `dataInfo` STRUCT<`deletedbyinference`:BOOLEAN,`invisible`:BOOLEAN>")
|
||||
|
||||
val emptyIds = spark.createDataFrame(spark.sparkContext.emptyRDD[Row].setName("empty"),
|
||||
idsSchema)
|
||||
|
||||
val ids = entities
|
||||
.foldLeft(emptyIds)((ds, entity) => {
|
||||
val entityPath = graphBasePath + '/' + entity
|
||||
if (HdfsSupport.exists(entityPath, spark.sparkContext.hadoopConfiguration)) {
|
||||
ds.union(spark.read.schema(idsSchema).json(entityPath))
|
||||
} else {
|
||||
ds
|
||||
}
|
||||
})
|
||||
.filter("dataInfo.deletedbyinference != true AND dataInfo.invisible != true")
|
||||
.select("id")
|
||||
.distinct()
|
||||
|
||||
val relations = spark.read.schema(Encoders.bean(classOf[Relation]).schema).json(inputPath)
|
||||
.filter("dataInfo.deletedbyinference != true AND dataInfo.invisible != true")
|
||||
|
||||
AbstractSparkAction.save(
|
||||
relations
|
||||
.join(ids, col("source") === ids("id"), "leftsemi")
|
||||
.join(ids, col("target") === ids("id"), "leftsemi"),
|
||||
outputPath,
|
||||
SaveMode.Overwrite
|
||||
)
|
||||
}
|
||||
}
|
|
@ -3,23 +3,19 @@ package eu.dnetlib.dhp.oa.dedup;
|
|||
|
||||
import static org.apache.spark.sql.functions.col;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.Iterator;
|
||||
import java.util.Objects;
|
||||
|
||||
import org.apache.commons.beanutils.BeanUtils;
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.commons.lang3.StringUtils;
|
||||
import org.apache.spark.SparkConf;
|
||||
import org.apache.spark.api.java.function.FilterFunction;
|
||||
import org.apache.spark.api.java.function.MapFunction;
|
||||
import org.apache.spark.api.java.function.ReduceFunction;
|
||||
import org.apache.spark.sql.*;
|
||||
import org.apache.spark.sql.catalyst.encoders.RowEncoder;
|
||||
import org.apache.spark.sql.types.StructType;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
|
||||
import eu.dnetlib.dhp.common.HdfsSupport;
|
||||
import eu.dnetlib.dhp.schema.common.EntityType;
|
||||
import eu.dnetlib.dhp.schema.common.ModelConstants;
|
||||
import eu.dnetlib.dhp.schema.common.ModelSupport;
|
||||
import eu.dnetlib.dhp.schema.oaf.DataInfo;
|
||||
|
@ -70,73 +66,63 @@ public class SparkPropagateRelation extends AbstractSparkAction {
|
|||
log.info("workingPath: '{}'", workingPath);
|
||||
log.info("graphOutputPath: '{}'", graphOutputPath);
|
||||
|
||||
final String outputRelationPath = DedupUtility.createEntityPath(graphOutputPath, "relation");
|
||||
removeOutputDir(spark, outputRelationPath);
|
||||
|
||||
Dataset<Relation> mergeRels = spark
|
||||
.read()
|
||||
.load(DedupUtility.createMergeRelPath(workingPath, "*", "*"))
|
||||
.as(REL_BEAN_ENC);
|
||||
|
||||
// <mergedObjectID, dedupID>
|
||||
Dataset<Row> mergedIds = mergeRels
|
||||
Dataset<Row> idsToMerge = mergeRels
|
||||
.where(col("relClass").equalTo(ModelConstants.MERGES))
|
||||
.select(col("source").as("dedupID"), col("target").as("mergedObjectID"))
|
||||
.distinct()
|
||||
.cache();
|
||||
.distinct();
|
||||
|
||||
Dataset<Row> allRels = spark
|
||||
.read()
|
||||
.schema(REL_BEAN_ENC.schema())
|
||||
.json(DedupUtility.createEntityPath(graphBasePath, "relation"));
|
||||
.json(graphBasePath + "/relation");
|
||||
|
||||
Dataset<Relation> dedupedRels = allRels
|
||||
.joinWith(mergedIds, allRels.col("source").equalTo(mergedIds.col("mergedObjectID")), "left_outer")
|
||||
.joinWith(mergedIds, col("_1.target").equalTo(mergedIds.col("mergedObjectID")), "left_outer")
|
||||
.joinWith(idsToMerge, allRels.col("source").equalTo(idsToMerge.col("mergedObjectID")), "left_outer")
|
||||
.joinWith(idsToMerge, col("_1.target").equalTo(idsToMerge.col("mergedObjectID")), "left_outer")
|
||||
.select("_1._1", "_1._2.dedupID", "_2.dedupID")
|
||||
.as(Encoders.tuple(REL_BEAN_ENC, Encoders.STRING(), Encoders.STRING()))
|
||||
.flatMap(SparkPropagateRelation::addInferredRelations, REL_KRYO_ENC);
|
||||
.map((MapFunction<Tuple3<Relation, String, String>, Relation>) t -> {
|
||||
Relation rel = t._1();
|
||||
String newSource = t._2();
|
||||
String newTarget = t._3();
|
||||
|
||||
Dataset<Relation> processedRelations = distinctRelations(
|
||||
dedupedRels.union(mergeRels.map((MapFunction<Relation, Relation>) r -> r, REL_KRYO_ENC)))
|
||||
.filter((FilterFunction<Relation>) r -> !Objects.equals(r.getSource(), r.getTarget()));
|
||||
if (rel.getDataInfo() == null) {
|
||||
rel.setDataInfo(new DataInfo());
|
||||
}
|
||||
|
||||
save(processedRelations, outputRelationPath, SaveMode.Overwrite);
|
||||
}
|
||||
if (newSource != null || newTarget != null) {
|
||||
rel.getDataInfo().setDeletedbyinference(false);
|
||||
|
||||
private static Iterator<Relation> addInferredRelations(Tuple3<Relation, String, String> t) throws Exception {
|
||||
Relation existingRel = t._1();
|
||||
String newSource = t._2();
|
||||
String newTarget = t._3();
|
||||
if (newSource != null)
|
||||
rel.setSource(newSource);
|
||||
|
||||
if (newSource == null && newTarget == null) {
|
||||
return Collections.singleton(t._1()).iterator();
|
||||
}
|
||||
if (newTarget != null)
|
||||
rel.setTarget(newTarget);
|
||||
}
|
||||
|
||||
// update existing relation
|
||||
if (existingRel.getDataInfo() == null) {
|
||||
existingRel.setDataInfo(new DataInfo());
|
||||
}
|
||||
existingRel.getDataInfo().setDeletedbyinference(true);
|
||||
return rel;
|
||||
}, REL_BEAN_ENC);
|
||||
|
||||
// Create new relation inferred by dedupIDs
|
||||
Relation inferredRel = (Relation) BeanUtils.cloneBean(existingRel);
|
||||
// ids of records that are both not deletedbyinference and not invisible
|
||||
Dataset<Row> ids = validIds(spark, graphBasePath);
|
||||
|
||||
inferredRel.setDataInfo((DataInfo) BeanUtils.cloneBean(existingRel.getDataInfo()));
|
||||
inferredRel.getDataInfo().setDeletedbyinference(false);
|
||||
// filter relations that point to valid records, can force them to be visible
|
||||
Dataset<Relation> cleanedRels = dedupedRels
|
||||
.join(ids, col("source").equalTo(ids.col("id")), "leftsemi")
|
||||
.join(ids, col("target").equalTo(ids.col("id")), "leftsemi")
|
||||
.as(REL_BEAN_ENC)
|
||||
.map((MapFunction<Relation, Relation>) r -> {
|
||||
r.getDataInfo().setInvisible(false);
|
||||
return r;
|
||||
}, REL_KRYO_ENC);
|
||||
|
||||
if (newSource != null)
|
||||
inferredRel.setSource(newSource);
|
||||
|
||||
if (newTarget != null)
|
||||
inferredRel.setTarget(newTarget);
|
||||
|
||||
return Arrays.asList(existingRel, inferredRel).iterator();
|
||||
}
|
||||
|
||||
private Dataset<Relation> distinctRelations(Dataset<Relation> rels) {
|
||||
return rels
|
||||
.filter(getRelationFilterFunction())
|
||||
Dataset<Relation> distinctRels = cleanedRels
|
||||
.groupByKey(
|
||||
(MapFunction<Relation, String>) r -> String
|
||||
.join(" ", r.getSource(), r.getTarget(), r.getRelType(), r.getSubRelType(), r.getRelClass()),
|
||||
|
@ -146,13 +132,33 @@ public class SparkPropagateRelation extends AbstractSparkAction {
|
|||
return b;
|
||||
})
|
||||
.map((MapFunction<Tuple2<String, Relation>, Relation>) Tuple2::_2, REL_BEAN_ENC);
|
||||
|
||||
final String outputRelationPath = graphOutputPath + "/relation";
|
||||
removeOutputDir(spark, outputRelationPath);
|
||||
save(
|
||||
distinctRels
|
||||
.union(mergeRels)
|
||||
.filter("source != target AND dataInfo.deletedbyinference != true AND dataInfo.invisible != true"),
|
||||
outputRelationPath,
|
||||
SaveMode.Overwrite);
|
||||
}
|
||||
|
||||
private FilterFunction<Relation> getRelationFilterFunction() {
|
||||
return r -> StringUtils.isNotBlank(r.getSource()) ||
|
||||
StringUtils.isNotBlank(r.getTarget()) ||
|
||||
StringUtils.isNotBlank(r.getRelType()) ||
|
||||
StringUtils.isNotBlank(r.getSubRelType()) ||
|
||||
StringUtils.isNotBlank(r.getRelClass());
|
||||
static Dataset<Row> validIds(SparkSession spark, String graphBasePath) {
|
||||
StructType idsSchema = StructType
|
||||
.fromDDL("`id` STRING, `dataInfo` STRUCT<`deletedbyinference`:BOOLEAN,`invisible`:BOOLEAN>");
|
||||
|
||||
Dataset<Row> allIds = spark.emptyDataset(RowEncoder.apply(idsSchema));
|
||||
|
||||
for (EntityType entityType : ModelSupport.entityTypes.keySet()) {
|
||||
String entityPath = graphBasePath + '/' + entityType.name();
|
||||
if (HdfsSupport.exists(entityPath, spark.sparkContext().hadoopConfiguration())) {
|
||||
allIds = allIds.union(spark.read().schema(idsSchema).json(entityPath));
|
||||
}
|
||||
}
|
||||
|
||||
return allIds
|
||||
.filter("dataInfo.deletedbyinference != true AND dataInfo.invisible != true")
|
||||
.select("id")
|
||||
.distinct();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -1,20 +0,0 @@
|
|||
[
|
||||
{
|
||||
"paramName": "i",
|
||||
"paramLongName": "graphBasePath",
|
||||
"paramDescription": "the base path of raw graph",
|
||||
"paramRequired": true
|
||||
},
|
||||
{
|
||||
"paramName": "w",
|
||||
"paramLongName": "inputPath",
|
||||
"paramDescription": "the path to the input relation to cleanup",
|
||||
"paramRequired": true
|
||||
},
|
||||
{
|
||||
"paramName": "o",
|
||||
"paramLongName": "outputPath",
|
||||
"paramDescription": "the path of the output relation cleaned",
|
||||
"paramRequired": true
|
||||
}
|
||||
]
|
|
@ -100,35 +100,9 @@
|
|||
--conf spark.sql.shuffle.partitions=15000
|
||||
</spark-opts>
|
||||
<arg>--graphBasePath</arg><arg>${graphBasePath}</arg>
|
||||
<arg>--graphOutputPath</arg><arg>${workingPath}/propagaterelation/</arg>
|
||||
<arg>--graphOutputPath</arg><arg>${graphOutputPath}</arg>
|
||||
<arg>--workingPath</arg><arg>${workingPath}</arg>
|
||||
</spark>
|
||||
<ok to="CleanRelation"/>
|
||||
<error to="Kill"/>
|
||||
</action>
|
||||
|
||||
<action name="CleanRelation">
|
||||
<spark xmlns="uri:oozie:spark-action:0.2">
|
||||
<master>yarn</master>
|
||||
<mode>cluster</mode>
|
||||
<name>Clean Relations</name>
|
||||
<class>eu.dnetlib.dhp.oa.dedup.SparkCleanRelation</class>
|
||||
<jar>dhp-dedup-openaire-${projectVersion}.jar</jar>
|
||||
<spark-opts>
|
||||
--executor-memory=${sparkExecutorMemory}
|
||||
--conf spark.executor.memoryOverhead=${sparkExecutorMemoryOverhead}
|
||||
--executor-cores=${sparkExecutorCores}
|
||||
--driver-memory=${sparkDriverMemory}
|
||||
--conf spark.extraListeners=${spark2ExtraListeners}
|
||||
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
|
||||
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
|
||||
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
|
||||
--conf spark.sql.shuffle.partitions=15000
|
||||
</spark-opts>
|
||||
<arg>--graphBasePath</arg><arg>${graphBasePath}</arg>
|
||||
<arg>--inputPath</arg><arg>${workingPath}/propagaterelation/relation</arg>
|
||||
<arg>--outputPath</arg><arg>${graphOutputPath}/relation</arg>
|
||||
</spark>
|
||||
<ok to="group_entities"/>
|
||||
<error to="Kill"/>
|
||||
</action>
|
||||
|
@ -152,31 +126,7 @@
|
|||
--conf spark.sql.shuffle.partitions=15000
|
||||
</spark-opts>
|
||||
<arg>--graphInputPath</arg><arg>${graphBasePath}</arg>
|
||||
<arg>--outputPath</arg><arg>${workingPath}/grouped_entities</arg>
|
||||
</spark>
|
||||
<ok to="dispatch_entities"/>
|
||||
<error to="Kill"/>
|
||||
</action>
|
||||
|
||||
<action name="dispatch_entities">
|
||||
<spark xmlns="uri:oozie:spark-action:0.2">
|
||||
<master>yarn</master>
|
||||
<mode>cluster</mode>
|
||||
<name>Dispatch grouped entitities</name>
|
||||
<class>eu.dnetlib.dhp.oa.merge.DispatchEntitiesSparkJob</class>
|
||||
<jar>dhp-dedup-openaire-${projectVersion}.jar</jar>
|
||||
<spark-opts>
|
||||
--executor-memory=${sparkExecutorMemory}
|
||||
--conf spark.executor.memoryOverhead=${sparkExecutorMemoryOverhead}
|
||||
--executor-cores=${sparkExecutorCores}
|
||||
--driver-memory=${sparkDriverMemory}
|
||||
--conf spark.extraListeners=${spark2ExtraListeners}
|
||||
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
|
||||
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
|
||||
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
|
||||
--conf spark.sql.shuffle.partitions=7680
|
||||
</spark-opts>
|
||||
<arg>--inputPath</arg><arg>${workingPath}/grouped_entities</arg>
|
||||
<arg>--checkpointPath</arg><arg>${workingPath}/grouped_entities</arg>
|
||||
<arg>--outputPath</arg><arg>${graphOutputPath}</arg>
|
||||
<arg>--filterInvisible</arg><arg>${filterInvisible}</arg>
|
||||
</spark>
|
||||
|
|
|
@ -3,7 +3,6 @@ package eu.dnetlib.dhp.oa.dedup;
|
|||
|
||||
import static java.nio.file.Files.createTempDirectory;
|
||||
|
||||
import static org.apache.spark.sql.functions.col;
|
||||
import static org.apache.spark.sql.functions.count;
|
||||
import static org.junit.jupiter.api.Assertions.*;
|
||||
import static org.mockito.Mockito.lenient;
|
||||
|
@ -23,14 +22,13 @@ import java.util.stream.Collectors;
|
|||
import org.apache.commons.io.FileUtils;
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.spark.SparkConf;
|
||||
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.FilterFunction;
|
||||
import org.apache.spark.api.java.function.MapFunction;
|
||||
import org.apache.spark.api.java.function.PairFunction;
|
||||
import org.apache.spark.sql.*;
|
||||
import org.apache.spark.sql.Dataset;
|
||||
import org.apache.spark.sql.Encoders;
|
||||
import org.apache.spark.sql.Row;
|
||||
import org.apache.spark.sql.SparkSession;
|
||||
import org.junit.jupiter.api.*;
|
||||
import org.junit.jupiter.api.extension.ExtendWith;
|
||||
import org.mockito.Mock;
|
||||
|
@ -46,8 +44,6 @@ import eu.dnetlib.dhp.schema.common.ModelConstants;
|
|||
import eu.dnetlib.dhp.schema.oaf.*;
|
||||
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException;
|
||||
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService;
|
||||
import eu.dnetlib.pace.util.MapDocumentUtil;
|
||||
import scala.Tuple2;
|
||||
|
||||
@ExtendWith(MockitoExtension.class)
|
||||
@TestMethodOrder(MethodOrderer.OrderAnnotation.class)
|
||||
|
@ -62,6 +58,8 @@ public class SparkDedupTest implements Serializable {
|
|||
private static String testGraphBasePath;
|
||||
private static String testOutputBasePath;
|
||||
private static String testDedupGraphBasePath;
|
||||
private static String testConsistencyGraphBasePath;
|
||||
|
||||
private static final String testActionSetId = "test-orchestrator";
|
||||
private static String whitelistPath;
|
||||
private static List<String> whiteList;
|
||||
|
@ -75,6 +73,7 @@ public class SparkDedupTest implements Serializable {
|
|||
.get(SparkDedupTest.class.getResource("/eu/dnetlib/dhp/dedup/entities").toURI())
|
||||
.toFile()
|
||||
.getAbsolutePath();
|
||||
|
||||
testOutputBasePath = createTempDirectory(SparkDedupTest.class.getSimpleName() + "-")
|
||||
.toAbsolutePath()
|
||||
.toString();
|
||||
|
@ -83,6 +82,10 @@ public class SparkDedupTest implements Serializable {
|
|||
.toAbsolutePath()
|
||||
.toString();
|
||||
|
||||
testConsistencyGraphBasePath = createTempDirectory(SparkDedupTest.class.getSimpleName() + "-")
|
||||
.toAbsolutePath()
|
||||
.toString();
|
||||
|
||||
whitelistPath = Paths
|
||||
.get(SparkDedupTest.class.getResource("/eu/dnetlib/dhp/dedup/whitelist.simrels.txt").toURI())
|
||||
.toFile()
|
||||
|
@ -674,22 +677,45 @@ public class SparkDedupTest implements Serializable {
|
|||
assertEquals(mergedOrp, deletedOrp);
|
||||
}
|
||||
|
||||
@Test
|
||||
@Order(6)
|
||||
void copyRelationsNoOpenorgsTest() throws Exception {
|
||||
|
||||
ArgumentApplicationParser parser = new ArgumentApplicationParser(
|
||||
IOUtils
|
||||
.toString(
|
||||
SparkCopyRelationsNoOpenorgs.class
|
||||
.getResourceAsStream(
|
||||
"/eu/dnetlib/dhp/oa/dedup/updateEntity_parameters.json")));
|
||||
parser
|
||||
.parseArgument(
|
||||
new String[] {
|
||||
"-i", testGraphBasePath, "-w", testOutputBasePath, "-o", testDedupGraphBasePath
|
||||
});
|
||||
|
||||
new SparkCopyRelationsNoOpenorgs(parser, spark).run(isLookUpService);
|
||||
|
||||
final Dataset<Row> outputRels = spark.read().text(testDedupGraphBasePath + "/relation");
|
||||
|
||||
System.out.println(outputRels.count());
|
||||
// assertEquals(2382, outputRels.count());
|
||||
}
|
||||
|
||||
@Test
|
||||
@Order(7)
|
||||
void propagateRelationTest() throws Exception {
|
||||
|
||||
ArgumentApplicationParser parser = new ArgumentApplicationParser(
|
||||
classPathResourceAsString("/eu/dnetlib/dhp/oa/dedup/propagateRelation_parameters.json"));
|
||||
String outputRelPath = testDedupGraphBasePath + "/propagaterelation";
|
||||
parser
|
||||
.parseArgument(
|
||||
new String[] {
|
||||
"-i", testGraphBasePath, "-w", testOutputBasePath, "-o", outputRelPath
|
||||
"-i", testDedupGraphBasePath, "-w", testOutputBasePath, "-o", testConsistencyGraphBasePath
|
||||
});
|
||||
|
||||
new SparkPropagateRelation(parser, spark).run(isLookUpService);
|
||||
|
||||
long relations = jsc.textFile(outputRelPath + "/relation").count();
|
||||
long relations = jsc.textFile(testDedupGraphBasePath + "/relation").count();
|
||||
|
||||
// assertEquals(4860, relations);
|
||||
System.out.println("relations = " + relations);
|
||||
|
@ -699,95 +725,52 @@ public class SparkDedupTest implements Serializable {
|
|||
.read()
|
||||
.load(DedupUtility.createMergeRelPath(testOutputBasePath, "*", "*"))
|
||||
.as(Encoders.bean(Relation.class));
|
||||
final JavaPairRDD<String, String> mergedIds = mergeRels
|
||||
.where("relClass == 'merges'")
|
||||
.select(mergeRels.col("target"))
|
||||
.distinct()
|
||||
.toJavaRDD()
|
||||
.mapToPair(
|
||||
(PairFunction<Row, String, String>) r -> new Tuple2<String, String>(r.getString(0), "d"));
|
||||
|
||||
JavaRDD<String> toCheck = jsc
|
||||
.textFile(outputRelPath + "/relation")
|
||||
.mapToPair(json -> new Tuple2<>(MapDocumentUtil.getJPathString("$.source", json), json))
|
||||
.join(mergedIds)
|
||||
.map(t -> t._2()._1())
|
||||
.mapToPair(json -> new Tuple2<>(MapDocumentUtil.getJPathString("$.target", json), json))
|
||||
.join(mergedIds)
|
||||
.map(t -> t._2()._1());
|
||||
Dataset<Row> inputRels = spark
|
||||
.read()
|
||||
.json(testDedupGraphBasePath + "/relation");
|
||||
|
||||
long deletedbyinference = toCheck.filter(this::isDeletedByInference).count();
|
||||
long updated = toCheck.count();
|
||||
Dataset<Row> outputRels = spark
|
||||
.read()
|
||||
.json(testConsistencyGraphBasePath + "/relation");
|
||||
|
||||
assertEquals(updated, deletedbyinference);
|
||||
assertEquals(
|
||||
0, outputRels
|
||||
.filter("dataInfo.deletedbyinference == true OR dataInfo.invisible == true")
|
||||
.count());
|
||||
|
||||
assertEquals(
|
||||
5, outputRels
|
||||
.filter("relClass NOT IN ('merges', 'isMergedIn')")
|
||||
.count());
|
||||
|
||||
assertEquals(5 + mergeRels.count(), outputRels.count());
|
||||
}
|
||||
|
||||
@Test
|
||||
@Order(8)
|
||||
void testCleanBaseRelations() throws Exception {
|
||||
ArgumentApplicationParser parser = new ArgumentApplicationParser(
|
||||
classPathResourceAsString("/eu/dnetlib/dhp/oa/dedup/cleanRelation_parameters.json"));
|
||||
|
||||
// append dangling relations to be cleaned up
|
||||
void testCleanedPropagatedRelations() throws Exception {
|
||||
Dataset<Row> df_before = spark
|
||||
.read()
|
||||
.schema(Encoders.bean(Relation.class).schema())
|
||||
.json(testGraphBasePath + "/relation");
|
||||
Dataset<Row> df_input = df_before
|
||||
.unionByName(df_before.drop("source").withColumn("source", functions.lit("n/a")))
|
||||
.unionByName(df_before.drop("target").withColumn("target", functions.lit("n/a")));
|
||||
df_input.write().mode(SaveMode.Overwrite).json(testOutputBasePath + "_tmp");
|
||||
|
||||
parser
|
||||
.parseArgument(
|
||||
new String[] {
|
||||
"--graphBasePath", testGraphBasePath,
|
||||
"--inputPath", testGraphBasePath + "/relation",
|
||||
"--outputPath", testDedupGraphBasePath + "/relation"
|
||||
});
|
||||
|
||||
new SparkCleanRelation(parser, spark).run(isLookUpService);
|
||||
.json(testDedupGraphBasePath + "/relation");
|
||||
|
||||
Dataset<Row> df_after = spark
|
||||
.read()
|
||||
.schema(Encoders.bean(Relation.class).schema())
|
||||
.json(testDedupGraphBasePath + "/relation");
|
||||
|
||||
assertNotEquals(df_before.count(), df_input.count());
|
||||
assertNotEquals(df_input.count(), df_after.count());
|
||||
assertEquals(5, df_after.count());
|
||||
}
|
||||
|
||||
@Test
|
||||
@Order(9)
|
||||
void testCleanDedupedRelations() throws Exception {
|
||||
ArgumentApplicationParser parser = new ArgumentApplicationParser(
|
||||
classPathResourceAsString("/eu/dnetlib/dhp/oa/dedup/cleanRelation_parameters.json"));
|
||||
|
||||
String inputRelPath = testDedupGraphBasePath + "/propagaterelation/relation";
|
||||
|
||||
// append dangling relations to be cleaned up
|
||||
Dataset<Row> df_before = spark.read().schema(Encoders.bean(Relation.class).schema()).json(inputRelPath);
|
||||
|
||||
df_before.filter(col("dataInfo.deletedbyinference").notEqual(true)).show(50, false);
|
||||
|
||||
parser
|
||||
.parseArgument(
|
||||
new String[] {
|
||||
"--graphBasePath", testGraphBasePath,
|
||||
"--inputPath", inputRelPath,
|
||||
"--outputPath", testDedupGraphBasePath + "/relation"
|
||||
});
|
||||
|
||||
new SparkCleanRelation(parser, spark).run(isLookUpService);
|
||||
|
||||
Dataset<Row> df_after = spark
|
||||
.read()
|
||||
.schema(Encoders.bean(Relation.class).schema())
|
||||
.json(testDedupGraphBasePath + "/relation");
|
||||
.json(testConsistencyGraphBasePath + "/relation");
|
||||
|
||||
assertNotEquals(df_before.count(), df_after.count());
|
||||
assertEquals(0, df_after.count());
|
||||
|
||||
assertEquals(
|
||||
0, df_after
|
||||
.filter("dataInfo.deletedbyinference == true OR dataInfo.invisible == true")
|
||||
.count());
|
||||
|
||||
assertEquals(
|
||||
5, df_after
|
||||
.filter("relClass NOT IN ('merges', 'isMergedIn')")
|
||||
.count());
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -813,6 +796,7 @@ public class SparkDedupTest implements Serializable {
|
|||
public static void finalCleanUp() throws IOException {
|
||||
FileUtils.deleteDirectory(new File(testOutputBasePath));
|
||||
FileUtils.deleteDirectory(new File(testDedupGraphBasePath));
|
||||
FileUtils.deleteDirectory(new File(testConsistencyGraphBasePath));
|
||||
}
|
||||
|
||||
public boolean isDeletedByInference(String s) {
|
||||
|
|
|
@ -3,6 +3,7 @@ package eu.dnetlib.dhp.oa.dedup;
|
|||
|
||||
import static java.nio.file.Files.createTempDirectory;
|
||||
|
||||
import static org.apache.spark.sql.functions.col;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.mockito.Mockito.lenient;
|
||||
|
||||
|
@ -15,10 +16,6 @@ import java.nio.file.Paths;
|
|||
import org.apache.commons.io.FileUtils;
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.spark.SparkConf;
|
||||
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;
|
||||
|
@ -33,8 +30,6 @@ import eu.dnetlib.dhp.application.ArgumentApplicationParser;
|
|||
import eu.dnetlib.dhp.schema.oaf.Relation;
|
||||
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException;
|
||||
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService;
|
||||
import eu.dnetlib.pace.util.MapDocumentUtil;
|
||||
import scala.Tuple2;
|
||||
|
||||
@ExtendWith(MockitoExtension.class)
|
||||
@TestMethodOrder(MethodOrderer.OrderAnnotation.class)
|
||||
|
@ -44,11 +39,11 @@ public class SparkOpenorgsProvisionTest implements Serializable {
|
|||
ISLookUpService isLookUpService;
|
||||
|
||||
private static SparkSession spark;
|
||||
private static JavaSparkContext jsc;
|
||||
|
||||
private static String testGraphBasePath;
|
||||
private static String testOutputBasePath;
|
||||
private static String testDedupGraphBasePath;
|
||||
private static String testConsistencyGraphBasePath;
|
||||
private static final String testActionSetId = "test-orchestrator";
|
||||
|
||||
@BeforeAll
|
||||
|
@ -64,6 +59,9 @@ public class SparkOpenorgsProvisionTest implements Serializable {
|
|||
testDedupGraphBasePath = createTempDirectory(SparkOpenorgsProvisionTest.class.getSimpleName() + "-")
|
||||
.toAbsolutePath()
|
||||
.toString();
|
||||
testConsistencyGraphBasePath = createTempDirectory(SparkOpenorgsProvisionTest.class.getSimpleName() + "-")
|
||||
.toAbsolutePath()
|
||||
.toString();
|
||||
|
||||
FileUtils.deleteDirectory(new File(testOutputBasePath));
|
||||
FileUtils.deleteDirectory(new File(testDedupGraphBasePath));
|
||||
|
@ -76,8 +74,13 @@ public class SparkOpenorgsProvisionTest implements Serializable {
|
|||
.master("local[*]")
|
||||
.config(conf)
|
||||
.getOrCreate();
|
||||
}
|
||||
|
||||
jsc = JavaSparkContext.fromSparkContext(spark.sparkContext());
|
||||
@AfterAll
|
||||
public static void finalCleanUp() throws IOException {
|
||||
FileUtils.deleteDirectory(new File(testOutputBasePath));
|
||||
FileUtils.deleteDirectory(new File(testDedupGraphBasePath));
|
||||
FileUtils.deleteDirectory(new File(testConsistencyGraphBasePath));
|
||||
}
|
||||
|
||||
@BeforeEach
|
||||
|
@ -186,26 +189,21 @@ public class SparkOpenorgsProvisionTest implements Serializable {
|
|||
|
||||
new SparkUpdateEntity(parser, spark).run(isLookUpService);
|
||||
|
||||
long organizations = jsc.textFile(testDedupGraphBasePath + "/organization").count();
|
||||
Dataset<Row> organizations = spark.read().json(testDedupGraphBasePath + "/organization");
|
||||
|
||||
long mergedOrgs = spark
|
||||
Dataset<Row> mergedOrgs = spark
|
||||
.read()
|
||||
.load(testOutputBasePath + "/" + testActionSetId + "/organization_mergerel")
|
||||
.as(Encoders.bean(Relation.class))
|
||||
.where("relClass=='merges'")
|
||||
.javaRDD()
|
||||
.map(Relation::getTarget)
|
||||
.distinct()
|
||||
.count();
|
||||
.select("target")
|
||||
.distinct();
|
||||
|
||||
assertEquals(80, organizations);
|
||||
assertEquals(80, organizations.count());
|
||||
|
||||
long deletedOrgs = jsc
|
||||
.textFile(testDedupGraphBasePath + "/organization")
|
||||
.filter(this::isDeletedByInference)
|
||||
.count();
|
||||
Dataset<Row> deletedOrgs = organizations
|
||||
.filter("dataInfo.deletedbyinference = TRUE");
|
||||
|
||||
assertEquals(mergedOrgs, deletedOrgs);
|
||||
assertEquals(mergedOrgs.count(), deletedOrgs.count());
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -226,10 +224,9 @@ public class SparkOpenorgsProvisionTest implements Serializable {
|
|||
|
||||
new SparkCopyRelationsNoOpenorgs(parser, spark).run(isLookUpService);
|
||||
|
||||
final JavaRDD<String> rels = jsc.textFile(testDedupGraphBasePath + "/relation");
|
||||
|
||||
assertEquals(2382, rels.count());
|
||||
final Dataset<Row> outputRels = spark.read().text(testDedupGraphBasePath + "/relation");
|
||||
|
||||
assertEquals(2382, outputRels.count());
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -244,51 +241,41 @@ public class SparkOpenorgsProvisionTest implements Serializable {
|
|||
parser
|
||||
.parseArgument(
|
||||
new String[] {
|
||||
"-i", testGraphBasePath, "-w", testOutputBasePath, "-o", testDedupGraphBasePath
|
||||
"-i", testDedupGraphBasePath, "-w", testOutputBasePath, "-o", testConsistencyGraphBasePath
|
||||
});
|
||||
|
||||
new SparkPropagateRelation(parser, spark).run(isLookUpService);
|
||||
|
||||
long relations = jsc.textFile(testDedupGraphBasePath + "/relation").count();
|
||||
|
||||
assertEquals(4896, relations);
|
||||
|
||||
// check deletedbyinference
|
||||
final Dataset<Relation> mergeRels = spark
|
||||
.read()
|
||||
.load(DedupUtility.createMergeRelPath(testOutputBasePath, "*", "*"))
|
||||
.as(Encoders.bean(Relation.class));
|
||||
final JavaPairRDD<String, String> mergedIds = mergeRels
|
||||
|
||||
Dataset<Row> inputRels = spark
|
||||
.read()
|
||||
.json(testDedupGraphBasePath + "/relation");
|
||||
|
||||
Dataset<Row> outputRels = spark
|
||||
.read()
|
||||
.json(testConsistencyGraphBasePath + "/relation");
|
||||
|
||||
final Dataset<Row> mergedIds = mergeRels
|
||||
.where("relClass == 'merges'")
|
||||
.select(mergeRels.col("target"))
|
||||
.distinct()
|
||||
.toJavaRDD()
|
||||
.mapToPair(
|
||||
(PairFunction<Row, String, String>) r -> new Tuple2<String, String>(r.getString(0), "d"));
|
||||
.select(col("target").as("id"))
|
||||
.distinct();
|
||||
|
||||
JavaRDD<String> toCheck = jsc
|
||||
.textFile(testDedupGraphBasePath + "/relation")
|
||||
.mapToPair(json -> new Tuple2<>(MapDocumentUtil.getJPathString("$.source", json), json))
|
||||
.join(mergedIds)
|
||||
.map(t -> t._2()._1())
|
||||
.mapToPair(json -> new Tuple2<>(MapDocumentUtil.getJPathString("$.target", json), json))
|
||||
.join(mergedIds)
|
||||
.map(t -> t._2()._1());
|
||||
Dataset<Row> toUpdateRels = inputRels
|
||||
.as("rel")
|
||||
.join(mergedIds.as("s"), col("rel.source").equalTo(col("s.id")), "left_outer")
|
||||
.join(mergedIds.as("t"), col("rel.target").equalTo(col("t.id")), "left_outer")
|
||||
.filter("s.id IS NOT NULL OR t.id IS NOT NULL")
|
||||
.distinct();
|
||||
|
||||
long deletedbyinference = toCheck.filter(this::isDeletedByInference).count();
|
||||
long updated = toCheck.count();
|
||||
Dataset<Row> updatedRels = inputRels
|
||||
.select("source", "target", "relClass")
|
||||
.except(outputRels.select("source", "target", "relClass"));
|
||||
|
||||
assertEquals(updated, deletedbyinference);
|
||||
assertEquals(toUpdateRels.count(), updatedRels.count());
|
||||
assertEquals(140, outputRels.count());
|
||||
}
|
||||
|
||||
@AfterAll
|
||||
public static void finalCleanUp() throws IOException {
|
||||
FileUtils.deleteDirectory(new File(testOutputBasePath));
|
||||
FileUtils.deleteDirectory(new File(testDedupGraphBasePath));
|
||||
}
|
||||
|
||||
public boolean isDeletedByInference(String s) {
|
||||
return s.contains("\"deletedbyinference\":true");
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -96,30 +96,7 @@
|
|||
--conf spark.sql.shuffle.partitions=15000
|
||||
</spark-opts>
|
||||
<arg>--graphInputPath</arg><arg>${graphBasePath}</arg>
|
||||
<arg>--outputPath</arg><arg>${workingPath}/grouped_entities</arg>
|
||||
</spark>
|
||||
<ok to="dispatch_entities"/>
|
||||
<error to="Kill"/>
|
||||
</action>
|
||||
|
||||
<action name="dispatch_entities">
|
||||
<spark xmlns="uri:oozie:spark-action:0.2">
|
||||
<master>yarn</master>
|
||||
<mode>cluster</mode>
|
||||
<name>Dispatch grouped entities</name>
|
||||
<class>eu.dnetlib.dhp.oa.merge.DispatchEntitiesSparkJob</class>
|
||||
<jar>dhp-graph-mapper-${projectVersion}.jar</jar>
|
||||
<spark-opts>
|
||||
--executor-cores=${sparkExecutorCores}
|
||||
--executor-memory=${sparkExecutorMemory}
|
||||
--driver-memory=${sparkDriverMemory}
|
||||
--conf spark.extraListeners=${spark2ExtraListeners}
|
||||
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
|
||||
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
|
||||
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
|
||||
--conf spark.sql.shuffle.partitions=7680
|
||||
</spark-opts>
|
||||
<arg>--inputPath</arg><arg>${workingPath}/grouped_entities</arg>
|
||||
<arg>--checkpointPath</arg><arg>${workingPath}/grouped_entities</arg>
|
||||
<arg>--outputPath</arg><arg>${graphOutputPath}</arg>
|
||||
<arg>--filterInvisible</arg><arg>${filterInvisible}</arg>
|
||||
</spark>
|
||||
|
|
|
@ -10,7 +10,6 @@ import java.nio.file.Path;
|
|||
import java.nio.file.Paths;
|
||||
|
||||
import org.apache.commons.io.FileUtils;
|
||||
import org.apache.commons.lang3.StringUtils;
|
||||
import org.apache.spark.SparkConf;
|
||||
import org.apache.spark.api.java.function.FilterFunction;
|
||||
import org.apache.spark.api.java.function.MapFunction;
|
||||
|
@ -23,9 +22,9 @@ import com.fasterxml.jackson.databind.DeserializationFeature;
|
|||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
|
||||
import eu.dnetlib.dhp.common.HdfsSupport;
|
||||
import eu.dnetlib.dhp.oa.merge.DispatchEntitiesSparkJob;
|
||||
import eu.dnetlib.dhp.oa.merge.GroupEntitiesSparkJob;
|
||||
import eu.dnetlib.dhp.schema.common.ModelSupport;
|
||||
import eu.dnetlib.dhp.schema.oaf.OafEntity;
|
||||
import eu.dnetlib.dhp.schema.oaf.Result;
|
||||
import eu.dnetlib.dhp.utils.DHPUtils;
|
||||
|
||||
|
@ -40,8 +39,9 @@ public class GroupEntitiesSparkJobTest {
|
|||
private static Path workingDir;
|
||||
private Path dataInputPath;
|
||||
|
||||
private Path groupEntityPath;
|
||||
private Path dispatchEntityPath;
|
||||
private Path checkpointPath;
|
||||
|
||||
private Path outputPath;
|
||||
|
||||
@BeforeAll
|
||||
public static void beforeAll() throws IOException {
|
||||
|
@ -58,8 +58,8 @@ public class GroupEntitiesSparkJobTest {
|
|||
@BeforeEach
|
||||
public void beforeEach() throws IOException, URISyntaxException {
|
||||
dataInputPath = Paths.get(ClassLoader.getSystemResource("eu/dnetlib/dhp/oa/graph/group").toURI());
|
||||
groupEntityPath = workingDir.resolve("grouped_entity");
|
||||
dispatchEntityPath = workingDir.resolve("dispatched_entity");
|
||||
checkpointPath = workingDir.resolve("grouped_entity");
|
||||
outputPath = workingDir.resolve("dispatched_entity");
|
||||
}
|
||||
|
||||
@AfterAll
|
||||
|
@ -76,39 +76,28 @@ public class GroupEntitiesSparkJobTest {
|
|||
Boolean.FALSE.toString(),
|
||||
"-graphInputPath",
|
||||
dataInputPath.toString(),
|
||||
"-checkpointPath",
|
||||
checkpointPath.toString(),
|
||||
"-outputPath",
|
||||
groupEntityPath.toString()
|
||||
outputPath.toString(),
|
||||
"-filterInvisible",
|
||||
Boolean.FALSE.toString()
|
||||
});
|
||||
|
||||
Dataset<Result> output = spark
|
||||
Dataset<OafEntity> checkpointTable = spark
|
||||
.read()
|
||||
.textFile(groupEntityPath.toString())
|
||||
.map((MapFunction<String, String>) s -> StringUtils.substringAfter(s, "|"), Encoders.STRING())
|
||||
.map((MapFunction<String, Result>) s -> mapper.readValue(s, Result.class), Encoders.bean(Result.class));
|
||||
.load(checkpointPath.toString())
|
||||
.selectExpr("COALESCE(*)")
|
||||
.as(Encoders.kryo(OafEntity.class));
|
||||
|
||||
assertEquals(
|
||||
1,
|
||||
output
|
||||
checkpointTable
|
||||
.filter(
|
||||
(FilterFunction<Result>) r -> "50|doi_________::09821844208a5cd6300b2bfb13bca1b9"
|
||||
(FilterFunction<OafEntity>) r -> "50|doi_________::09821844208a5cd6300b2bfb13bca1b9"
|
||||
.equals(r.getId()) &&
|
||||
r.getCollectedfrom().stream().anyMatch(kv -> kv.getValue().equalsIgnoreCase("zenodo")))
|
||||
.count());
|
||||
}
|
||||
|
||||
@Test
|
||||
@Order(2)
|
||||
void testDispatchEntities() throws Exception {
|
||||
DispatchEntitiesSparkJob.main(new String[] {
|
||||
"-isSparkSessionManaged",
|
||||
Boolean.FALSE.toString(),
|
||||
"-inputPath",
|
||||
groupEntityPath.toString(),
|
||||
"-outputPath",
|
||||
dispatchEntityPath.resolve(".").toString(),
|
||||
"-filterInvisible",
|
||||
Boolean.TRUE.toString()
|
||||
});
|
||||
|
||||
Dataset<Result> output = spark
|
||||
.read()
|
||||
|
@ -116,7 +105,7 @@ public class GroupEntitiesSparkJobTest {
|
|||
DHPUtils
|
||||
.toSeq(
|
||||
HdfsSupport
|
||||
.listFiles(dispatchEntityPath.toString(), spark.sparkContext().hadoopConfiguration())))
|
||||
.listFiles(outputPath.toString(), spark.sparkContext().hadoopConfiguration())))
|
||||
.map((MapFunction<String, Result>) s -> mapper.readValue(s, Result.class), Encoders.bean(Result.class));
|
||||
|
||||
assertEquals(3, output.count());
|
||||
|
|
|
@ -49,7 +49,7 @@ public class DownloadCsvTest {
|
|||
@Test
|
||||
void getUnibiFileTest() throws CollectorException, IOException, ClassNotFoundException {
|
||||
|
||||
String fileURL = "https://pub.uni-bielefeld.de/download/2944717/2944718/issn_gold_oa_version_4.csv";
|
||||
String fileURL = "https://pub.uni-bielefeld.de/download/2944717/2944718/issn_gold_oa_version_5.csv";
|
||||
|
||||
final String outputFile = workingDir + "/unibi_gold.json";
|
||||
new DownloadCSV()
|
||||
|
|
|
@ -1067,6 +1067,28 @@ class MappersTest {
|
|||
System.out.println("***************");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testD4ScienceTraining() throws IOException {
|
||||
final String xml = IOUtils
|
||||
.toString(Objects.requireNonNull(getClass().getResourceAsStream("d4science-1-training.xml")));
|
||||
final List<Oaf> list = new OdfToOafMapper(vocs, false, true).processMdRecord(xml);
|
||||
final OtherResearchProduct trainingMaterial = (OtherResearchProduct) list.get(0);
|
||||
System.out.println("***************");
|
||||
System.out.println(new ObjectMapper().writeValueAsString(trainingMaterial));
|
||||
System.out.println("***************");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testD4ScienceDataset() throws IOException {
|
||||
final String xml = IOUtils
|
||||
.toString(Objects.requireNonNull(getClass().getResourceAsStream("d4science-2-dataset.xml")));
|
||||
final List<Oaf> list = new OdfToOafMapper(vocs, false, true).processMdRecord(xml);
|
||||
final Dataset trainingMaterial = (Dataset) list.get(0);
|
||||
System.out.println("***************");
|
||||
System.out.println(new ObjectMapper().writeValueAsString(trainingMaterial));
|
||||
System.out.println("***************");
|
||||
}
|
||||
|
||||
@Test
|
||||
void testNotWellFormed() throws IOException {
|
||||
final String xml = IOUtils
|
||||
|
|
|
@ -0,0 +1,93 @@
|
|||
<?xml version="1.0" encoding="UTF-8"?>
|
||||
<oai:record xmlns:dr="http://www.driver-repository.eu/namespace/dr"
|
||||
xmlns:dri="http://www.driver-repository.eu/namespace/dri"
|
||||
xmlns:oaf="http://namespace.openaire.eu/oaf" xmlns:oai="http://www.openarchives.org/OAI/2.0/">
|
||||
<oai:header>
|
||||
<dri:objIdentifier>alessia_____::104c2d4ba8878c16fa824dce5b1bea57</dri:objIdentifier>
|
||||
<dri:recordIdentifier>12d8f77e-d66f-46f5-8d88-af7db23bc4c9</dri:recordIdentifier>
|
||||
<dri:dateOfCollection>2023-09-08T10:12:35.864+02:00</dri:dateOfCollection>
|
||||
<oaf:datasourceprefix>alessia_____</oaf:datasourceprefix>
|
||||
<dr:dateOfTransformation>2023-09-08T11:31:45.692+02:00</dr:dateOfTransformation>
|
||||
</oai:header>
|
||||
<oai:metadata>
|
||||
<datacite:resource
|
||||
xmlns:datacite="http://datacite.org/schema/kernel-4"
|
||||
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://datacite.org/schema/kernel-4 http://schema.datacite.org/meta/kernel-4/metadata.xsd">
|
||||
<datacite:identifier identifierType="URL">http://data.d4science.org/ctlg/ResourceCatalogue/visual_analytics_for_data_scientists</datacite:identifier>
|
||||
<datacite:alternateIdentifiers/>
|
||||
<datacite:creators>
|
||||
<datacite:creator>
|
||||
<datacite:creatorName>BRAGHIERI MARCO</datacite:creatorName>
|
||||
</datacite:creator>
|
||||
</datacite:creators>
|
||||
<datacite:titles>
|
||||
<datacite:title>Visual Analytics for Data Scientists</datacite:title>
|
||||
</datacite:titles>
|
||||
<datacite:publisher>SoBigData++</datacite:publisher>
|
||||
<datacite:publicationYear/>
|
||||
<datacite:dates>
|
||||
<datacite:date dateType="Issued"/>
|
||||
</datacite:dates>
|
||||
<datacite:resourceType resourceTypeGeneral="TrainingMaterial">TrainingMaterial</datacite:resourceType>
|
||||
<datacite:descriptions>
|
||||
<datacite:description descriptionType="Abstract">Participants to this module shall
|
||||
- Learn the principles and rules underlying the design of visual data
|
||||
representations and human-computer interactions
|
||||
- Understand, adapt and apply representative visual analytics methods and systems for diverse types
|
||||
of data and problems
|
||||
- Analyse and evaluate the structure and properties
|
||||
of data to select or devise appropriate methods for data exploration
|
||||
- Combine visualization, interactive techniques, and computational
|
||||
processing to develop practical data analysis for problem solving
|
||||
|
||||
(This teaching material on Visual Analytics for Data Scientists is part of a MSc module at City University London).
|
||||
|
||||
The author did not intend to violate any copyright on figures or content. In case you are the legal owner of any copyrighted content, please contact info@sobigdata.eu and we will immediately remove it</datacite:description>
|
||||
</datacite:descriptions>
|
||||
<datacite:subjects>
|
||||
<datacite:subject>Visual analytics</datacite:subject>
|
||||
</datacite:subjects>
|
||||
<datacite:formats>
|
||||
<datacite:format>Slides</datacite:format>
|
||||
<datacite:format>Other</datacite:format>
|
||||
<datacite:format>PDF</datacite:format>
|
||||
<datacite:format>PDF</datacite:format>
|
||||
<datacite:format>PDF</datacite:format>
|
||||
<datacite:format>PDF</datacite:format>
|
||||
<datacite:format>PDF</datacite:format>
|
||||
<datacite:format>PDF</datacite:format>
|
||||
<datacite:format>PDF</datacite:format>
|
||||
<datacite:format>PDF</datacite:format>
|
||||
<datacite:format>PDF</datacite:format>
|
||||
<datacite:format>PDF</datacite:format>
|
||||
<datacite:format>ZIP</datacite:format>
|
||||
</datacite:formats>
|
||||
</datacite:resource>
|
||||
<oaf:accessrights>OPEN</oaf:accessrights>
|
||||
<dr:CobjCategory type="other">0010</dr:CobjCategory>
|
||||
<oaf:dateAccepted/>
|
||||
<oaf:hostedBy id="alessia_____::alessia" name="Alessia"/>
|
||||
<oaf:collectedFrom id="alessia_____::alessia" name="Alessia"/>
|
||||
<oaf:license>other-open</oaf:license>
|
||||
<oaf:projectid>corda__h2020::871042</oaf:projectid>
|
||||
</oai:metadata>
|
||||
<about xmlns:dc="http://purl.org/dc/elements/1.1/"
|
||||
xmlns:prov="http://www.openarchives.org/OAI/2.0/provenance" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
|
||||
<provenance xmlns="http://www.openarchives.org/OAI/2.0/provenance" xsi:schemaLocation="http://www.openarchives.org/OAI/2.0/provenance http://www.openarchives.org/OAI/2.0/provenance.xsd">
|
||||
<originDescription altered="true" harvestDate="2023-09-08T10:12:35.864+02:00">
|
||||
<baseURL>https%3A%2F%2Fapi.d4science.org%2Fcatalogue%2Fitems</baseURL>
|
||||
<identifier/>
|
||||
<datestamp/>
|
||||
<metadataNamespace/>
|
||||
</originDescription>
|
||||
</provenance>
|
||||
<oaf:datainfo>
|
||||
<oaf:inferred>false</oaf:inferred>
|
||||
<oaf:deletedbyinference>false</oaf:deletedbyinference>
|
||||
<oaf:trust>0.9</oaf:trust>
|
||||
<oaf:inferenceprovenance/>
|
||||
<oaf:provenanceaction classid="sysimport:crosswalk"
|
||||
classname="Harvested" schemeid="dnet:provenanceActions" schemename="dnet:provenanceActions"/>
|
||||
</oaf:datainfo>
|
||||
</about>
|
||||
</oai:record>
|
|
@ -0,0 +1,72 @@
|
|||
<?xml version="1.0" encoding="UTF-8"?>
|
||||
<oai:record xmlns:dr="http://www.driver-repository.eu/namespace/dr"
|
||||
xmlns:dri="http://www.driver-repository.eu/namespace/dri"
|
||||
xmlns:oaf="http://namespace.openaire.eu/oaf" xmlns:oai="http://www.openarchives.org/OAI/2.0/">
|
||||
<oai:header>
|
||||
<dri:objIdentifier>alessia_____::028879484548f4e1c630e1c503e35231</dri:objIdentifier>
|
||||
<dri:recordIdentifier>4fed018e-c2ff-4afa-b7b5-1ca1beebf850</dri:recordIdentifier>
|
||||
<dri:dateOfCollection>2023-09-08T12:14:27.615+02:00</dri:dateOfCollection>
|
||||
<oaf:datasourceprefix>alessia_____</oaf:datasourceprefix>
|
||||
<dr:dateOfTransformation>2023-09-08T12:14:51.7+02:00</dr:dateOfTransformation>
|
||||
</oai:header>
|
||||
<oai:metadata>
|
||||
<datacite:resource
|
||||
xmlns:datacite="http://datacite.org/schema/kernel-4"
|
||||
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://datacite.org/schema/kernel-4 http://schema.datacite.org/meta/kernel-4/metadata.xsd">
|
||||
<datacite:identifier identifierType="URL">http://data.d4science.org/ctlg/ResourceCatalogue/city-to-city_migration</datacite:identifier>
|
||||
<datacite:alternateIdentifiers>
|
||||
<datacite:alternateIdentifier type="URL"/>
|
||||
</datacite:alternateIdentifiers>
|
||||
<datacite:creators>
|
||||
<datacite:creator>
|
||||
<datacite:creatorName>Pappalardo, Luca</datacite:creatorName>
|
||||
<datacite:affiliation/>
|
||||
<datacite:nameIdentifier nameIdentifierScheme="ORCID" schemeURI="http://orcid.org">0000-0002-1547-6007</datacite:nameIdentifier>
|
||||
</datacite:creator>
|
||||
</datacite:creators>
|
||||
<datacite:titles>
|
||||
<datacite:title>City-to-city migration</datacite:title>
|
||||
</datacite:titles>
|
||||
<datacite:publisher>SoBigData++</datacite:publisher>
|
||||
<datacite:publicationYear/>
|
||||
<datacite:dates>
|
||||
<datacite:date dateType="Issued">2018-02-15</datacite:date>
|
||||
</datacite:dates>
|
||||
<datacite:resourceType resourceTypeGeneral="Dataset">Dataset</datacite:resourceType>
|
||||
<datacite:descriptions>
|
||||
<datacite:description descriptionType="Abstract">Census data recording the migration of people between metropolitan areas in
|
||||
the US</datacite:description>
|
||||
</datacite:descriptions>
|
||||
<datacite:subjects>
|
||||
<datacite:subject>Human Mobility data</datacite:subject>
|
||||
</datacite:subjects>
|
||||
<datacite:formats/>
|
||||
</datacite:resource>
|
||||
<oaf:accessrights>OPEN</oaf:accessrights>
|
||||
<dr:CobjCategory type="dataset">0021</dr:CobjCategory>
|
||||
<oaf:dateAccepted>2018-02-15</oaf:dateAccepted>
|
||||
<oaf:hostedBy id="alessia_____::alessia" name="Alessia"/>
|
||||
<oaf:collectedFrom id="alessia_____::alessia" name="Alessia"/>
|
||||
<oaf:license>AFL-3.0</oaf:license>
|
||||
<oaf:projectid>corda__h2020::871042</oaf:projectid>
|
||||
</oai:metadata>
|
||||
<about xmlns:dc="http://purl.org/dc/elements/1.1/"
|
||||
xmlns:prov="http://www.openarchives.org/OAI/2.0/provenance" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
|
||||
<provenance xmlns="http://www.openarchives.org/OAI/2.0/provenance" xsi:schemaLocation="http://www.openarchives.org/OAI/2.0/provenance http://www.openarchives.org/OAI/2.0/provenance.xsd">
|
||||
<originDescription altered="true" harvestDate="2023-09-08T12:14:27.615+02:00">
|
||||
<baseURL>https%3A%2F%2Fapi.d4science.org%2Fcatalogue%2Fitems</baseURL>
|
||||
<identifier/>
|
||||
<datestamp/>
|
||||
<metadataNamespace/>
|
||||
</originDescription>
|
||||
</provenance>
|
||||
<oaf:datainfo>
|
||||
<oaf:inferred>false</oaf:inferred>
|
||||
<oaf:deletedbyinference>false</oaf:deletedbyinference>
|
||||
<oaf:trust>0.9</oaf:trust>
|
||||
<oaf:inferenceprovenance/>
|
||||
<oaf:provenanceaction classid="sysimport:crosswalk"
|
||||
classname="Harvested" schemeid="dnet:provenanceActions" schemename="dnet:provenanceActions"/>
|
||||
</oaf:datainfo>
|
||||
</about>
|
||||
</oai:record>
|
|
@ -24,10 +24,7 @@ import eu.dnetlib.dhp.oa.provision.model.RelatedEntity;
|
|||
import eu.dnetlib.dhp.oa.provision.model.RelatedEntityWrapper;
|
||||
import eu.dnetlib.dhp.oa.provision.utils.ContextMapper;
|
||||
import eu.dnetlib.dhp.oa.provision.utils.XmlRecordFactory;
|
||||
import eu.dnetlib.dhp.schema.oaf.Datasource;
|
||||
import eu.dnetlib.dhp.schema.oaf.Project;
|
||||
import eu.dnetlib.dhp.schema.oaf.Publication;
|
||||
import eu.dnetlib.dhp.schema.oaf.Relation;
|
||||
import eu.dnetlib.dhp.schema.oaf.*;
|
||||
|
||||
public class XmlRecordFactoryTest {
|
||||
|
||||
|
@ -196,4 +193,51 @@ public class XmlRecordFactoryTest {
|
|||
assertEquals("dnet:pid_types", ((Element) pids.get(0)).attribute("schemeid").getValue());
|
||||
assertEquals("dnet:pid_types", ((Element) pids.get(0)).attribute("schemename").getValue());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testD4ScienceTraining() throws DocumentException, IOException {
|
||||
final ContextMapper contextMapper = new ContextMapper();
|
||||
|
||||
final XmlRecordFactory xmlRecordFactory = new XmlRecordFactory(contextMapper, false,
|
||||
XmlConverterJob.schemaLocation);
|
||||
|
||||
final OtherResearchProduct p = OBJECT_MAPPER
|
||||
.readValue(
|
||||
IOUtils.toString(getClass().getResourceAsStream("d4science-1-training.json")),
|
||||
OtherResearchProduct.class);
|
||||
|
||||
final String xml = xmlRecordFactory.build(new JoinedEntity<>(p));
|
||||
|
||||
assertNotNull(xml);
|
||||
|
||||
final Document doc = new SAXReader().read(new StringReader(xml));
|
||||
|
||||
assertNotNull(doc);
|
||||
System.out.println(doc.asXML());
|
||||
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testD4ScienceDataset() throws DocumentException, IOException {
|
||||
final ContextMapper contextMapper = new ContextMapper();
|
||||
|
||||
final XmlRecordFactory xmlRecordFactory = new XmlRecordFactory(contextMapper, false,
|
||||
XmlConverterJob.schemaLocation);
|
||||
|
||||
final OtherResearchProduct p = OBJECT_MAPPER
|
||||
.readValue(
|
||||
IOUtils.toString(getClass().getResourceAsStream("d4science-2-dataset.json")),
|
||||
OtherResearchProduct.class);
|
||||
|
||||
final String xml = xmlRecordFactory.build(new JoinedEntity<>(p));
|
||||
|
||||
assertNotNull(xml);
|
||||
|
||||
final Document doc = new SAXReader().read(new StringReader(xml));
|
||||
|
||||
assertNotNull(doc);
|
||||
System.out.println(doc.asXML());
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
|
File diff suppressed because one or more lines are too long
File diff suppressed because one or more lines are too long
|
@ -39,7 +39,8 @@
|
|||
<switch>
|
||||
<!-- The default will be set as the normal start, a.k.a. get-doi-synonyms -->
|
||||
<!-- If any different condition is set, go to the corresponding start -->
|
||||
<case to="non-iterative-rankings">${wf:conf('resume') eq "rankings-start"}</case>
|
||||
<case to="spark-cc">${wf:conf('resume') eq "cc"}</case>
|
||||
<case to="spark-ram">${wf:conf('resume') eq "ram"}</case>
|
||||
<case to="spark-impulse">${wf:conf('resume') eq "impulse"}</case>
|
||||
<case to="spark-pagerank">${wf:conf('resume') eq "pagerank"}</case>
|
||||
<case to="spark-attrank">${wf:conf('resume') eq "attrank"}</case>
|
||||
|
@ -89,18 +90,11 @@
|
|||
<file>${nameNode}${wfAppPath}/create_openaire_ranking_graph.py#create_openaire_ranking_graph.py</file>
|
||||
</spark>
|
||||
|
||||
<ok to="non-iterative-rankings" />
|
||||
<ok to="spark-cc"/>
|
||||
<error to="openaire-graph-error" />
|
||||
|
||||
</action>
|
||||
|
||||
<!-- Citation Count and RAM are calculated in parallel-->
|
||||
<fork name="non-iterative-rankings">
|
||||
<path start="spark-cc"/>
|
||||
<!-- <path start="spark-impulse"/> -->
|
||||
<path start="spark-ram"/>
|
||||
</fork>
|
||||
|
||||
<!-- Run Citation Count calculation -->
|
||||
<action name="spark-cc">
|
||||
<spark xmlns="uri:oozie:spark-action:0.2">
|
||||
|
@ -129,7 +123,7 @@
|
|||
<file>${wfAppPath}/bip-ranker/CC.py#CC.py</file>
|
||||
</spark>
|
||||
|
||||
<ok to="join-non-iterative-rankings" />
|
||||
<ok to="spark-ram" />
|
||||
<error to="cc-fail" />
|
||||
|
||||
</action>
|
||||
|
@ -165,14 +159,11 @@
|
|||
<file>${wfAppPath}/bip-ranker/TAR.py#TAR.py</file>
|
||||
</spark>
|
||||
|
||||
<ok to="join-non-iterative-rankings" />
|
||||
<ok to="spark-impulse" />
|
||||
<error to="ram-fail" />
|
||||
|
||||
</action>
|
||||
|
||||
<!-- Join non-iterative methods -->
|
||||
<join name="join-non-iterative-rankings" to="spark-impulse"/>
|
||||
|
||||
<action name="spark-impulse">
|
||||
<spark xmlns="uri:oozie:spark-action:0.2">
|
||||
|
||||
|
|
|
@ -0,0 +1,110 @@
|
|||
<?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">
|
||||
<modelVersion>4.0.0</modelVersion>
|
||||
<parent>
|
||||
<groupId>eu.dnetlib.dhp</groupId>
|
||||
<artifactId>dhp-workflows</artifactId>
|
||||
<version>1.2.5-SNAPSHOT</version>
|
||||
</parent>
|
||||
<artifactId>dhp-swh</artifactId>
|
||||
|
||||
<dependencies>
|
||||
<dependency>
|
||||
<groupId>org.apache.spark</groupId>
|
||||
<artifactId>spark-core_${scala.binary.version}</artifactId>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>org.apache.spark</groupId>
|
||||
<artifactId>spark-sql_${scala.binary.version}</artifactId>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>eu.dnetlib.dhp</groupId>
|
||||
<artifactId>dhp-common</artifactId>
|
||||
<version>${project.version}</version>
|
||||
<exclusions>
|
||||
<exclusion>
|
||||
<groupId>net.sf.saxon</groupId>
|
||||
<artifactId>Saxon-HE</artifactId>
|
||||
</exclusion>
|
||||
</exclusions>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>dom4j</groupId>
|
||||
<artifactId>dom4j</artifactId>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>xml-apis</groupId>
|
||||
<artifactId>xml-apis</artifactId>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>jaxen</groupId>
|
||||
<artifactId>jaxen</artifactId>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>org.apache.hadoop</groupId>
|
||||
<artifactId>hadoop-distcp</artifactId>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>eu.dnetlib</groupId>
|
||||
<artifactId>dnet-actionmanager-api</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>eu.dnetlib</groupId>
|
||||
<artifactId>dnet-actionmanager-common</artifactId>
|
||||
<exclusions>
|
||||
<exclusion>
|
||||
<groupId>eu.dnetlib</groupId>
|
||||
<artifactId>dnet-openaireplus-mapping-utils</artifactId>
|
||||
</exclusion>
|
||||
<exclusion>
|
||||
<groupId>saxonica</groupId>
|
||||
<artifactId>saxon</artifactId>
|
||||
</exclusion>
|
||||
<exclusion>
|
||||
<groupId>saxonica</groupId>
|
||||
<artifactId>saxon-dom</artifactId>
|
||||
</exclusion>
|
||||
<exclusion>
|
||||
<groupId>jgrapht</groupId>
|
||||
<artifactId>jgrapht</artifactId>
|
||||
</exclusion>
|
||||
<exclusion>
|
||||
<groupId>net.sf.ehcache</groupId>
|
||||
<artifactId>ehcache</artifactId>
|
||||
</exclusion>
|
||||
<exclusion>
|
||||
<groupId>org.springframework</groupId>
|
||||
<artifactId>spring-test</artifactId>
|
||||
</exclusion>
|
||||
<exclusion>
|
||||
<groupId>org.apache.*</groupId>
|
||||
<artifactId>*</artifactId>
|
||||
</exclusion>
|
||||
<exclusion>
|
||||
<groupId>apache</groupId>
|
||||
<artifactId>*</artifactId>
|
||||
</exclusion>
|
||||
</exclusions>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>org.apache.httpcomponents</groupId>
|
||||
<artifactId>httpclient</artifactId>
|
||||
<version>4.5.13</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.datanucleus</groupId>
|
||||
<artifactId>datanucleus-core</artifactId>
|
||||
<version>3.2.10</version>
|
||||
<scope>compile</scope>
|
||||
</dependency>
|
||||
|
||||
</dependencies>
|
||||
</project>
|
|
@ -0,0 +1,176 @@
|
|||
|
||||
package eu.dnetlib.dhp.swh;
|
||||
|
||||
import static eu.dnetlib.dhp.utils.DHPUtils.getHadoopConfiguration;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.net.URL;
|
||||
import java.text.SimpleDateFormat;
|
||||
import java.util.Date;
|
||||
import java.util.Optional;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import org.apache.commons.cli.ParseException;
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.io.SequenceFile;
|
||||
import org.apache.hadoop.io.Text;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
|
||||
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
|
||||
import eu.dnetlib.dhp.common.collection.CollectorException;
|
||||
import eu.dnetlib.dhp.common.collection.HttpClientParams;
|
||||
import eu.dnetlib.dhp.schema.common.ModelSupport;
|
||||
import eu.dnetlib.dhp.schema.oaf.utils.GraphCleaningFunctions;
|
||||
import eu.dnetlib.dhp.swh.models.LastVisitData;
|
||||
import eu.dnetlib.dhp.swh.utils.SWHConnection;
|
||||
import eu.dnetlib.dhp.swh.utils.SWHConstants;
|
||||
import eu.dnetlib.dhp.swh.utils.SWHUtils;
|
||||
|
||||
/**
|
||||
* Sends archive requests to the SWH API for those software repository URLs that are missing from them
|
||||
*
|
||||
* @author Serafeim Chatzopoulos
|
||||
*/
|
||||
public class ArchiveRepositoryURLs {
|
||||
|
||||
private static final Logger log = LoggerFactory.getLogger(ArchiveRepositoryURLs.class);
|
||||
private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
|
||||
|
||||
private static SWHConnection swhConnection = null;
|
||||
|
||||
public static void main(final String[] args) throws IOException, ParseException {
|
||||
final ArgumentApplicationParser argumentParser = new ArgumentApplicationParser(
|
||||
IOUtils
|
||||
.toString(
|
||||
CollectLastVisitRepositoryData.class
|
||||
.getResourceAsStream(
|
||||
"/eu/dnetlib/dhp/swh/input_archive_repository_urls.json")));
|
||||
argumentParser.parseArgument(args);
|
||||
|
||||
final String hdfsuri = argumentParser.get("namenode");
|
||||
log.info("hdfsURI: {}", hdfsuri);
|
||||
|
||||
final String inputPath = argumentParser.get("lastVisitsPath");
|
||||
log.info("inputPath: {}", inputPath);
|
||||
|
||||
final String outputPath = argumentParser.get("archiveRequestsPath");
|
||||
log.info("outputPath: {}", outputPath);
|
||||
|
||||
final Integer archiveThresholdInDays = Integer.parseInt(argumentParser.get("archiveThresholdInDays"));
|
||||
log.info("archiveThresholdInDays: {}", archiveThresholdInDays);
|
||||
|
||||
final String apiAccessToken = argumentParser.get("apiAccessToken");
|
||||
log.info("apiAccessToken: {}", apiAccessToken);
|
||||
|
||||
final HttpClientParams clientParams = SWHUtils.getClientParams(argumentParser);
|
||||
|
||||
swhConnection = new SWHConnection(clientParams, apiAccessToken);
|
||||
|
||||
final FileSystem fs = FileSystem.get(getHadoopConfiguration(hdfsuri));
|
||||
|
||||
archive(fs, inputPath, outputPath, archiveThresholdInDays);
|
||||
|
||||
}
|
||||
|
||||
private static void archive(FileSystem fs, String inputPath, String outputPath, Integer archiveThresholdInDays)
|
||||
throws IOException {
|
||||
|
||||
SequenceFile.Reader fr = SWHUtils.getSequenceFileReader(fs, inputPath);
|
||||
SequenceFile.Writer fw = SWHUtils.getSequenceFileWriter(fs, outputPath);
|
||||
|
||||
// Create key and value objects to hold data
|
||||
Text repoUrl = new Text();
|
||||
Text lastVisitData = new Text();
|
||||
|
||||
// Read key-value pairs from the SequenceFile and handle appropriately
|
||||
while (fr.next(repoUrl, lastVisitData)) {
|
||||
|
||||
String response = null;
|
||||
try {
|
||||
response = handleRecord(repoUrl.toString(), lastVisitData.toString(), archiveThresholdInDays);
|
||||
} catch (java.text.ParseException e) {
|
||||
log.error("Could not handle record with repo Url: {}", repoUrl.toString());
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
|
||||
// response is equal to null when no need for request
|
||||
if (response != null) {
|
||||
SWHUtils.appendToSequenceFile(fw, repoUrl.toString(), response);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
// Close readers
|
||||
fw.close();
|
||||
fr.close();
|
||||
}
|
||||
|
||||
public static String handleRecord(String repoUrl, String lastVisitData, Integer archiveThresholdInDays)
|
||||
throws IOException, java.text.ParseException {
|
||||
|
||||
log.info("{ Key: {}, Value: {} }", repoUrl, lastVisitData);
|
||||
|
||||
LastVisitData lastVisit = OBJECT_MAPPER.readValue(lastVisitData, LastVisitData.class);
|
||||
|
||||
// a previous attempt for archival has been made, and repository URL was not found
|
||||
// avoid performing the same archive request again
|
||||
if (lastVisit.getStatus() != null &&
|
||||
lastVisit.getStatus().equals(SWHConstants.VISIT_STATUS_NOT_FOUND)) {
|
||||
|
||||
log.info("Avoid request -- previous archive request returned NOT_FOUND");
|
||||
return null;
|
||||
}
|
||||
|
||||
// if we have last visit data
|
||||
if (lastVisit.getSnapshot() != null) {
|
||||
|
||||
String cleanDate = GraphCleaningFunctions.cleanDate(lastVisit.getDate());
|
||||
|
||||
// and the last visit date can be parsed
|
||||
if (cleanDate != null) {
|
||||
|
||||
SimpleDateFormat formatter = new SimpleDateFormat(ModelSupport.DATE_FORMAT);
|
||||
Date lastVisitDate = formatter.parse(cleanDate);
|
||||
|
||||
// OR last visit time < (now() - archiveThresholdInDays)
|
||||
long diffInMillies = Math.abs((new Date()).getTime() - lastVisitDate.getTime());
|
||||
long diffInDays = TimeUnit.DAYS.convert(diffInMillies, TimeUnit.MILLISECONDS);
|
||||
log.info("Date diff from now (in days): {}", diffInDays);
|
||||
|
||||
// do not perform a request, if the last visit date is no older than $archiveThresholdInDays
|
||||
if (archiveThresholdInDays >= diffInDays) {
|
||||
log.info("Avoid request -- no older than {} days", archiveThresholdInDays);
|
||||
return null;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// ELSE perform an archive request
|
||||
log.info("Perform archive request for: {}", repoUrl);
|
||||
|
||||
// if last visit data are available, re-use version control type,
|
||||
// else use the default one (i.e., git)
|
||||
String visitType = Optional
|
||||
.ofNullable(lastVisit.getType())
|
||||
.orElse(SWHConstants.DEFAULT_VISIT_TYPE);
|
||||
|
||||
URL url = new URL(String.format(SWHConstants.SWH_ARCHIVE_URL, visitType, repoUrl.trim()));
|
||||
|
||||
log.info("Sending archive request: {}", url);
|
||||
|
||||
String response;
|
||||
try {
|
||||
response = swhConnection.call(url.toString());
|
||||
} catch (CollectorException e) {
|
||||
log.error("Error in request: {}", url);
|
||||
response = "{}";
|
||||
}
|
||||
|
||||
return response;
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,119 @@
|
|||
|
||||
package eu.dnetlib.dhp.swh;
|
||||
|
||||
import static eu.dnetlib.dhp.utils.DHPUtils.getHadoopConfiguration;
|
||||
|
||||
import java.io.BufferedReader;
|
||||
import java.io.IOException;
|
||||
import java.net.URL;
|
||||
|
||||
import org.apache.commons.cli.ParseException;
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.io.SequenceFile;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
|
||||
import eu.dnetlib.dhp.common.collection.CollectorException;
|
||||
import eu.dnetlib.dhp.common.collection.HttpClientParams;
|
||||
import eu.dnetlib.dhp.swh.utils.SWHConnection;
|
||||
import eu.dnetlib.dhp.swh.utils.SWHConstants;
|
||||
import eu.dnetlib.dhp.swh.utils.SWHUtils;
|
||||
|
||||
/**
|
||||
* Given a file with software repository URLs, this class
|
||||
* collects last visit data from the Software Heritage API.
|
||||
*
|
||||
* @author Serafeim Chatzopoulos
|
||||
*/
|
||||
public class CollectLastVisitRepositoryData {
|
||||
|
||||
private static final Logger log = LoggerFactory.getLogger(CollectLastVisitRepositoryData.class);
|
||||
private static SWHConnection swhConnection = null;
|
||||
|
||||
public static void main(final String[] args)
|
||||
throws IOException, ParseException {
|
||||
final ArgumentApplicationParser argumentParser = new ArgumentApplicationParser(
|
||||
IOUtils
|
||||
.toString(
|
||||
CollectLastVisitRepositoryData.class
|
||||
.getResourceAsStream(
|
||||
"/eu/dnetlib/dhp/swh/input_collect_last_visit_repository_data.json")));
|
||||
argumentParser.parseArgument(args);
|
||||
|
||||
log.info("Java Xmx: {}m", Runtime.getRuntime().maxMemory() / (1024 * 1024));
|
||||
|
||||
final String hdfsuri = argumentParser.get("namenode");
|
||||
log.info("hdfsURI: {}", hdfsuri);
|
||||
|
||||
final String inputPath = argumentParser.get("softwareCodeRepositoryURLs");
|
||||
log.info("inputPath: {}", inputPath);
|
||||
|
||||
final String outputPath = argumentParser.get("lastVisitsPath");
|
||||
log.info("outputPath: {}", outputPath);
|
||||
|
||||
final String apiAccessToken = argumentParser.get("apiAccessToken");
|
||||
log.info("apiAccessToken: {}", apiAccessToken);
|
||||
|
||||
final HttpClientParams clientParams = SWHUtils.getClientParams(argumentParser);
|
||||
|
||||
swhConnection = new SWHConnection(clientParams, apiAccessToken);
|
||||
|
||||
final FileSystem fs = FileSystem.get(getHadoopConfiguration(hdfsuri));
|
||||
|
||||
collect(fs, inputPath, outputPath);
|
||||
|
||||
fs.close();
|
||||
}
|
||||
|
||||
private static void collect(FileSystem fs, String inputPath, String outputPath)
|
||||
throws IOException {
|
||||
|
||||
SequenceFile.Writer fw = SWHUtils.getSequenceFileWriter(fs, outputPath);
|
||||
|
||||
// Specify the HDFS directory path you want to read
|
||||
Path directoryPath = new Path(inputPath);
|
||||
|
||||
// List all files in the directory
|
||||
FileStatus[] partStatuses = fs.listStatus(directoryPath);
|
||||
|
||||
for (FileStatus partStatus : partStatuses) {
|
||||
|
||||
// Check if it's a file (not a directory)
|
||||
if (partStatus.isFile()) {
|
||||
handleFile(fs, partStatus.getPath(), fw);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
fw.close();
|
||||
}
|
||||
|
||||
private static void handleFile(FileSystem fs, Path partInputPath, SequenceFile.Writer fw)
|
||||
throws IOException {
|
||||
|
||||
BufferedReader br = SWHUtils.getFileReader(fs, partInputPath);
|
||||
|
||||
String repoUrl;
|
||||
while ((repoUrl = br.readLine()) != null) {
|
||||
|
||||
URL url = new URL(String.format(SWHConstants.SWH_LATEST_VISIT_URL, repoUrl.trim()));
|
||||
|
||||
String response;
|
||||
try {
|
||||
response = swhConnection.call(url.toString());
|
||||
} catch (CollectorException e) {
|
||||
log.error("Error in request: {}", url);
|
||||
response = "{}";
|
||||
}
|
||||
|
||||
SWHUtils.appendToSequenceFile(fw, repoUrl, response);
|
||||
}
|
||||
|
||||
br.close();
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,93 @@
|
|||
|
||||
package eu.dnetlib.dhp.swh;
|
||||
|
||||
import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkHiveSession;
|
||||
|
||||
import java.util.Optional;
|
||||
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.spark.SparkConf;
|
||||
import org.apache.spark.sql.Dataset;
|
||||
import org.apache.spark.sql.Row;
|
||||
import org.apache.spark.sql.SaveMode;
|
||||
import org.apache.spark.sql.SparkSession;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
|
||||
import eu.dnetlib.dhp.schema.oaf.Result;
|
||||
|
||||
/**
|
||||
* Collects unique software repository URLs in the Graph using Hive
|
||||
*
|
||||
* @author Serafeim Chatzopoulos
|
||||
*/
|
||||
public class CollectSoftwareRepositoryURLs {
|
||||
|
||||
private static final Logger log = LoggerFactory.getLogger(CollectSoftwareRepositoryURLs.class);
|
||||
|
||||
public static <I extends Result> void main(String[] args) throws Exception {
|
||||
|
||||
String jsonConfiguration = IOUtils
|
||||
.toString(
|
||||
CollectSoftwareRepositoryURLs.class
|
||||
.getResourceAsStream("/eu/dnetlib/dhp/swh/input_collect_software_repository_urls.json"));
|
||||
|
||||
final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration);
|
||||
parser.parseArgument(args);
|
||||
|
||||
final Boolean isSparkSessionManaged = Optional
|
||||
.ofNullable(parser.get("isSparkSessionManaged"))
|
||||
.map(Boolean::valueOf)
|
||||
.orElse(Boolean.TRUE);
|
||||
log.info("isSparkSessionManaged: {}", isSparkSessionManaged);
|
||||
|
||||
final String hiveDbName = parser.get("hiveDbName");
|
||||
log.info("hiveDbName: {}", hiveDbName);
|
||||
|
||||
final String outputPath = parser.get("softwareCodeRepositoryURLs");
|
||||
log.info("softwareCodeRepositoryURLs: {}", outputPath);
|
||||
|
||||
final String hiveMetastoreUris = parser.get("hiveMetastoreUris");
|
||||
log.info("hiveMetastoreUris: {}", hiveMetastoreUris);
|
||||
|
||||
final Integer softwareLimit = Integer.parseInt(parser.get("softwareLimit"));
|
||||
log.info("softwareLimit: {}", softwareLimit);
|
||||
|
||||
SparkConf conf = new SparkConf();
|
||||
conf.set("hive.metastore.uris", hiveMetastoreUris);
|
||||
|
||||
runWithSparkHiveSession(
|
||||
conf,
|
||||
isSparkSessionManaged,
|
||||
spark -> {
|
||||
doRun(spark, hiveDbName, softwareLimit, outputPath);
|
||||
});
|
||||
}
|
||||
|
||||
private static <I extends Result> void doRun(SparkSession spark, String hiveDbName, Integer limit,
|
||||
String outputPath) {
|
||||
|
||||
String queryTemplate = "SELECT distinct coderepositoryurl.value " +
|
||||
"FROM %s.software " +
|
||||
"WHERE coderepositoryurl.value IS NOT NULL " +
|
||||
"AND datainfo.deletedbyinference = FALSE " +
|
||||
"AND datainfo.invisible = FALSE ";
|
||||
|
||||
if (limit != null) {
|
||||
queryTemplate += String.format("LIMIT %s", limit);
|
||||
}
|
||||
|
||||
String query = String.format(queryTemplate, hiveDbName);
|
||||
|
||||
log.info("Hive query to fetch software code URLs: {}", query);
|
||||
|
||||
Dataset<Row> df = spark.sql(query);
|
||||
|
||||
// write distinct repository URLs
|
||||
df
|
||||
.write()
|
||||
.mode(SaveMode.Overwrite)
|
||||
.csv(outputPath);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,185 @@
|
|||
|
||||
package eu.dnetlib.dhp.swh;
|
||||
|
||||
import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession;
|
||||
import static org.apache.spark.sql.functions.col;
|
||||
|
||||
import java.io.Serializable;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.Optional;
|
||||
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.hadoop.io.Text;
|
||||
import org.apache.hadoop.io.compress.GzipCodec;
|
||||
import org.apache.hadoop.mapred.SequenceFileOutputFormat;
|
||||
import org.apache.spark.SparkConf;
|
||||
import org.apache.spark.api.java.JavaPairRDD;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.apache.spark.api.java.function.MapFunction;
|
||||
import org.apache.spark.sql.*;
|
||||
import org.apache.spark.sql.Dataset;
|
||||
import org.apache.spark.sql.types.DataTypes;
|
||||
import org.apache.spark.sql.types.StructField;
|
||||
import org.apache.spark.sql.types.StructType;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
|
||||
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
|
||||
import eu.dnetlib.dhp.schema.action.AtomicAction;
|
||||
import eu.dnetlib.dhp.schema.common.ModelConstants;
|
||||
import eu.dnetlib.dhp.schema.oaf.*;
|
||||
import eu.dnetlib.dhp.schema.oaf.utils.OafMapperUtils;
|
||||
import eu.dnetlib.dhp.swh.models.LastVisitData;
|
||||
import eu.dnetlib.dhp.swh.utils.SWHConstants;
|
||||
import scala.Tuple2;
|
||||
|
||||
/**
|
||||
* Creates action sets for Software Heritage data
|
||||
*
|
||||
* @author Serafeim Chatzopoulos
|
||||
*/
|
||||
public class PrepareSWHActionsets {
|
||||
|
||||
private static final Logger log = LoggerFactory.getLogger(PrepareSWHActionsets.class);
|
||||
private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
|
||||
|
||||
public static <I extends Result> void main(String[] args) throws Exception {
|
||||
|
||||
String jsonConfiguration = IOUtils
|
||||
.toString(
|
||||
PrepareSWHActionsets.class
|
||||
.getResourceAsStream(
|
||||
"/eu/dnetlib/dhp/swh/input_prepare_swh_actionsets.json"));
|
||||
|
||||
final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration);
|
||||
parser.parseArgument(args);
|
||||
|
||||
final Boolean isSparkSessionManaged = Optional
|
||||
.ofNullable(parser.get("isSparkSessionManaged"))
|
||||
.map(Boolean::valueOf)
|
||||
.orElse(Boolean.TRUE);
|
||||
log.info("isSparkSessionManaged: {}", isSparkSessionManaged);
|
||||
|
||||
final String inputPath = parser.get("lastVisitsPath");
|
||||
log.info("inputPath: {}", inputPath);
|
||||
|
||||
final String softwareInputPath = parser.get("softwareInputPath");
|
||||
log.info("softwareInputPath: {}", softwareInputPath);
|
||||
|
||||
final String outputPath = parser.get("actionsetsPath");
|
||||
log.info("outputPath: {}", outputPath);
|
||||
|
||||
SparkConf conf = new SparkConf();
|
||||
|
||||
runWithSparkSession(
|
||||
conf,
|
||||
isSparkSessionManaged,
|
||||
spark -> {
|
||||
JavaPairRDD<Text, Text> softwareRDD = prepareActionsets(spark, inputPath, softwareInputPath);
|
||||
softwareRDD
|
||||
.saveAsHadoopFile(
|
||||
outputPath, Text.class, Text.class, SequenceFileOutputFormat.class, GzipCodec.class);
|
||||
});
|
||||
}
|
||||
|
||||
private static Dataset<Row> loadSWHData(SparkSession spark, String inputPath) {
|
||||
|
||||
JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext());
|
||||
|
||||
// read from file and transform to <origin, snapshotId> tuples
|
||||
// Note: snapshot id is the SWH id for us
|
||||
JavaRDD<Row> swhRDD = sc
|
||||
.sequenceFile(inputPath, Text.class, Text.class)
|
||||
.map(t -> t._2().toString())
|
||||
.map(t -> OBJECT_MAPPER.readValue(t, LastVisitData.class))
|
||||
.filter(t -> t.getOrigin() != null && t.getSnapshot() != null) // response from SWH API is empty if repo URL
|
||||
// was not found
|
||||
.map(item -> RowFactory.create(item.getOrigin(), item.getSnapshot()));
|
||||
|
||||
// convert RDD to 2-column DF
|
||||
List<StructField> fields = Arrays
|
||||
.asList(
|
||||
DataTypes.createStructField("repoUrl", DataTypes.StringType, true),
|
||||
DataTypes.createStructField("swhId", DataTypes.StringType, true));
|
||||
StructType schema = DataTypes.createStructType(fields);
|
||||
|
||||
return spark.createDataFrame(swhRDD, schema);
|
||||
}
|
||||
|
||||
private static Dataset<Row> loadGraphSoftwareData(SparkSession spark, String softwareInputPath) {
|
||||
return spark
|
||||
.read()
|
||||
.textFile(softwareInputPath)
|
||||
.map(
|
||||
(MapFunction<String, Software>) t -> OBJECT_MAPPER.readValue(t, Software.class),
|
||||
Encoders.bean(Software.class))
|
||||
.filter(t -> t.getCodeRepositoryUrl() != null)
|
||||
.select(col("id"), col("codeRepositoryUrl.value").as("repoUrl"));
|
||||
}
|
||||
|
||||
private static <I extends Software> JavaPairRDD<Text, Text> prepareActionsets(SparkSession spark, String inputPath,
|
||||
String softwareInputPath) {
|
||||
|
||||
Dataset<Row> swhDF = loadSWHData(spark, inputPath);
|
||||
// swhDF.show(false);
|
||||
|
||||
Dataset<Row> graphSoftwareDF = loadGraphSoftwareData(spark, softwareInputPath);
|
||||
// graphSoftwareDF.show(5);
|
||||
|
||||
Dataset<Row> joinedDF = graphSoftwareDF.join(swhDF, "repoUrl").select("id", "swhid");
|
||||
// joinedDF.show(false);
|
||||
|
||||
return joinedDF.map((MapFunction<Row, Software>) row -> {
|
||||
|
||||
Software s = new Software();
|
||||
|
||||
// set openaire id
|
||||
s.setId(row.getString(row.fieldIndex("id")));
|
||||
|
||||
// set swh id
|
||||
Qualifier qualifier = OafMapperUtils
|
||||
.qualifier(
|
||||
SWHConstants.SWHID,
|
||||
SWHConstants.SWHID_CLASSNAME,
|
||||
ModelConstants.DNET_PID_TYPES,
|
||||
ModelConstants.DNET_PID_TYPES);
|
||||
|
||||
DataInfo dataInfo = OafMapperUtils
|
||||
.dataInfo(
|
||||
false,
|
||||
null,
|
||||
false,
|
||||
false,
|
||||
ModelConstants.PROVENANCE_ACTION_SET_QUALIFIER,
|
||||
"");
|
||||
|
||||
s
|
||||
.setPid(
|
||||
Arrays
|
||||
.asList(
|
||||
OafMapperUtils
|
||||
.structuredProperty(
|
||||
String.format("swh:1:snp:%s", row.getString(row.fieldIndex("swhid"))),
|
||||
qualifier,
|
||||
dataInfo)));
|
||||
|
||||
// add SWH in the `collectedFrom` field
|
||||
KeyValue kv = new KeyValue();
|
||||
kv.setKey(SWHConstants.SWH_ID);
|
||||
kv.setValue(SWHConstants.SWH_NAME);
|
||||
|
||||
s.setCollectedfrom(Arrays.asList(kv));
|
||||
|
||||
return s;
|
||||
}, Encoders.bean(Software.class))
|
||||
.toJavaRDD()
|
||||
.map(p -> new AtomicAction(Software.class, p))
|
||||
.mapToPair(
|
||||
aa -> new Tuple2<>(new Text(aa.getClazz().getCanonicalName()),
|
||||
new Text(OBJECT_MAPPER.writeValueAsString(aa))));
|
||||
}
|
||||
}
|
|
@ -0,0 +1,71 @@
|
|||
|
||||
package eu.dnetlib.dhp.swh.models;
|
||||
|
||||
import java.io.Serializable;
|
||||
|
||||
import com.cloudera.com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
|
||||
|
||||
@JsonIgnoreProperties(ignoreUnknown = true)
|
||||
public class LastVisitData implements Serializable {
|
||||
|
||||
private String origin;
|
||||
private String type;
|
||||
private String date;
|
||||
|
||||
@JsonProperty("snapshot")
|
||||
private String snapshotId;
|
||||
|
||||
private String status;
|
||||
|
||||
public String getType() {
|
||||
return type;
|
||||
}
|
||||
|
||||
public void setType(String type) {
|
||||
this.type = type;
|
||||
}
|
||||
|
||||
public String getDate() {
|
||||
return date;
|
||||
}
|
||||
|
||||
public void setDate(String date) {
|
||||
this.date = date;
|
||||
}
|
||||
|
||||
public String getSnapshot() {
|
||||
return snapshotId;
|
||||
}
|
||||
|
||||
public void setSnapshot(String snapshotId) {
|
||||
this.snapshotId = snapshotId;
|
||||
}
|
||||
|
||||
public String getStatus() {
|
||||
return status;
|
||||
}
|
||||
|
||||
public void setStatus(String status) {
|
||||
this.status = status;
|
||||
}
|
||||
|
||||
public String getOrigin() {
|
||||
return origin;
|
||||
}
|
||||
|
||||
public void setOrigin(String origin) {
|
||||
this.origin = origin;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "LastVisitData{" +
|
||||
"origin='" + origin + '\'' +
|
||||
", type='" + type + '\'' +
|
||||
", date='" + date + '\'' +
|
||||
", snapshotId='" + snapshotId + '\'' +
|
||||
", status='" + status + '\'' +
|
||||
'}';
|
||||
}
|
||||
}
|
|
@ -0,0 +1,40 @@
|
|||
|
||||
package eu.dnetlib.dhp.swh.utils;
|
||||
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.http.HttpHeaders;
|
||||
|
||||
import eu.dnetlib.dhp.common.collection.CollectorException;
|
||||
import eu.dnetlib.dhp.common.collection.HttpClientParams;
|
||||
import eu.dnetlib.dhp.common.collection.HttpConnector2;
|
||||
|
||||
public class SWHConnection {
|
||||
|
||||
HttpConnector2 conn;
|
||||
|
||||
public SWHConnection(HttpClientParams clientParams, String accessToken) {
|
||||
|
||||
// set custom headers
|
||||
Map<String, String> headers = new HashMap<String, String>() {
|
||||
{
|
||||
put(HttpHeaders.ACCEPT, "application/json");
|
||||
if (accessToken != null) {
|
||||
put(HttpHeaders.AUTHORIZATION, String.format("Bearer %s", accessToken));
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
clientParams.setHeaders(headers);
|
||||
|
||||
// create http connector
|
||||
conn = new HttpConnector2(clientParams);
|
||||
|
||||
}
|
||||
|
||||
public String call(String url) throws CollectorException {
|
||||
return conn.getInputSource(url);
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,21 @@
|
|||
|
||||
package eu.dnetlib.dhp.swh.utils;
|
||||
|
||||
public class SWHConstants {
|
||||
public static final String SWH_LATEST_VISIT_URL = "https://archive.softwareheritage.org/api/1/origin/%s/visit/latest/";
|
||||
|
||||
public static final String SWH_ARCHIVE_URL = "https://archive.softwareheritage.org/api/1/origin/save/%s/url/%s/";
|
||||
|
||||
public static final String DEFAULT_VISIT_TYPE = "git";
|
||||
|
||||
public static final String VISIT_STATUS_NOT_FOUND = "not_found";
|
||||
|
||||
public static final String SWHID = "swhid";
|
||||
|
||||
public static final String SWHID_CLASSNAME = "Software Heritage Identifier";
|
||||
|
||||
public static final String SWH_ID = "10|openaire____::dbfd07503aaa1ed31beed7dec942f3f4";
|
||||
|
||||
public static final String SWH_NAME = "Software Heritage";
|
||||
|
||||
}
|
|
@ -0,0 +1,95 @@
|
|||
|
||||
package eu.dnetlib.dhp.swh.utils;
|
||||
|
||||
import static eu.dnetlib.dhp.common.Constants.*;
|
||||
|
||||
import java.io.BufferedReader;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStreamReader;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.util.Optional;
|
||||
|
||||
import org.apache.hadoop.fs.FSDataInputStream;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.io.SequenceFile;
|
||||
import org.apache.hadoop.io.Text;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
|
||||
import eu.dnetlib.dhp.common.collection.HttpClientParams;
|
||||
|
||||
public class SWHUtils {
|
||||
|
||||
private static final Logger log = LoggerFactory.getLogger(SWHUtils.class);
|
||||
|
||||
public static HttpClientParams getClientParams(ArgumentApplicationParser argumentParser) {
|
||||
|
||||
final HttpClientParams clientParams = new HttpClientParams();
|
||||
clientParams
|
||||
.setMaxNumberOfRetry(
|
||||
Optional
|
||||
.ofNullable(argumentParser.get(MAX_NUMBER_OF_RETRY))
|
||||
.map(Integer::parseInt)
|
||||
.orElse(HttpClientParams._maxNumberOfRetry));
|
||||
log.info("maxNumberOfRetry is {}", clientParams.getMaxNumberOfRetry());
|
||||
|
||||
clientParams
|
||||
.setRequestDelay(
|
||||
Optional
|
||||
.ofNullable(argumentParser.get(REQUEST_DELAY))
|
||||
.map(Integer::parseInt)
|
||||
.orElse(HttpClientParams._requestDelay));
|
||||
log.info("requestDelay is {}", clientParams.getRequestDelay());
|
||||
|
||||
clientParams
|
||||
.setRetryDelay(
|
||||
Optional
|
||||
.ofNullable(argumentParser.get(RETRY_DELAY))
|
||||
.map(Integer::parseInt)
|
||||
.orElse(HttpClientParams._retryDelay));
|
||||
log.info("retryDelay is {}", clientParams.getRetryDelay());
|
||||
|
||||
clientParams
|
||||
.setRequestMethod(
|
||||
Optional
|
||||
.ofNullable(argumentParser.get(REQUEST_METHOD))
|
||||
.orElse(HttpClientParams._requestMethod));
|
||||
log.info("requestMethod is {}", clientParams.getRequestMethod());
|
||||
|
||||
return clientParams;
|
||||
}
|
||||
|
||||
public static BufferedReader getFileReader(FileSystem fs, Path inputPath) throws IOException {
|
||||
FSDataInputStream inputStream = fs.open(inputPath);
|
||||
return new BufferedReader(
|
||||
new InputStreamReader(inputStream, StandardCharsets.UTF_8));
|
||||
}
|
||||
|
||||
public static SequenceFile.Writer getSequenceFileWriter(FileSystem fs, String outputPath) throws IOException {
|
||||
return SequenceFile
|
||||
.createWriter(
|
||||
fs.getConf(),
|
||||
SequenceFile.Writer.file(new Path(outputPath)),
|
||||
SequenceFile.Writer.keyClass(Text.class),
|
||||
SequenceFile.Writer.valueClass(Text.class));
|
||||
}
|
||||
|
||||
public static SequenceFile.Reader getSequenceFileReader(FileSystem fs, String inputPath) throws IOException {
|
||||
Path filePath = new Path(inputPath);
|
||||
SequenceFile.Reader.Option fileOption = SequenceFile.Reader.file(filePath);
|
||||
|
||||
return new SequenceFile.Reader(fs.getConf(), fileOption);
|
||||
}
|
||||
|
||||
public static void appendToSequenceFile(SequenceFile.Writer fw, String keyStr, String valueStr) throws IOException {
|
||||
Text key = new Text();
|
||||
key.set(keyStr);
|
||||
|
||||
Text value = new Text();
|
||||
value.set(valueStr);
|
||||
|
||||
fw.append(key, value);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,56 @@
|
|||
[
|
||||
{
|
||||
"paramName": "n",
|
||||
"paramLongName": "namenode",
|
||||
"paramDescription": "the Name Node URI",
|
||||
"paramRequired": true
|
||||
},
|
||||
{
|
||||
"paramName": "lv",
|
||||
"paramLongName": "lastVisitsPath",
|
||||
"paramDescription": "the URL where to store last visits data",
|
||||
"paramRequired": true
|
||||
},
|
||||
{
|
||||
"paramName": "arp",
|
||||
"paramLongName": "archiveRequestsPath",
|
||||
"paramDescription": "the URL where to store the responses of the archive requests",
|
||||
"paramRequired": true
|
||||
},
|
||||
{
|
||||
"paramName": "mnr",
|
||||
"paramLongName": "maxNumberOfRetry",
|
||||
"paramDescription": "the maximum number of admitted connection retries",
|
||||
"paramRequired": false
|
||||
},
|
||||
{
|
||||
"paramName": "rqd",
|
||||
"paramLongName": "requestDelay",
|
||||
"paramDescription": "the delay (ms) between requests",
|
||||
"paramRequired": false
|
||||
},
|
||||
{
|
||||
"paramName": "rtd",
|
||||
"paramLongName": "retryDelay",
|
||||
"paramDescription": "the delay (ms) between retries",
|
||||
"paramRequired": false
|
||||
},
|
||||
{
|
||||
"paramName": "rm",
|
||||
"paramLongName": "requestMethod",
|
||||
"paramDescription": "the method of the requests to perform",
|
||||
"paramRequired": false
|
||||
},
|
||||
{
|
||||
"paramName": "atid",
|
||||
"paramLongName": "archiveThresholdInDays",
|
||||
"paramDescription": "the thershold (in days) required to issue an archive request",
|
||||
"paramRequired": false
|
||||
},
|
||||
{
|
||||
"paramName": "aat",
|
||||
"paramLongName": "apiAccessToken",
|
||||
"paramDescription": "the API access token of the SWH API",
|
||||
"paramRequired": false
|
||||
}
|
||||
]
|
|
@ -0,0 +1,50 @@
|
|||
[
|
||||
{
|
||||
"paramName": "n",
|
||||
"paramLongName": "namenode",
|
||||
"paramDescription": "the Name Node URI",
|
||||
"paramRequired": true
|
||||
},
|
||||
{
|
||||
"paramName": "scr",
|
||||
"paramLongName": "softwareCodeRepositoryURLs",
|
||||
"paramDescription": "the URL from where to read software repository URLs",
|
||||
"paramRequired": true
|
||||
},
|
||||
{
|
||||
"paramName": "lv",
|
||||
"paramLongName": "lastVisitsPath",
|
||||
"paramDescription": "the URL where to store last visits data",
|
||||
"paramRequired": true
|
||||
},
|
||||
{
|
||||
"paramName": "mnr",
|
||||
"paramLongName": "maxNumberOfRetry",
|
||||
"paramDescription": "the maximum number of admitted connection retries",
|
||||
"paramRequired": false
|
||||
},
|
||||
{
|
||||
"paramName": "rqd",
|
||||
"paramLongName": "requestDelay",
|
||||
"paramDescription": "the delay (ms) between requests",
|
||||
"paramRequired": false
|
||||
},
|
||||
{
|
||||
"paramName": "rtd",
|
||||
"paramLongName": "retryDelay",
|
||||
"paramDescription": "the delay (ms) between retries",
|
||||
"paramRequired": false
|
||||
},
|
||||
{
|
||||
"paramName": "rm",
|
||||
"paramLongName": "requestMethod",
|
||||
"paramDescription": "the method of the requests to perform",
|
||||
"paramRequired": false
|
||||
},
|
||||
{
|
||||
"paramName": "aat",
|
||||
"paramLongName": "apiAccessToken",
|
||||
"paramDescription": "the API access token of the SWH API",
|
||||
"paramRequired": false
|
||||
}
|
||||
]
|
|
@ -0,0 +1,32 @@
|
|||
[
|
||||
{
|
||||
"paramName": "issm",
|
||||
"paramLongName": "isSparkSessionManaged",
|
||||
"paramDescription": "when true will stop SparkSession after job execution",
|
||||
"paramRequired": false
|
||||
},
|
||||
{
|
||||
"paramName": "scr",
|
||||
"paramLongName": "softwareCodeRepositoryURLs",
|
||||
"paramDescription": "the URL where to store software repository URLs",
|
||||
"paramRequired": true
|
||||
},
|
||||
{
|
||||
"paramName": "db",
|
||||
"paramLongName": "hiveDbName",
|
||||
"paramDescription": "the target hive database name",
|
||||
"paramRequired": true
|
||||
},
|
||||
{
|
||||
"paramName": "hmu",
|
||||
"paramLongName": "hiveMetastoreUris",
|
||||
"paramDescription": "the hive metastore uris",
|
||||
"paramRequired": true
|
||||
},
|
||||
{
|
||||
"paramName": "slim",
|
||||
"paramLongName": "softwareLimit",
|
||||
"paramDescription": "limit on the number of software repo URL to fetch",
|
||||
"paramRequired": false
|
||||
}
|
||||
]
|
|
@ -0,0 +1,26 @@
|
|||
[
|
||||
{
|
||||
"paramName": "issm",
|
||||
"paramLongName": "isSparkSessionManaged",
|
||||
"paramDescription": "when true will stop SparkSession after job execution",
|
||||
"paramRequired": false
|
||||
},
|
||||
{
|
||||
"paramName": "lv",
|
||||
"paramLongName": "lastVisitsPath",
|
||||
"paramDescription": "the URL where to store last visits data",
|
||||
"paramRequired": true
|
||||
},
|
||||
{
|
||||
"paramName": "ap",
|
||||
"paramLongName": "actionsetsPath",
|
||||
"paramDescription": "the URL path where to store actionsets",
|
||||
"paramRequired": true
|
||||
},
|
||||
{
|
||||
"paramName": "sip",
|
||||
"paramLongName": "softwareInputPath",
|
||||
"paramDescription": "the URL path of the software in the graph",
|
||||
"paramRequired": true
|
||||
}
|
||||
]
|
|
@ -0,0 +1,19 @@
|
|||
# hive
|
||||
hiveDbName=openaire_prod_20230914
|
||||
|
||||
# input/output files
|
||||
softwareCodeRepositoryURLs=${workingDir}/1_code_repo_urls.csv
|
||||
lastVisitsPath=${workingDir}/2_last_visits.seq
|
||||
archiveRequestsPath=${workingDir}/3_archive_requests.seq
|
||||
actionsetsPath=${workingDir}/4_actionsets
|
||||
graphPath=/tmp/prod_provision/graph/18_graph_blacklisted
|
||||
|
||||
apiAccessToken=eyJhbGciOiJIUzI1NiIsInR5cCIgOiAiSldUIiwia2lkIiA6ICJhMTMxYTQ1My1hM2IyLTQwMTUtODQ2Ny05MzAyZjk3MTFkOGEifQ.eyJpYXQiOjE2OTQ2MzYwMjAsImp0aSI6IjkwZjdkNTNjLTQ5YTktNGFiMy1hY2E0LTcwMTViMjEyZTNjNiIsImlzcyI6Imh0dHBzOi8vYXV0aC5zb2Z0d2FyZWhlcml0YWdlLm9yZy9hdXRoL3JlYWxtcy9Tb2Z0d2FyZUhlcml0YWdlIiwiYXVkIjoiaHR0cHM6Ly9hdXRoLnNvZnR3YXJlaGVyaXRhZ2Uub3JnL2F1dGgvcmVhbG1zL1NvZnR3YXJlSGVyaXRhZ2UiLCJzdWIiOiIzMTY5OWZkNC0xNmE0LTQxOWItYTdhMi00NjI5MDY4ZjI3OWEiLCJ0eXAiOiJPZmZsaW5lIiwiYXpwIjoic3doLXdlYiIsInNlc3Npb25fc3RhdGUiOiIzMjYzMzEwMS00ZDRkLTQwMjItODU2NC1iMzNlMTJiNTE3ZDkiLCJzY29wZSI6Im9wZW5pZCBvZmZsaW5lX2FjY2VzcyBwcm9maWxlIGVtYWlsIn0.XHj1VIZu1dZ4Ej32-oU84mFmaox9cLNjXosNxwZM0Xs
|
||||
|
||||
maxNumberOfRetry=2
|
||||
retryDelay=1
|
||||
requestDelay=100
|
||||
|
||||
softwareLimit=500
|
||||
|
||||
resume=collect-software-repository-urls
|
|
@ -0,0 +1,54 @@
|
|||
<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>hiveMetastoreUris</name>
|
||||
<value>thrift://iis-cdh5-test-m3.ocean.icm.edu.pl:9083</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>spark2YarnHistoryServerAddress</name>
|
||||
<value>http://iis-cdh5-test-gw.ocean.icm.edu.pl:18089</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>spark2EventLogDir</name>
|
||||
<value>/user/spark/spark2ApplicationHistory</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>spark2ExtraListeners</name>
|
||||
<value>"com.cloudera.spark.lineage.NavigatorAppListener"</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>spark2SqlQueryExecutionListeners</name>
|
||||
<value>"com.cloudera.spark.lineage.NavigatorQueryListener"</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>oozieActionShareLibForSpark2</name>
|
||||
<value>spark2</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>resourceManager</name>
|
||||
<value>http://iis-cdh5-test-m2.ocean.icm.edu.pl:8088/cluster</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>oozie.launcher.mapreduce.user.classpath.first</name>
|
||||
<value>true</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>sparkSqlWarehouseDir</name>
|
||||
<value>/user/hive/warehouse</value>
|
||||
</property>
|
||||
</configuration>
|
|
@ -0,0 +1,183 @@
|
|||
<workflow-app name="Software-Heritage-Integration-Workflow" xmlns="uri:oozie:workflow:0.5">
|
||||
|
||||
<!-- Custom parameters -->
|
||||
<parameters>
|
||||
<property>
|
||||
<name>hiveDbName</name>
|
||||
<description>The name of the Hive DB to be used</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>softwareCodeRepositoryURLs</name>
|
||||
<description>The path in the HDFS to save the software repository URLs</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>lastVisitsPath</name>
|
||||
<description>The path in the HDFS to save the responses of the last visit requests</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>archiveRequestsPath</name>
|
||||
<description>The path in the HDFS to save the responses of the archive requests</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>actionsetsPath</name>
|
||||
<description>The path in the HDFS to save the action sets</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>graphPath</name>
|
||||
<description>The path in the HDFS to the base folder of the graph</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>maxNumberOfRetry</name>
|
||||
<description>Max number of retries for failed API calls</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>retryDelay</name>
|
||||
<description>Retry delay for failed requests (in sec)</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>requestDelay</name>
|
||||
<description>Delay between API requests (in ms)</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>apiAccessToken</name>
|
||||
<description>The API Key of the SWH API</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>softwareLimit</name>
|
||||
<description>Limit on the number of repo URLs to use (Optional); for debug purposes</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>resumeFrom</name>
|
||||
<description>Variable that indicates the step to start from</description>
|
||||
</property>
|
||||
</parameters>
|
||||
|
||||
<!-- Global variables -->
|
||||
<global>
|
||||
<job-tracker>${jobTracker}</job-tracker>
|
||||
<name-node>${nameNode}</name-node>
|
||||
<configuration>
|
||||
<property>
|
||||
<name>oozie.action.sharelib.for.spark</name>
|
||||
<value>${oozieActionShareLibForSpark2}</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>actionsetsPath</name>
|
||||
<value>${actionsetsPath}</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>apiAccessToken</name>
|
||||
<value>${apiAccessToken}</value>
|
||||
</property>
|
||||
</configuration>
|
||||
</global>
|
||||
|
||||
<start to="startFrom"/>
|
||||
|
||||
<kill name="Kill">
|
||||
<message>Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}]</message>
|
||||
</kill>
|
||||
|
||||
<decision name="startFrom">
|
||||
<switch>
|
||||
<case to="collect-software-repository-urls">${wf:conf('resumeFrom') eq 'collect-software-repository-urls'}</case>
|
||||
<case to="create-swh-actionsets">${wf:conf('resumeFrom') eq 'create-swh-actionsets'}</case>
|
||||
<default to="collect-software-repository-urls"/>
|
||||
</switch>
|
||||
</decision>
|
||||
|
||||
<action name="collect-software-repository-urls">
|
||||
<spark xmlns="uri:oozie:spark-action:0.2">
|
||||
<master>yarn</master>
|
||||
<mode>cluster</mode>
|
||||
<name>Collect software repository URLs</name>
|
||||
<class>eu.dnetlib.dhp.swh.CollectSoftwareRepositoryURLs</class>
|
||||
<jar>dhp-swh-${projectVersion}.jar</jar>
|
||||
<spark-opts>
|
||||
--executor-memory=${sparkExecutorMemory}
|
||||
--executor-cores=${sparkExecutorCores}
|
||||
--driver-memory=${sparkDriverMemory}
|
||||
--conf spark.extraListeners=${spark2ExtraListeners}
|
||||
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
|
||||
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
|
||||
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
|
||||
--conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir}
|
||||
</spark-opts>
|
||||
|
||||
<arg>--softwareCodeRepositoryURLs</arg><arg>${softwareCodeRepositoryURLs}</arg>
|
||||
<arg>--hiveDbName</arg><arg>${hiveDbName}</arg>
|
||||
<arg>--hiveMetastoreUris</arg><arg>${hiveMetastoreUris}</arg>
|
||||
<arg>--softwareLimit</arg><arg>${softwareLimit}</arg>
|
||||
</spark>
|
||||
<ok to="collect-repository-last-visit-data"/>
|
||||
<error to="Kill"/>
|
||||
</action>
|
||||
|
||||
<action name="collect-repository-last-visit-data">
|
||||
<java>
|
||||
<main-class>eu.dnetlib.dhp.swh.CollectLastVisitRepositoryData</main-class>
|
||||
|
||||
<arg>--namenode</arg><arg>${nameNode}</arg>
|
||||
<arg>--softwareCodeRepositoryURLs</arg><arg>${softwareCodeRepositoryURLs}</arg>
|
||||
<arg>--lastVisitsPath</arg><arg>${lastVisitsPath}</arg>
|
||||
|
||||
<arg>--maxNumberOfRetry</arg><arg>${maxNumberOfRetry}</arg>
|
||||
<arg>--requestDelay</arg><arg>${requestDelay}</arg>
|
||||
<arg>--retryDelay</arg><arg>${retryDelay}</arg>
|
||||
<arg>--requestMethod</arg><arg>GET</arg>
|
||||
<arg>--apiAccessToken</arg><arg>${apiAccessToken}</arg>
|
||||
|
||||
</java>
|
||||
<ok to="archive-repository-urls"/>
|
||||
<error to="Kill"/>
|
||||
</action>
|
||||
|
||||
<action name="archive-repository-urls">
|
||||
<java>
|
||||
<main-class>eu.dnetlib.dhp.swh.ArchiveRepositoryURLs</main-class>
|
||||
|
||||
<arg>--namenode</arg><arg>${nameNode}</arg>
|
||||
<arg>--lastVisitsPath</arg><arg>${lastVisitsPath}</arg>
|
||||
<arg>--archiveRequestsPath</arg><arg>${archiveRequestsPath}</arg>
|
||||
<arg>--archiveThresholdInDays</arg><arg>365</arg>
|
||||
|
||||
<arg>--maxNumberOfRetry</arg><arg>${maxNumberOfRetry}</arg>
|
||||
<arg>--requestDelay</arg><arg>${requestDelay}</arg>
|
||||
<arg>--retryDelay</arg><arg>${retryDelay}</arg>
|
||||
<arg>--requestMethod</arg><arg>POST</arg>
|
||||
<arg>--apiAccessToken</arg><arg>${apiAccessToken}</arg>
|
||||
|
||||
</java>
|
||||
<ok to="create-swh-actionsets"/>
|
||||
<error to="Kill"/>
|
||||
</action>
|
||||
|
||||
<action name="create-swh-actionsets">
|
||||
<spark xmlns="uri:oozie:spark-action:0.2">
|
||||
<master>yarn</master>
|
||||
<mode>cluster</mode>
|
||||
<name>Create actionsets for SWH data</name>
|
||||
<class>eu.dnetlib.dhp.swh.PrepareSWHActionsets</class>
|
||||
<jar>dhp-swh-${projectVersion}.jar</jar>
|
||||
<spark-opts>
|
||||
--executor-memory=${sparkExecutorMemory}
|
||||
--executor-cores=${sparkExecutorCores}
|
||||
--driver-memory=${sparkDriverMemory}
|
||||
--conf spark.extraListeners=${spark2ExtraListeners}
|
||||
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
|
||||
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
|
||||
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
|
||||
--conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir}
|
||||
</spark-opts>
|
||||
|
||||
<arg>--lastVisitsPath</arg><arg>${lastVisitsPath}</arg>
|
||||
<arg>--actionsetsPath</arg><arg>${actionsetsPath}</arg>
|
||||
<arg>--softwareInputPath</arg><arg>${graphPath}/software</arg>
|
||||
</spark>
|
||||
<ok to="End"/>
|
||||
<error to="Kill"/>
|
||||
</action>
|
||||
|
||||
<end name="End"/>
|
||||
|
||||
</workflow-app>
|
|
@ -0,0 +1,38 @@
|
|||
|
||||
package eu.dnetlib.dhp.swh;
|
||||
|
||||
import java.io.BufferedReader;
|
||||
import java.io.File;
|
||||
import java.io.FileReader;
|
||||
import java.io.IOException;
|
||||
import java.text.ParseException;
|
||||
import java.util.Arrays;
|
||||
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import eu.dnetlib.dhp.swh.utils.SWHUtils;
|
||||
|
||||
public class ArchiveRepositoryURLsTest {
|
||||
|
||||
@Test
|
||||
void testArchive() throws IOException, ParseException {
|
||||
String inputPath = getClass()
|
||||
.getResource("/eu/dnetlib/dhp/swh/lastVisitDataToArchive.csv")
|
||||
.getPath();
|
||||
|
||||
File file = new File(inputPath);
|
||||
FileReader fr = new FileReader(file);
|
||||
BufferedReader br = new BufferedReader(fr); // creates a buffering character input stream
|
||||
|
||||
String line;
|
||||
while ((line = br.readLine()) != null) {
|
||||
String[] tokens = line.split("\t");
|
||||
|
||||
String response = ArchiveRepositoryURLs.handleRecord(tokens[0], tokens[1], 365);
|
||||
System.out.println(tokens[0] + "\t" + response);
|
||||
System.out.println();
|
||||
}
|
||||
fr.close();
|
||||
}
|
||||
}
|
|
@ -0,0 +1,97 @@
|
|||
|
||||
package eu.dnetlib.dhp.swh;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.nio.file.Files;
|
||||
import java.nio.file.Path;
|
||||
|
||||
import org.apache.commons.io.FileUtils;
|
||||
import org.apache.hadoop.io.Text;
|
||||
import org.apache.spark.SparkConf;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.apache.spark.sql.Dataset;
|
||||
import org.apache.spark.sql.Encoders;
|
||||
import org.apache.spark.sql.Row;
|
||||
import org.apache.spark.sql.SparkSession;
|
||||
import org.junit.jupiter.api.AfterAll;
|
||||
import org.junit.jupiter.api.Assertions;
|
||||
import org.junit.jupiter.api.BeforeAll;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
|
||||
import eu.dnetlib.dhp.schema.action.AtomicAction;
|
||||
import eu.dnetlib.dhp.schema.common.ModelConstants;
|
||||
import eu.dnetlib.dhp.schema.oaf.Relation;
|
||||
import eu.dnetlib.dhp.schema.oaf.utils.CleaningFunctions;
|
||||
import eu.dnetlib.dhp.schema.oaf.utils.IdentifierFactory;
|
||||
|
||||
public class PrepareSWHActionsetsTest {
|
||||
|
||||
private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
|
||||
|
||||
private static SparkSession spark;
|
||||
|
||||
private static Path workingDir;
|
||||
|
||||
private static final Logger log = LoggerFactory
|
||||
.getLogger(PrepareSWHActionsetsTest.class);
|
||||
|
||||
@BeforeAll
|
||||
public static void beforeAll() throws IOException {
|
||||
workingDir = Files.createTempDirectory(PrepareSWHActionsetsTest.class.getSimpleName());
|
||||
|
||||
log.info("Using work dir {}", workingDir);
|
||||
|
||||
SparkConf conf = new SparkConf();
|
||||
conf.setAppName(PrepareSWHActionsetsTest.class.getSimpleName());
|
||||
|
||||
conf.setMaster("local[*]");
|
||||
conf.set("spark.driver.host", "localhost");
|
||||
conf.set("hive.metastore.local", "true");
|
||||
conf.set("spark.ui.enabled", "false");
|
||||
conf.set("spark.sql.warehouse.dir", workingDir.toString());
|
||||
conf.set("hive.metastore.warehouse.dir", workingDir.resolve("warehouse").toString());
|
||||
|
||||
spark = SparkSession
|
||||
.builder()
|
||||
.appName(PrepareSWHActionsetsTest.class.getSimpleName())
|
||||
.config(conf)
|
||||
.getOrCreate();
|
||||
}
|
||||
|
||||
@AfterAll
|
||||
public static void afterAll() throws IOException {
|
||||
FileUtils.deleteDirectory(workingDir.toFile());
|
||||
spark.stop();
|
||||
}
|
||||
|
||||
@Test
|
||||
void testRun() throws Exception {
|
||||
|
||||
String lastVisitsPath = getClass()
|
||||
.getResource("/eu/dnetlib/dhp/swh/last_visits_data.seq")
|
||||
.getPath();
|
||||
|
||||
String outputPath = workingDir.toString() + "/actionSet";
|
||||
|
||||
String softwareInputPath = getClass()
|
||||
.getResource("/eu/dnetlib/dhp/swh/software.json.gz")
|
||||
.getPath();
|
||||
|
||||
PrepareSWHActionsets
|
||||
.main(
|
||||
new String[] {
|
||||
"-isSparkSessionManaged", Boolean.FALSE.toString(),
|
||||
"-lastVisitsPath", lastVisitsPath,
|
||||
"-softwareInputPath", softwareInputPath,
|
||||
"-actionsetsPath", outputPath
|
||||
});
|
||||
|
||||
}
|
||||
}
|
|
@ -0,0 +1,58 @@
|
|||
|
||||
package eu.dnetlib.dhp.swh;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.net.MalformedURLException;
|
||||
import java.net.URL;
|
||||
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import eu.dnetlib.dhp.common.collection.CollectorException;
|
||||
import eu.dnetlib.dhp.common.collection.HttpClientParams;
|
||||
import eu.dnetlib.dhp.swh.utils.SWHConnection;
|
||||
import eu.dnetlib.dhp.swh.utils.SWHConstants;
|
||||
|
||||
//import org.apache.hadoop.hdfs.MiniDFSCluster;
|
||||
|
||||
public class SWHConnectionTest {
|
||||
private static final Logger log = LoggerFactory.getLogger(SWHConnectionTest.class);
|
||||
|
||||
@Test
|
||||
void testGetCall() throws IOException {
|
||||
|
||||
HttpClientParams clientParams = new HttpClientParams();
|
||||
clientParams.setRequestMethod("GET");
|
||||
|
||||
SWHConnection swhConnection = new SWHConnection(clientParams, null);
|
||||
|
||||
String repoUrl = "https://github.com/stanford-futuredata/FAST";
|
||||
URL url = new URL(String.format(SWHConstants.SWH_LATEST_VISIT_URL, repoUrl));
|
||||
String response = null;
|
||||
try {
|
||||
response = swhConnection.call(url.toString());
|
||||
} catch (CollectorException e) {
|
||||
System.out.println("Error in request: " + url);
|
||||
}
|
||||
System.out.println(response);
|
||||
}
|
||||
|
||||
@Test
|
||||
void testPostCall() throws MalformedURLException {
|
||||
HttpClientParams clientParams = new HttpClientParams();
|
||||
clientParams.setRequestMethod("POST");
|
||||
|
||||
SWHConnection swhConnection = new SWHConnection(clientParams, null);
|
||||
|
||||
String repoUrl = "https://github.com/stanford-futuredata/FAST";
|
||||
URL url = new URL(String.format(SWHConstants.SWH_ARCHIVE_URL, SWHConstants.DEFAULT_VISIT_TYPE, repoUrl));
|
||||
String response = null;
|
||||
try {
|
||||
response = swhConnection.call(url.toString());
|
||||
} catch (CollectorException e) {
|
||||
System.out.println("Error in request: " + url);
|
||||
}
|
||||
System.out.println(response);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,7 @@
|
|||
https://bitbucket.org/samskillman/yt-stokes {"origin":"https://bitbucket.org/samskillman/yt-stokes","visit":43,"date":"2021-09-13T21:59:27.125171+00:00","status":"failed","snapshot":null,"type":"hg","metadata":{},"origin_url":"https://archive.softwareheritage.org/api/1/origin/https://bitbucket.org/samskillman/yt-stokes/get/","snapshot_url":null}
|
||||
https://github.com/bioinsilico/BIPSPI {"origin":"https://github.com/bioinsilico/BIPSPI","visit":1,"date":"2020-03-18T14:50:21.541822+00:00","status":"full","snapshot":"c6c69d2cd73ce89811448da5f031611df6f63bdb","type":"git","metadata":{},"origin_url":"https://archive.softwareheritage.org/api/1/origin/https://github.com/bioinsilico/BIPSPI/get/","snapshot_url":"https://archive.softwareheritage.org/api/1/snapshot/c6c69d2cd73ce89811448da5f031611df6f63bdb/"}
|
||||
https://github.com/mloop/kdiff-type1-error-rate/blob/master/analysis/simulation.R {}
|
||||
https://github.com/schwanbeck/YSMR {"origin":"https://github.com/schwanbeck/YSMR","visit":6,"date":"2023-08-02T15:25:02.650676+00:00","status":"full","snapshot":"a9d1c5f0bca2def198b89f65bc9f7da3be8439ed","type":"git","metadata":{},"origin_url":"https://archive.softwareheritage.org/api/1/origin/https://github.com/schwanbeck/YSMR/get/","snapshot_url":"https://archive.softwareheritage.org/api/1/snapshot/a9d1c5f0bca2def198b89f65bc9f7da3be8439ed/"}
|
||||
https://github.com/lvclark/TASSELGBS_combine {"origin":"https://github.com/lvclark/TASSELGBS_combine","visit":1,"date":"2020-04-12T20:44:09.405589+00:00","status":"full","snapshot":"ffa6fefd3f5becefbea9fe0e6d5d93859c95c071","type":"git","metadata":{},"origin_url":"https://archive.softwareheritage.org/api/1/origin/https://github.com/lvclark/TASSELGBS_combine/get/","snapshot_url":"https://archive.softwareheritage.org/api/1/snapshot/ffa6fefd3f5becefbea9fe0e6d5d93859c95c071/"}
|
||||
https://github.com/PRIDE-Toolsuite/inspector-example-files {"origin":"https://github.com/PRIDE-Toolsuite/inspector-example-files","visit":12,"date":"2021-01-25T08:54:13.394674+00:00","status":"full","snapshot":"0b56eb0ad07cf778df6dabefc4b73636e0ae8b37","type":"git","metadata":{},"origin_url":"https://archive.softwareheritage.org/api/1/origin/https://github.com/PRIDE-Toolsuite/inspector-example-files/get/","snapshot_url":"https://archive.softwareheritage.org/api/1/snapshot/0b56eb0ad07cf778df6dabefc4b73636e0ae8b37/"}
|
||||
https://bitbucket.org/matwey/chelyabinsk {"origin":"https://bitbucket.org/matwey/chelyabinsk","visit":6,"date":"2021-09-24T19:32:43.322909+00:00","status":"full","snapshot":"215913858c3ee0e61e1aaea18241c5ee006da1b0","type":"hg","metadata":{},"origin_url":"https://archive.softwareheritage.org/api/1/origin/https://bitbucket.org/matwey/chelyabinsk/get/","snapshot_url":"https://archive.softwareheritage.org/api/1/snapshot/215913858c3ee0e61e1aaea18241c5ee006da1b0/"}
|
Can't render this file because it contains an unexpected character in line 1 and column 46.
|
Binary file not shown.
Binary file not shown.
|
@ -39,6 +39,7 @@
|
|||
<module>dhp-broker-events</module>
|
||||
<module>dhp-doiboost</module>
|
||||
<module>dhp-impact-indicators</module>
|
||||
<module>dhp-swh</module>
|
||||
</modules>
|
||||
|
||||
<pluginRepositories>
|
||||
|
|
10
pom.xml
10
pom.xml
|
@ -112,6 +112,16 @@
|
|||
<url>https://maven.d4science.org/nexus/content/repositories/dnet-deps</url>
|
||||
<layout>default</layout>
|
||||
</repository>
|
||||
<repository>
|
||||
<id>maven-restlet</id>
|
||||
<name>Restlet repository</name>
|
||||
<url>https://maven.restlet.talend.com</url>
|
||||
</repository>
|
||||
<repository>
|
||||
<id>conjars</id>
|
||||
<name>conjars</name>
|
||||
<url>https://conjars.wensel.net/repo/</url>
|
||||
</repository>
|
||||
</repositories>
|
||||
|
||||
<dependencies>
|
||||
|
|
Loading…
Reference in New Issue