forked from D-Net/dnet-hadoop
[graph cleaning] WIP: refactoring of the cleaning stages, unit tests
This commit is contained in:
parent
6d3d18d8b5
commit
4f5ba0ed52
|
@ -3,15 +3,18 @@ package eu.dnetlib.dhp.oa.graph.clean;
|
||||||
|
|
||||||
import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession;
|
import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession;
|
||||||
|
|
||||||
import java.util.List;
|
import java.util.*;
|
||||||
import java.util.Optional;
|
|
||||||
import java.util.Set;
|
|
||||||
import java.util.stream.Collectors;
|
import java.util.stream.Collectors;
|
||||||
|
import java.util.stream.Stream;
|
||||||
|
|
||||||
|
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException;
|
||||||
import org.apache.commons.io.IOUtils;
|
import org.apache.commons.io.IOUtils;
|
||||||
|
import org.apache.commons.lang3.StringUtils;
|
||||||
import org.apache.spark.SparkConf;
|
import org.apache.spark.SparkConf;
|
||||||
import org.apache.spark.api.java.function.FilterFunction;
|
import org.apache.spark.api.java.function.FilterFunction;
|
||||||
|
import org.apache.spark.api.java.function.FlatMapFunction;
|
||||||
import org.apache.spark.api.java.function.MapFunction;
|
import org.apache.spark.api.java.function.MapFunction;
|
||||||
|
import org.apache.spark.api.java.function.MapGroupsFunction;
|
||||||
import org.apache.spark.sql.Dataset;
|
import org.apache.spark.sql.Dataset;
|
||||||
import org.apache.spark.sql.Encoders;
|
import org.apache.spark.sql.Encoders;
|
||||||
import org.apache.spark.sql.SaveMode;
|
import org.apache.spark.sql.SaveMode;
|
||||||
|
@ -24,15 +27,19 @@ import com.google.common.collect.Sets;
|
||||||
|
|
||||||
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
|
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
|
||||||
import eu.dnetlib.dhp.common.HdfsSupport;
|
import eu.dnetlib.dhp.common.HdfsSupport;
|
||||||
|
import eu.dnetlib.dhp.common.action.model.MasterDuplicate;
|
||||||
import eu.dnetlib.dhp.common.vocabulary.VocabularyGroup;
|
import eu.dnetlib.dhp.common.vocabulary.VocabularyGroup;
|
||||||
|
import eu.dnetlib.dhp.oa.graph.clean.cfhb.IdCfHbMapping;
|
||||||
import eu.dnetlib.dhp.schema.common.ModelConstants;
|
import eu.dnetlib.dhp.schema.common.ModelConstants;
|
||||||
import eu.dnetlib.dhp.schema.common.ModelSupport;
|
import eu.dnetlib.dhp.schema.common.ModelSupport;
|
||||||
|
import eu.dnetlib.dhp.schema.oaf.KeyValue;
|
||||||
import eu.dnetlib.dhp.schema.oaf.Oaf;
|
import eu.dnetlib.dhp.schema.oaf.Oaf;
|
||||||
import eu.dnetlib.dhp.schema.oaf.OafEntity;
|
import eu.dnetlib.dhp.schema.oaf.OafEntity;
|
||||||
import eu.dnetlib.dhp.schema.oaf.Result;
|
import eu.dnetlib.dhp.schema.oaf.Result;
|
||||||
import eu.dnetlib.dhp.schema.oaf.utils.GraphCleaningFunctions;
|
import eu.dnetlib.dhp.schema.oaf.utils.GraphCleaningFunctions;
|
||||||
import eu.dnetlib.dhp.utils.ISLookupClientFactory;
|
import eu.dnetlib.dhp.utils.ISLookupClientFactory;
|
||||||
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService;
|
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService;
|
||||||
|
import scala.Tuple2;
|
||||||
|
|
||||||
public class CleanGraphSparkJob {
|
public class CleanGraphSparkJob {
|
||||||
|
|
||||||
|
@ -40,31 +47,43 @@ public class CleanGraphSparkJob {
|
||||||
|
|
||||||
private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
|
private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
|
||||||
|
|
||||||
public static void main(String[] args) throws Exception {
|
private ArgumentApplicationParser parser;
|
||||||
|
|
||||||
|
public CleanGraphSparkJob(ArgumentApplicationParser parser) {
|
||||||
|
this.parser = parser;
|
||||||
|
}
|
||||||
|
|
||||||
|
public static void main(String[] args) throws Exception {
|
||||||
String jsonConfiguration = IOUtils
|
String jsonConfiguration = IOUtils
|
||||||
.toString(
|
.toString(
|
||||||
CleanGraphSparkJob.class
|
CleanGraphSparkJob.class
|
||||||
.getResourceAsStream(
|
.getResourceAsStream(
|
||||||
"/eu/dnetlib/dhp/oa/graph/input_clean_graph_parameters.json"));
|
"/eu/dnetlib/dhp/oa/graph/input_clean_graph_parameters.json"));
|
||||||
final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration);
|
final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration);
|
||||||
parser.parseArgument(args);
|
parser.parseArgument(args);
|
||||||
|
|
||||||
Boolean isSparkSessionManaged = Optional
|
Boolean isSparkSessionManaged = Optional
|
||||||
.ofNullable(parser.get("isSparkSessionManaged"))
|
.ofNullable(parser.get("isSparkSessionManaged"))
|
||||||
.map(Boolean::valueOf)
|
.map(Boolean::valueOf)
|
||||||
.orElse(Boolean.TRUE);
|
.orElse(Boolean.TRUE);
|
||||||
log.info("isSparkSessionManaged: {}", isSparkSessionManaged);
|
log.info("isSparkSessionManaged: {}", isSparkSessionManaged);
|
||||||
|
|
||||||
|
String isLookupUrl = parser.get("isLookupUrl");
|
||||||
|
log.info("isLookupUrl: {}", isLookupUrl);
|
||||||
|
|
||||||
|
ISLookUpService isLookup = ISLookupClientFactory.getLookUpService(isLookupUrl);
|
||||||
|
|
||||||
|
new CleanGraphSparkJob(parser).run(isSparkSessionManaged, isLookup);
|
||||||
|
}
|
||||||
|
|
||||||
|
public void run(Boolean isSparkSessionManaged, ISLookUpService isLookUpService) throws ISLookUpException, ClassNotFoundException {
|
||||||
|
|
||||||
String inputPath = parser.get("inputPath");
|
String inputPath = parser.get("inputPath");
|
||||||
log.info("inputPath: {}", inputPath);
|
log.info("inputPath: {}", inputPath);
|
||||||
|
|
||||||
String outputPath = parser.get("outputPath");
|
String outputPath = parser.get("outputPath");
|
||||||
log.info("outputPath: {}", outputPath);
|
log.info("outputPath: {}", outputPath);
|
||||||
|
|
||||||
String isLookupUrl = parser.get("isLookupUrl");
|
|
||||||
log.info("isLookupUrl: {}", isLookupUrl);
|
|
||||||
|
|
||||||
String graphTableClassName = parser.get("graphTableClassName");
|
String graphTableClassName = parser.get("graphTableClassName");
|
||||||
log.info("graphTableClassName: {}", graphTableClassName);
|
log.info("graphTableClassName: {}", graphTableClassName);
|
||||||
|
|
||||||
|
@ -80,27 +99,38 @@ public class CleanGraphSparkJob {
|
||||||
String country = parser.get("country");
|
String country = parser.get("country");
|
||||||
log.info("country: {}", country);
|
log.info("country: {}", country);
|
||||||
|
|
||||||
String[] verifyCountryParam = parser.get("verifyCountryParam").split(";");
|
String[] verifyCountryParam = Optional.ofNullable(parser.get("verifyCountryParam"))
|
||||||
|
.map(s -> s.split(";"))
|
||||||
|
.orElse(new String[]{});
|
||||||
log.info("verifyCountryParam: {}", verifyCountryParam);
|
log.info("verifyCountryParam: {}", verifyCountryParam);
|
||||||
|
|
||||||
String collectedfrom = parser.get("collectedfrom");
|
String collectedfrom = parser.get("collectedfrom");
|
||||||
log.info("collectedfrom: {}", collectedfrom);
|
log.info("collectedfrom: {}", collectedfrom);
|
||||||
|
|
||||||
|
String dsMasterDuplicatePath = parser.get("masterDuplicatePath");
|
||||||
|
log.info("masterDuplicatePath: {}", dsMasterDuplicatePath);
|
||||||
|
|
||||||
|
Boolean deepClean = Optional
|
||||||
|
.ofNullable(parser.get("deepClean"))
|
||||||
|
.map(Boolean::valueOf)
|
||||||
|
.orElse(Boolean.FALSE);
|
||||||
|
log.info("deepClean: {}", deepClean);
|
||||||
|
|
||||||
Class<? extends OafEntity> entityClazz = (Class<? extends OafEntity>) Class.forName(graphTableClassName);
|
Class<? extends OafEntity> entityClazz = (Class<? extends OafEntity>) Class.forName(graphTableClassName);
|
||||||
|
|
||||||
final ISLookUpService isLookupService = ISLookupClientFactory.getLookUpService(isLookupUrl);
|
final VocabularyGroup vocs = VocabularyGroup.loadVocsFromIS(isLookUpService);
|
||||||
final VocabularyGroup vocs = VocabularyGroup.loadVocsFromIS(isLookupService);
|
|
||||||
|
|
||||||
SparkConf conf = new SparkConf();
|
SparkConf conf = new SparkConf();
|
||||||
|
conf.setAppName(CleanGraphSparkJob.class.getSimpleName() + "#" + entityClazz.getSimpleName());
|
||||||
runWithSparkSession(
|
runWithSparkSession(
|
||||||
conf,
|
conf,
|
||||||
isSparkSessionManaged,
|
isSparkSessionManaged,
|
||||||
spark -> {
|
spark -> {
|
||||||
HdfsSupport.remove(outputPath, spark.sparkContext().hadoopConfiguration());
|
HdfsSupport.remove(outputPath, spark.sparkContext().hadoopConfiguration());
|
||||||
cleanGraphTable(
|
cleanGraphTable(
|
||||||
spark, vocs, inputPath, entityClazz, outputPath, contextId, verifyParam, datasourcePath, country,
|
spark, vocs, inputPath, entityClazz, outputPath, contextId, verifyParam, datasourcePath, country,
|
||||||
verifyCountryParam, collectedfrom);
|
verifyCountryParam, collectedfrom, dsMasterDuplicatePath, deepClean);
|
||||||
});
|
});
|
||||||
}
|
}
|
||||||
|
|
||||||
private static <T extends Oaf> void cleanGraphTable(
|
private static <T extends Oaf> void cleanGraphTable(
|
||||||
|
@ -109,33 +139,74 @@ public class CleanGraphSparkJob {
|
||||||
String inputPath,
|
String inputPath,
|
||||||
Class<T> clazz,
|
Class<T> clazz,
|
||||||
String outputPath, String contextId, String verifyParam, String datasourcePath, String country,
|
String outputPath, String contextId, String verifyParam, String datasourcePath, String country,
|
||||||
String[] verifyCountryParam, String collectedfrom) {
|
String[] verifyCountryParam, String collectedfrom, String dsMasterDuplicatePath,
|
||||||
|
Boolean deepClean) {
|
||||||
Set<String> hostedBy = Sets
|
|
||||||
.newHashSet(
|
|
||||||
spark
|
|
||||||
.read()
|
|
||||||
.textFile(datasourcePath)
|
|
||||||
.collectAsList());
|
|
||||||
|
|
||||||
final CleaningRuleMap mapping = CleaningRuleMap.create(vocs);
|
final CleaningRuleMap mapping = CleaningRuleMap.create(vocs);
|
||||||
|
|
||||||
readTableFromPath(spark, inputPath, clazz)
|
final Dataset<T> cleaned_basic = readTableFromPath(spark, inputPath, clazz)
|
||||||
.map((MapFunction<T, T>) GraphCleaningFunctions::fixVocabularyNames, Encoders.bean(clazz))
|
.map((MapFunction<T, T>) GraphCleaningFunctions::fixVocabularyNames, Encoders.bean(clazz))
|
||||||
.map((MapFunction<T, T>) value -> OafCleaner.apply(value, mapping), Encoders.bean(clazz))
|
.map((MapFunction<T, T>) value -> OafCleaner.apply(value, mapping), Encoders.bean(clazz))
|
||||||
.map((MapFunction<T, T>) value -> GraphCleaningFunctions.cleanup(value, vocs), Encoders.bean(clazz))
|
.map((MapFunction<T, T>) value -> GraphCleaningFunctions.cleanup(value, vocs), Encoders.bean(clazz))
|
||||||
.filter((FilterFunction<T>) GraphCleaningFunctions::filter)
|
.filter((FilterFunction<T>) GraphCleaningFunctions::filter);
|
||||||
.map(
|
|
||||||
(MapFunction<T, T>) value -> GraphCleaningFunctions.cleanContext(value, contextId, verifyParam),
|
if (Boolean.FALSE.equals(deepClean)) {
|
||||||
Encoders.bean(clazz))
|
cleaned_basic
|
||||||
.map(
|
.write()
|
||||||
(MapFunction<T, T>) value -> GraphCleaningFunctions
|
.mode(SaveMode.Overwrite)
|
||||||
.cleanCountry(value, verifyCountryParam, hostedBy, collectedfrom, country),
|
.option("compression", "gzip")
|
||||||
Encoders.bean(clazz))
|
.json(outputPath);
|
||||||
.write()
|
|
||||||
.mode(SaveMode.Overwrite)
|
} else if (Boolean.TRUE.equals(ModelSupport.isSubClass(clazz, Result.class))) {
|
||||||
.option("compression", "gzip")
|
|
||||||
.json(outputPath);
|
// read the master-duplicate tuples
|
||||||
|
Dataset<MasterDuplicate> md = spark
|
||||||
|
.read()
|
||||||
|
.textFile(dsMasterDuplicatePath)
|
||||||
|
.map(as(MasterDuplicate.class), Encoders.bean(MasterDuplicate.class));
|
||||||
|
|
||||||
|
// prepare the resolved CF|HB references with the corresponding EMPTY master ID
|
||||||
|
Dataset<IdCfHbMapping> resolved = spark
|
||||||
|
.read()
|
||||||
|
.textFile(inputPath)
|
||||||
|
.map(as(clazz), Encoders.bean(clazz))
|
||||||
|
.flatMap(flattenCfHbFn(), Encoders.bean(IdCfHbMapping.class));
|
||||||
|
|
||||||
|
// set the EMPTY master ID/NAME and save it
|
||||||
|
resolved
|
||||||
|
.joinWith(md, resolved.col("cfhb").equalTo(md.col("duplicateId")))
|
||||||
|
.map(asIdCfHbMapping(), Encoders.bean(IdCfHbMapping.class))
|
||||||
|
.filter((FilterFunction<IdCfHbMapping>) m -> Objects.nonNull(m.getMasterId()));
|
||||||
|
|
||||||
|
// load the hostedby mapping
|
||||||
|
Set<String> hostedBy = Sets
|
||||||
|
.newHashSet(
|
||||||
|
spark
|
||||||
|
.read()
|
||||||
|
.textFile(datasourcePath)
|
||||||
|
.collectAsList());
|
||||||
|
|
||||||
|
// perform the deep cleaning steps
|
||||||
|
final Dataset<T> cleaned_deep = cleaned_basic
|
||||||
|
.map(
|
||||||
|
(MapFunction<T, T>) value -> GraphCleaningFunctions.cleanContext(value, contextId, verifyParam),
|
||||||
|
Encoders.bean(clazz))
|
||||||
|
.map(
|
||||||
|
(MapFunction<T, T>) value -> GraphCleaningFunctions
|
||||||
|
.cleanCountry(value, verifyCountryParam, hostedBy, collectedfrom, country),
|
||||||
|
Encoders.bean(clazz));
|
||||||
|
|
||||||
|
// Join the results with the resolved CF|HB mapping, apply the mapping and save it
|
||||||
|
cleaned_deep
|
||||||
|
.joinWith(resolved, cleaned_deep.col("id").equalTo(resolved.col("resultId")), "left")
|
||||||
|
.groupByKey(
|
||||||
|
(MapFunction<Tuple2<T, IdCfHbMapping>, String>) t -> ((Result) t._1()).getId(), Encoders.STRING())
|
||||||
|
.mapGroups(getMapGroupsFunction(), Encoders.bean(clazz))
|
||||||
|
.write()
|
||||||
|
.mode(SaveMode.Overwrite)
|
||||||
|
.option("compression", "gzip")
|
||||||
|
.json(outputPath);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private static <T extends Oaf> Dataset<T> readTableFromPath(
|
private static <T extends Oaf> Dataset<T> readTableFromPath(
|
||||||
|
@ -145,9 +216,98 @@ public class CleanGraphSparkJob {
|
||||||
return spark
|
return spark
|
||||||
.read()
|
.read()
|
||||||
.textFile(inputEntityPath)
|
.textFile(inputEntityPath)
|
||||||
.map(
|
.map(as(clazz), Encoders.bean(clazz));
|
||||||
(MapFunction<String, T>) value -> OBJECT_MAPPER.readValue(value, clazz),
|
}
|
||||||
Encoders.bean(clazz));
|
|
||||||
|
private static <R> MapFunction<String, R> as(Class<R> clazz) {
|
||||||
|
return s -> OBJECT_MAPPER.readValue(s, clazz);
|
||||||
|
}
|
||||||
|
|
||||||
|
private static <T extends Oaf> FlatMapFunction<T, IdCfHbMapping> flattenCfHbFn() {
|
||||||
|
return r -> Stream
|
||||||
|
.concat(
|
||||||
|
Optional
|
||||||
|
.ofNullable(r.getCollectedfrom())
|
||||||
|
.map(cf -> cf.stream().map(KeyValue::getKey))
|
||||||
|
.orElse(Stream.empty()),
|
||||||
|
Stream
|
||||||
|
.concat(
|
||||||
|
Optional
|
||||||
|
.ofNullable(((Result) r).getInstance())
|
||||||
|
.map(
|
||||||
|
instances -> instances
|
||||||
|
.stream()
|
||||||
|
.map(i -> Optional.ofNullable(i.getHostedby()).map(KeyValue::getKey).orElse("")))
|
||||||
|
.orElse(Stream.empty())
|
||||||
|
.filter(StringUtils::isNotBlank),
|
||||||
|
Optional
|
||||||
|
.ofNullable(((Result) r).getInstance())
|
||||||
|
.map(
|
||||||
|
instances -> instances
|
||||||
|
.stream()
|
||||||
|
.map(
|
||||||
|
i -> Optional
|
||||||
|
.ofNullable(i.getCollectedfrom())
|
||||||
|
.map(KeyValue::getKey)
|
||||||
|
.orElse("")))
|
||||||
|
.orElse(Stream.empty())
|
||||||
|
.filter(StringUtils::isNotBlank)))
|
||||||
|
.distinct()
|
||||||
|
.filter(StringUtils::isNotBlank)
|
||||||
|
.map(cfHb -> asIdCfHbMapping(((Result) r).getId(), cfHb))
|
||||||
|
.iterator();
|
||||||
|
}
|
||||||
|
|
||||||
|
private static MapFunction<Tuple2<IdCfHbMapping, MasterDuplicate>, IdCfHbMapping> asIdCfHbMapping() {
|
||||||
|
return t -> {
|
||||||
|
final IdCfHbMapping mapping = t._1();
|
||||||
|
Optional
|
||||||
|
.ofNullable(t._2())
|
||||||
|
.ifPresent(t2 -> {
|
||||||
|
mapping.setMasterId(t2.getMasterId());
|
||||||
|
mapping.setMasterName(t2.getMasterName());
|
||||||
|
|
||||||
|
});
|
||||||
|
return mapping;
|
||||||
|
};
|
||||||
|
}
|
||||||
|
|
||||||
|
private static IdCfHbMapping asIdCfHbMapping(String resultId, String cfHb) {
|
||||||
|
IdCfHbMapping m = new IdCfHbMapping(resultId);
|
||||||
|
m.setCfhb(cfHb);
|
||||||
|
return m;
|
||||||
|
}
|
||||||
|
|
||||||
|
private static <T extends Oaf> MapGroupsFunction<String, Tuple2<T, IdCfHbMapping>, T> getMapGroupsFunction() {
|
||||||
|
return new MapGroupsFunction<String, Tuple2<T, IdCfHbMapping>, T>() {
|
||||||
|
@Override
|
||||||
|
public T call(String key, Iterator<Tuple2<T, IdCfHbMapping>> values) {
|
||||||
|
final Tuple2<T, IdCfHbMapping> first = values.next();
|
||||||
|
final T res = first._1();
|
||||||
|
|
||||||
|
updateResult(res, first._2());
|
||||||
|
values.forEachRemaining(t -> updateResult(res, t._2()));
|
||||||
|
return res;
|
||||||
|
}
|
||||||
|
|
||||||
|
private void updateResult(T res, IdCfHbMapping m) {
|
||||||
|
if (Objects.nonNull(m)) {
|
||||||
|
res.getCollectedfrom().forEach(kv -> updateKeyValue(kv, m));
|
||||||
|
((Result) res).getInstance().forEach(i -> {
|
||||||
|
updateKeyValue(i.getHostedby(), m);
|
||||||
|
updateKeyValue(i.getCollectedfrom(), m);
|
||||||
|
});
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private void updateKeyValue(final KeyValue kv, final IdCfHbMapping a) {
|
||||||
|
if (kv.getKey().equals(a.getCfhb())) {
|
||||||
|
kv.setKey(a.getMasterId());
|
||||||
|
kv.setValue(a.getMasterName());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
};
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -83,12 +83,17 @@
|
||||||
</property>
|
</property>
|
||||||
</parameters>
|
</parameters>
|
||||||
|
|
||||||
<start to="select_datasourceId_from_country"/>
|
<start to="prepare_info"/>
|
||||||
|
|
||||||
<kill name="Kill">
|
<kill name="Kill">
|
||||||
<message>Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}]</message>
|
<message>Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}]</message>
|
||||||
</kill>
|
</kill>
|
||||||
|
|
||||||
|
<fork name="prepare_info">
|
||||||
|
<path start="select_datasourceId_from_country"/>
|
||||||
|
<path start="get_ds_master_duplicate"/>
|
||||||
|
</fork>
|
||||||
|
|
||||||
<action name="select_datasourceId_from_country">
|
<action name="select_datasourceId_from_country">
|
||||||
<spark xmlns="uri:oozie:spark-action:0.2">
|
<spark xmlns="uri:oozie:spark-action:0.2">
|
||||||
<master>yarn</master>
|
<master>yarn</master>
|
||||||
|
@ -110,10 +115,25 @@
|
||||||
<arg>--workingDir</arg><arg>${workingDir}/working/hostedby</arg>
|
<arg>--workingDir</arg><arg>${workingDir}/working/hostedby</arg>
|
||||||
<arg>--country</arg><arg>${country}</arg>
|
<arg>--country</arg><arg>${country}</arg>
|
||||||
</spark>
|
</spark>
|
||||||
<ok to="fork_clean_graph"/>
|
<ok to="wait_prepare"/>
|
||||||
<error to="Kill"/>
|
<error to="Kill"/>
|
||||||
</action>
|
</action>
|
||||||
|
|
||||||
|
<action name="get_ds_master_duplicate">
|
||||||
|
<java>
|
||||||
|
<main-class>eu.dnetlib.dhp.oa.graph.clean.MasterDuplicateAction</main-class>
|
||||||
|
<arg>--postgresUrl</arg><arg>${postgresURL}</arg>
|
||||||
|
<arg>--postgresUser</arg><arg>${postgresUser}</arg>
|
||||||
|
<arg>--postgresPassword</arg><arg>${postgresPassword}</arg>
|
||||||
|
<arg>--hdfsPath</arg><arg>${workingDir}/masterduplicate</arg>
|
||||||
|
<arg>--hdfsNameNode</arg><arg>${nameNode}</arg>
|
||||||
|
</java>
|
||||||
|
<ok to="wait_prepare"/>
|
||||||
|
<error to="Kill"/>
|
||||||
|
</action>
|
||||||
|
|
||||||
|
<join name="wait_prepare" to="fork_clean_graph"/>
|
||||||
|
|
||||||
<fork name="fork_clean_graph">
|
<fork name="fork_clean_graph">
|
||||||
<path start="clean_publication"/>
|
<path start="clean_publication"/>
|
||||||
<path start="clean_dataset"/>
|
<path start="clean_dataset"/>
|
||||||
|
@ -152,6 +172,8 @@
|
||||||
<arg>--verifyCountryParam</arg><arg>${verifyCountryParam}</arg>
|
<arg>--verifyCountryParam</arg><arg>${verifyCountryParam}</arg>
|
||||||
<arg>--hostedBy</arg><arg>${workingDir}/working/hostedby</arg>
|
<arg>--hostedBy</arg><arg>${workingDir}/working/hostedby</arg>
|
||||||
<arg>--collectedfrom</arg><arg>${collectedfrom}</arg>
|
<arg>--collectedfrom</arg><arg>${collectedfrom}</arg>
|
||||||
|
<arg>--masterDuplicatePath</arg><arg>${workingDir}/masterduplicate</arg>
|
||||||
|
<arg>--deepClean</arg><arg>${shouldClean}</arg>
|
||||||
</spark>
|
</spark>
|
||||||
<ok to="wait_clean"/>
|
<ok to="wait_clean"/>
|
||||||
<error to="Kill"/>
|
<error to="Kill"/>
|
||||||
|
@ -184,6 +206,8 @@
|
||||||
<arg>--verifyCountryParam</arg><arg>${verifyCountryParam}</arg>
|
<arg>--verifyCountryParam</arg><arg>${verifyCountryParam}</arg>
|
||||||
<arg>--hostedBy</arg><arg>${workingDir}/working/hostedby</arg>
|
<arg>--hostedBy</arg><arg>${workingDir}/working/hostedby</arg>
|
||||||
<arg>--collectedfrom</arg><arg>${collectedfrom}</arg>
|
<arg>--collectedfrom</arg><arg>${collectedfrom}</arg>
|
||||||
|
<arg>--masterDuplicatePath</arg><arg>${workingDir}/masterduplicate</arg>
|
||||||
|
<arg>--deepClean</arg><arg>${shouldClean}</arg>
|
||||||
</spark>
|
</spark>
|
||||||
<ok to="wait_clean"/>
|
<ok to="wait_clean"/>
|
||||||
<error to="Kill"/>
|
<error to="Kill"/>
|
||||||
|
@ -216,6 +240,8 @@
|
||||||
<arg>--verifyCountryParam</arg><arg>${verifyCountryParam}</arg>
|
<arg>--verifyCountryParam</arg><arg>${verifyCountryParam}</arg>
|
||||||
<arg>--hostedBy</arg><arg>${workingDir}/working/hostedby</arg>
|
<arg>--hostedBy</arg><arg>${workingDir}/working/hostedby</arg>
|
||||||
<arg>--collectedfrom</arg><arg>${collectedfrom}</arg>
|
<arg>--collectedfrom</arg><arg>${collectedfrom}</arg>
|
||||||
|
<arg>--masterDuplicatePath</arg><arg>${workingDir}/masterduplicate</arg>
|
||||||
|
<arg>--deepClean</arg><arg>${shouldClean}</arg>
|
||||||
</spark>
|
</spark>
|
||||||
<ok to="wait_clean"/>
|
<ok to="wait_clean"/>
|
||||||
<error to="Kill"/>
|
<error to="Kill"/>
|
||||||
|
@ -248,6 +274,8 @@
|
||||||
<arg>--verifyCountryParam</arg><arg>${verifyCountryParam}</arg>
|
<arg>--verifyCountryParam</arg><arg>${verifyCountryParam}</arg>
|
||||||
<arg>--hostedBy</arg><arg>${workingDir}/working/hostedby</arg>
|
<arg>--hostedBy</arg><arg>${workingDir}/working/hostedby</arg>
|
||||||
<arg>--collectedfrom</arg><arg>${collectedfrom}</arg>
|
<arg>--collectedfrom</arg><arg>${collectedfrom}</arg>
|
||||||
|
<arg>--masterDuplicatePath</arg><arg>${workingDir}/masterduplicate</arg>
|
||||||
|
<arg>--deepClean</arg><arg>${shouldClean}</arg>
|
||||||
</spark>
|
</spark>
|
||||||
<ok to="wait_clean"/>
|
<ok to="wait_clean"/>
|
||||||
<error to="Kill"/>
|
<error to="Kill"/>
|
||||||
|
@ -280,6 +308,8 @@
|
||||||
<arg>--verifyCountryParam</arg><arg>${verifyCountryParam}</arg>
|
<arg>--verifyCountryParam</arg><arg>${verifyCountryParam}</arg>
|
||||||
<arg>--hostedBy</arg><arg>${workingDir}/working/hostedby</arg>
|
<arg>--hostedBy</arg><arg>${workingDir}/working/hostedby</arg>
|
||||||
<arg>--collectedfrom</arg><arg>${collectedfrom}</arg>
|
<arg>--collectedfrom</arg><arg>${collectedfrom}</arg>
|
||||||
|
<arg>--masterDuplicatePath</arg><arg>${workingDir}/masterduplicate</arg>
|
||||||
|
<arg>--deepClean</arg><arg>${shouldClean}</arg>
|
||||||
</spark>
|
</spark>
|
||||||
<ok to="wait_clean"/>
|
<ok to="wait_clean"/>
|
||||||
<error to="Kill"/>
|
<error to="Kill"/>
|
||||||
|
@ -312,6 +342,8 @@
|
||||||
<arg>--verifyCountryParam</arg><arg>${verifyCountryParam}</arg>
|
<arg>--verifyCountryParam</arg><arg>${verifyCountryParam}</arg>
|
||||||
<arg>--hostedBy</arg><arg>${workingDir}/working/hostedby</arg>
|
<arg>--hostedBy</arg><arg>${workingDir}/working/hostedby</arg>
|
||||||
<arg>--collectedfrom</arg><arg>${collectedfrom}</arg>
|
<arg>--collectedfrom</arg><arg>${collectedfrom}</arg>
|
||||||
|
<arg>--masterDuplicatePath</arg><arg>${workingDir}/masterduplicate</arg>
|
||||||
|
<arg>--deepClean</arg><arg>${shouldClean}</arg>
|
||||||
</spark>
|
</spark>
|
||||||
<ok to="wait_clean"/>
|
<ok to="wait_clean"/>
|
||||||
<error to="Kill"/>
|
<error to="Kill"/>
|
||||||
|
@ -344,6 +376,8 @@
|
||||||
<arg>--verifyCountryParam</arg><arg>${verifyCountryParam}</arg>
|
<arg>--verifyCountryParam</arg><arg>${verifyCountryParam}</arg>
|
||||||
<arg>--hostedBy</arg><arg>${workingDir}/working/hostedby</arg>
|
<arg>--hostedBy</arg><arg>${workingDir}/working/hostedby</arg>
|
||||||
<arg>--collectedfrom</arg><arg>${collectedfrom}</arg>
|
<arg>--collectedfrom</arg><arg>${collectedfrom}</arg>
|
||||||
|
<arg>--masterDuplicatePath</arg><arg>${workingDir}/masterduplicate</arg>
|
||||||
|
<arg>--deepClean</arg><arg>${shouldClean}</arg>
|
||||||
</spark>
|
</spark>
|
||||||
<ok to="wait_clean"/>
|
<ok to="wait_clean"/>
|
||||||
<error to="Kill"/>
|
<error to="Kill"/>
|
||||||
|
@ -376,206 +410,14 @@
|
||||||
<arg>--verifyCountryParam</arg><arg>${verifyCountryParam}</arg>
|
<arg>--verifyCountryParam</arg><arg>${verifyCountryParam}</arg>
|
||||||
<arg>--hostedBy</arg><arg>${workingDir}/working/hostedby</arg>
|
<arg>--hostedBy</arg><arg>${workingDir}/working/hostedby</arg>
|
||||||
<arg>--collectedfrom</arg><arg>${collectedfrom}</arg>
|
<arg>--collectedfrom</arg><arg>${collectedfrom}</arg>
|
||||||
|
<arg>--masterDuplicatePath</arg><arg>${workingDir}/masterduplicate</arg>
|
||||||
|
<arg>--deepClean</arg><arg>${shouldClean}</arg>
|
||||||
</spark>
|
</spark>
|
||||||
<ok to="wait_clean"/>
|
<ok to="wait_clean"/>
|
||||||
<error to="Kill"/>
|
<error to="Kill"/>
|
||||||
</action>
|
</action>
|
||||||
|
|
||||||
<join name="wait_clean" to="should_patch_datasource_ids"/>
|
<join name="wait_clean" to="End"/>
|
||||||
|
|
||||||
<decision name="should_patch_datasource_ids">
|
|
||||||
<switch>
|
|
||||||
<case to="get_ds_master_duplicate">${wf:conf('shouldClean') eq true}</case>
|
|
||||||
<default to="End"/>
|
|
||||||
</switch>
|
|
||||||
</decision>
|
|
||||||
|
|
||||||
<action name="get_ds_master_duplicate">
|
|
||||||
<java>
|
|
||||||
<main-class>eu.dnetlib.dhp.oa.graph.clean.MasterDuplicateAction</main-class>
|
|
||||||
<arg>--postgresUrl</arg><arg>${postgresURL}</arg>
|
|
||||||
<arg>--postgresUser</arg><arg>${postgresUser}</arg>
|
|
||||||
<arg>--postgresPassword</arg><arg>${postgresPassword}</arg>
|
|
||||||
<arg>--hdfsPath</arg><arg>${workingDir}/masterduplicate</arg>
|
|
||||||
<arg>--hdfsNameNode</arg><arg>${nameNode}</arg>
|
|
||||||
</java>
|
|
||||||
<ok to="fork_patch_cfhb"/>
|
|
||||||
<error to="Kill"/>
|
|
||||||
</action>
|
|
||||||
|
|
||||||
<fork name="fork_patch_cfhb">
|
|
||||||
<path start="patch_publication_cfhb"/>
|
|
||||||
<path start="patch_dataset_cfhb"/>
|
|
||||||
<path start="patch_otherresearchproduct_cfhb"/>
|
|
||||||
<path start="patch_software_cfhb"/>
|
|
||||||
</fork>
|
|
||||||
|
|
||||||
<action name="patch_publication_cfhb">
|
|
||||||
<spark xmlns="uri:oozie:spark-action:0.2">
|
|
||||||
<master>yarn</master>
|
|
||||||
<mode>cluster</mode>
|
|
||||||
<name>patch publication cfhb</name>
|
|
||||||
<class>eu.dnetlib.dhp.oa.graph.clean.cfhb.CleanCfHbSparkJob</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>${graphOutputPath}/publication</arg>
|
|
||||||
<arg>--resolvedPath</arg><arg>${workingDir}/cfHbResolved/publication</arg>
|
|
||||||
<arg>--outputPath</arg><arg>${workingDir}/cfHbPatched/publication</arg>
|
|
||||||
<arg>--graphTableClassName</arg><arg>eu.dnetlib.dhp.schema.oaf.Publication</arg>
|
|
||||||
<arg>--masterDuplicatePath</arg><arg>${workingDir}/masterduplicate</arg>
|
|
||||||
</spark>
|
|
||||||
<ok to="wait_clean_cfhb"/>
|
|
||||||
<error to="Kill"/>
|
|
||||||
</action>
|
|
||||||
|
|
||||||
<action name="patch_dataset_cfhb">
|
|
||||||
<spark xmlns="uri:oozie:spark-action:0.2">
|
|
||||||
<master>yarn</master>
|
|
||||||
<mode>cluster</mode>
|
|
||||||
<name>patch dataset cfhb</name>
|
|
||||||
<class>eu.dnetlib.dhp.oa.graph.clean.cfhb.CleanCfHbSparkJob</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>${graphOutputPath}/dataset</arg>
|
|
||||||
<arg>--resolvedPath</arg><arg>${workingDir}/cfHbResolved/dataset</arg>
|
|
||||||
<arg>--outputPath</arg><arg>${workingDir}/cfHbPatched/dataset</arg>
|
|
||||||
<arg>--graphTableClassName</arg><arg>eu.dnetlib.dhp.schema.oaf.Dataset</arg>
|
|
||||||
<arg>--masterDuplicatePath</arg><arg>${workingDir}/masterduplicate</arg>
|
|
||||||
</spark>
|
|
||||||
<ok to="wait_clean_cfhb"/>
|
|
||||||
<error to="Kill"/>
|
|
||||||
</action>
|
|
||||||
|
|
||||||
<action name="patch_otherresearchproduct_cfhb">
|
|
||||||
<spark xmlns="uri:oozie:spark-action:0.2">
|
|
||||||
<master>yarn</master>
|
|
||||||
<mode>cluster</mode>
|
|
||||||
<name>patch otherresearchproduct cfhb</name>
|
|
||||||
<class>eu.dnetlib.dhp.oa.graph.clean.cfhb.CleanCfHbSparkJob</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>${graphOutputPath}/otherresearchproduct</arg>
|
|
||||||
<arg>--resolvedPath</arg><arg>${workingDir}/cfHbResolved/otherresearchproduct</arg>
|
|
||||||
<arg>--outputPath</arg><arg>${workingDir}/cfHbPatched/otherresearchproduct</arg>
|
|
||||||
<arg>--graphTableClassName</arg><arg>eu.dnetlib.dhp.schema.oaf.OtherResearchProduct</arg>
|
|
||||||
<arg>--masterDuplicatePath</arg><arg>${workingDir}/masterduplicate</arg>
|
|
||||||
</spark>
|
|
||||||
<ok to="wait_clean_cfhb"/>
|
|
||||||
<error to="Kill"/>
|
|
||||||
</action>
|
|
||||||
|
|
||||||
<action name="patch_software_cfhb">
|
|
||||||
<spark xmlns="uri:oozie:spark-action:0.2">
|
|
||||||
<master>yarn</master>
|
|
||||||
<mode>cluster</mode>
|
|
||||||
<name>patch software cfhb</name>
|
|
||||||
<class>eu.dnetlib.dhp.oa.graph.clean.cfhb.CleanCfHbSparkJob</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>${graphOutputPath}/software</arg>
|
|
||||||
<arg>--resolvedPath</arg><arg>${workingDir}/cfHbResolved/software</arg>
|
|
||||||
<arg>--outputPath</arg><arg>${workingDir}/cfHbPatched/software</arg>
|
|
||||||
<arg>--graphTableClassName</arg><arg>eu.dnetlib.dhp.schema.oaf.Software</arg>
|
|
||||||
<arg>--masterDuplicatePath</arg><arg>${workingDir}/masterduplicate</arg>
|
|
||||||
</spark>
|
|
||||||
<ok to="wait_clean_cfhb"/>
|
|
||||||
<error to="Kill"/>
|
|
||||||
</action>
|
|
||||||
|
|
||||||
<join name="wait_clean_cfhb" to="fork_copy_cfhb_patched_results"/>
|
|
||||||
|
|
||||||
<fork name="fork_copy_cfhb_patched_results">
|
|
||||||
<path start="copy_cfhb_patched_publication"/>
|
|
||||||
<path start="copy_cfhb_patched_dataset"/>
|
|
||||||
<path start="copy_cfhb_patched_otherresearchproduct"/>
|
|
||||||
<path start="copy_cfhb_patched_software"/>
|
|
||||||
</fork>
|
|
||||||
|
|
||||||
<action name="copy_cfhb_patched_publication">
|
|
||||||
<distcp xmlns="uri:oozie:distcp-action:0.2">
|
|
||||||
<prepare>
|
|
||||||
<delete path="${graphOutputPath}/publication"/>
|
|
||||||
</prepare>
|
|
||||||
<arg>${workingDir}/cfHbPatched/publication</arg>
|
|
||||||
<arg>${graphOutputPath}/publication</arg>
|
|
||||||
</distcp>
|
|
||||||
<ok to="copy_wait"/>
|
|
||||||
<error to="Kill"/>
|
|
||||||
</action>
|
|
||||||
|
|
||||||
<action name="copy_cfhb_patched_dataset">
|
|
||||||
<distcp xmlns="uri:oozie:distcp-action:0.2">
|
|
||||||
<prepare>
|
|
||||||
<delete path="${graphOutputPath}/dataset"/>
|
|
||||||
</prepare>
|
|
||||||
<arg>${workingDir}/cfHbPatched/dataset</arg>
|
|
||||||
<arg>${graphOutputPath}/dataset</arg>
|
|
||||||
</distcp>
|
|
||||||
<ok to="copy_wait"/>
|
|
||||||
<error to="Kill"/>
|
|
||||||
</action>
|
|
||||||
|
|
||||||
<action name="copy_cfhb_patched_otherresearchproduct">
|
|
||||||
<distcp xmlns="uri:oozie:distcp-action:0.2">
|
|
||||||
<prepare>
|
|
||||||
<delete path="${graphOutputPath}/otherresearchproduct"/>
|
|
||||||
</prepare>
|
|
||||||
<arg>${workingDir}/cfHbPatched/otherresearchproduct</arg>
|
|
||||||
<arg>${graphOutputPath}/otherresearchproduct</arg>
|
|
||||||
</distcp>
|
|
||||||
<ok to="copy_wait"/>
|
|
||||||
<error to="Kill"/>
|
|
||||||
</action>
|
|
||||||
|
|
||||||
<action name="copy_cfhb_patched_software">
|
|
||||||
<distcp xmlns="uri:oozie:distcp-action:0.2">
|
|
||||||
<prepare>
|
|
||||||
<delete path="${graphOutputPath}/software"/>
|
|
||||||
</prepare>
|
|
||||||
<arg>${workingDir}/cfHbPatched/software</arg>
|
|
||||||
<arg>${graphOutputPath}/software</arg>
|
|
||||||
</distcp>
|
|
||||||
<ok to="copy_wait"/>
|
|
||||||
<error to="Kill"/>
|
|
||||||
</action>
|
|
||||||
|
|
||||||
<join name="copy_wait" to="End"/>
|
|
||||||
|
|
||||||
<end name="End"/>
|
<end name="End"/>
|
||||||
|
|
||||||
|
|
|
@ -33,30 +33,48 @@
|
||||||
"paramName": "ci",
|
"paramName": "ci",
|
||||||
"paramLongName": "contextId",
|
"paramLongName": "contextId",
|
||||||
"paramDescription": "the id of the context to be removed",
|
"paramDescription": "the id of the context to be removed",
|
||||||
"paramRequired": true
|
"paramRequired": false
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"paramName": "vf",
|
||||||
|
"paramLongName": "verifyParam",
|
||||||
|
"paramDescription": "the parameter to be verified to remove the context",
|
||||||
|
"paramRequired": false
|
||||||
},
|
},
|
||||||
{
|
{
|
||||||
"paramName": "c",
|
"paramName": "c",
|
||||||
"paramLongName": "country",
|
"paramLongName": "country",
|
||||||
"paramDescription": "the id of the context to be removed",
|
"paramDescription": "the id of the context to be removed",
|
||||||
"paramRequired": true
|
"paramRequired": false
|
||||||
},
|
},
|
||||||
{
|
{
|
||||||
"paramName": "vfc",
|
"paramName": "vfc",
|
||||||
"paramLongName": "verifyCountryParam",
|
"paramLongName": "verifyCountryParam",
|
||||||
"paramDescription": "the parameter to be verified to remove the country",
|
"paramDescription": "the parameter to be verified to remove the country",
|
||||||
"paramRequired": true
|
"paramRequired": false
|
||||||
},
|
},
|
||||||
{
|
{
|
||||||
"paramName": "cf",
|
"paramName": "cf",
|
||||||
"paramLongName": "collectedfrom",
|
"paramLongName": "collectedfrom",
|
||||||
"paramDescription": "the collectedfrom value for which we should apply the cleaning",
|
"paramDescription": "the collectedfrom value for which we should apply the cleaning",
|
||||||
"paramRequired": true
|
"paramRequired": false
|
||||||
},
|
},
|
||||||
{
|
{
|
||||||
"paramName": "hb",
|
"paramName": "hb",
|
||||||
"paramLongName": "hostedBy",
|
"paramLongName": "hostedBy",
|
||||||
"paramDescription": "the set of datasources having the specified country in the graph searched for in the hostedby of the results",
|
"paramDescription": "the set of datasources having the specified country in the graph searched for in the hostedby of the results",
|
||||||
|
"paramRequired": false
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"paramName": "md",
|
||||||
|
"paramLongName": "masterDuplicatePath",
|
||||||
|
"paramDescription": "path to the file on HDFS holding the datasource id tuples [master, duplicate]",
|
||||||
|
"paramRequired": false
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"paramName": "dc",
|
||||||
|
"paramLongName": "deepClean",
|
||||||
|
"paramDescription": "flag to activate further cleaning steps",
|
||||||
"paramRequired": true
|
"paramRequired": true
|
||||||
}
|
}
|
||||||
]
|
]
|
||||||
|
|
|
@ -0,0 +1,435 @@
|
||||||
|
package eu.dnetlib.dhp.oa.graph.clean;
|
||||||
|
|
||||||
|
import com.fasterxml.jackson.databind.DeserializationFeature;
|
||||||
|
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||||
|
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
|
||||||
|
import eu.dnetlib.dhp.common.vocabulary.VocabularyGroup;
|
||||||
|
import eu.dnetlib.dhp.schema.common.ModelConstants;
|
||||||
|
import eu.dnetlib.dhp.schema.oaf.*;
|
||||||
|
import eu.dnetlib.dhp.schema.oaf.utils.GraphCleaningFunctions;
|
||||||
|
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException;
|
||||||
|
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService;
|
||||||
|
import org.apache.commons.cli.ParseException;
|
||||||
|
import org.apache.commons.io.FileUtils;
|
||||||
|
import org.apache.commons.io.IOUtils;
|
||||||
|
import org.apache.commons.io.filefilter.*;
|
||||||
|
import org.apache.commons.lang3.StringUtils;
|
||||||
|
import org.apache.spark.SparkConf;
|
||||||
|
import org.apache.spark.api.java.function.ForeachFunction;
|
||||||
|
import org.apache.spark.api.java.function.MapFunction;
|
||||||
|
import org.apache.spark.sql.Dataset;
|
||||||
|
import org.apache.spark.sql.Encoders;
|
||||||
|
import org.apache.spark.sql.SparkSession;
|
||||||
|
import org.junit.jupiter.api.AfterAll;
|
||||||
|
import org.junit.jupiter.api.BeforeAll;
|
||||||
|
import org.junit.jupiter.api.BeforeEach;
|
||||||
|
import org.junit.jupiter.api.Test;
|
||||||
|
import org.junit.jupiter.api.extension.ExtendWith;
|
||||||
|
import org.mockito.Mock;
|
||||||
|
import org.mockito.junit.jupiter.MockitoExtension;
|
||||||
|
import org.slf4j.Logger;
|
||||||
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
|
import java.io.File;
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.net.URISyntaxException;
|
||||||
|
import java.nio.file.Files;
|
||||||
|
import java.nio.file.Path;
|
||||||
|
import java.nio.file.Paths;
|
||||||
|
import java.util.Collection;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Optional;
|
||||||
|
import java.util.Set;
|
||||||
|
import java.util.stream.Collectors;
|
||||||
|
import java.util.stream.Stream;
|
||||||
|
|
||||||
|
import static org.junit.jupiter.api.Assertions.*;
|
||||||
|
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||||
|
import static org.mockito.Mockito.lenient;
|
||||||
|
|
||||||
|
@ExtendWith(MockitoExtension.class)
|
||||||
|
public class CleanGraphSparkJobTest {
|
||||||
|
|
||||||
|
private static final Logger log = LoggerFactory.getLogger(CleanContextTest.class);
|
||||||
|
|
||||||
|
public static final ObjectMapper MAPPER = new ObjectMapper()
|
||||||
|
.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
|
||||||
|
|
||||||
|
@Mock
|
||||||
|
private ISLookUpService isLookUpService;
|
||||||
|
|
||||||
|
private VocabularyGroup vocabularies;
|
||||||
|
|
||||||
|
private CleaningRuleMap mapping;
|
||||||
|
|
||||||
|
private static SparkSession spark;
|
||||||
|
|
||||||
|
private static Path workingDir;
|
||||||
|
|
||||||
|
private static Path testBaseTmpPath;
|
||||||
|
|
||||||
|
private static String graphInputPath;
|
||||||
|
|
||||||
|
private static String graphOutputPath;
|
||||||
|
|
||||||
|
private static String dsMasterDuplicatePath;
|
||||||
|
|
||||||
|
@BeforeAll
|
||||||
|
public static void beforeAll() throws IOException, URISyntaxException {
|
||||||
|
testBaseTmpPath = Files.createTempDirectory(CleanGraphSparkJobTest.class.getSimpleName());
|
||||||
|
log.info("using test base path {}", testBaseTmpPath);
|
||||||
|
|
||||||
|
File basePath = Paths
|
||||||
|
.get(CleanGraphSparkJobTest.class.getResource("/eu/dnetlib/dhp/oa/graph/clean/graph").toURI())
|
||||||
|
.toFile();
|
||||||
|
|
||||||
|
|
||||||
|
List<File> paths = FileUtils
|
||||||
|
.listFilesAndDirs(basePath, FalseFileFilter.FALSE, TrueFileFilter.TRUE)
|
||||||
|
.stream()
|
||||||
|
.filter(f -> !f.getAbsolutePath().endsWith("/graph"))
|
||||||
|
.collect(Collectors.toList());
|
||||||
|
|
||||||
|
for(File path : paths) {
|
||||||
|
String type = StringUtils.substringAfterLast(path.getAbsolutePath(), "/");
|
||||||
|
FileUtils
|
||||||
|
.copyDirectory(
|
||||||
|
path,
|
||||||
|
testBaseTmpPath.resolve("input").resolve("graph").resolve(type).toFile());
|
||||||
|
}
|
||||||
|
|
||||||
|
FileUtils
|
||||||
|
.copyFileToDirectory(
|
||||||
|
Paths
|
||||||
|
.get(
|
||||||
|
CleanGraphSparkJobTest.class
|
||||||
|
.getResource("/eu/dnetlib/dhp/oa/graph/clean/cfhb/masterduplicate.json")
|
||||||
|
.toURI())
|
||||||
|
.toFile(),
|
||||||
|
testBaseTmpPath.resolve("workingDir").resolve("masterduplicate").toFile());
|
||||||
|
|
||||||
|
graphInputPath = testBaseTmpPath.resolve("input").resolve("graph").toString();
|
||||||
|
graphOutputPath = testBaseTmpPath.resolve("output").resolve("graph").toString();
|
||||||
|
dsMasterDuplicatePath = testBaseTmpPath.resolve("workingDir").resolve("masterduplicate").toString();
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
workingDir = Files.createTempDirectory(CleanGraphSparkJobTest.class.getSimpleName());
|
||||||
|
log.info("using work dir {}", workingDir);
|
||||||
|
|
||||||
|
SparkConf conf = new SparkConf();
|
||||||
|
conf.setAppName(CleanGraphSparkJobTest.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()
|
||||||
|
.config(conf)
|
||||||
|
.getOrCreate();
|
||||||
|
}
|
||||||
|
|
||||||
|
@BeforeEach
|
||||||
|
public void setUp() throws ISLookUpException, IOException {
|
||||||
|
lenient().when(isLookUpService.quickSearchProfile(VocabularyGroup.VOCABULARIES_XQUERY)).thenReturn(vocs());
|
||||||
|
lenient()
|
||||||
|
.when(isLookUpService.quickSearchProfile(VocabularyGroup.VOCABULARY_SYNONYMS_XQUERY))
|
||||||
|
.thenReturn(synonyms());
|
||||||
|
|
||||||
|
vocabularies = VocabularyGroup.loadVocsFromIS(isLookUpService);
|
||||||
|
mapping = CleaningRuleMap.create(vocabularies);
|
||||||
|
}
|
||||||
|
|
||||||
|
@AfterAll
|
||||||
|
public static void afterAll() throws IOException {
|
||||||
|
FileUtils.deleteDirectory(workingDir.toFile());
|
||||||
|
spark.stop();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
void testCleanRelations() throws Exception {
|
||||||
|
|
||||||
|
spark.read()
|
||||||
|
.textFile(graphInputPath.toString() + "/relation")
|
||||||
|
.map(as(Relation.class), Encoders.bean(Relation.class))
|
||||||
|
.collectAsList()
|
||||||
|
.forEach(r -> assertFalse(vocabularies.getTerms(ModelConstants.DNET_RELATION_RELCLASS).contains(r.getRelClass())));
|
||||||
|
|
||||||
|
new CleanGraphSparkJob(
|
||||||
|
args("/eu/dnetlib/dhp/oa/graph/input_clean_graph_parameters.json",
|
||||||
|
new String[] {
|
||||||
|
"--inputPath", graphInputPath.toString() + "/relation",
|
||||||
|
"--outputPath", graphOutputPath.toString() + "/relation",
|
||||||
|
"--isLookupUrl", "lookupurl",
|
||||||
|
"--graphTableClassName", Relation.class.getCanonicalName(),
|
||||||
|
"--deepClean", "false"
|
||||||
|
})).run(false, isLookUpService);
|
||||||
|
|
||||||
|
spark.read()
|
||||||
|
.textFile(graphOutputPath.toString() + "/relation")
|
||||||
|
.map(as(Relation.class), Encoders.bean(Relation.class))
|
||||||
|
.collectAsList()
|
||||||
|
.forEach(r -> {
|
||||||
|
|
||||||
|
assertTrue(vocabularies.getTerms(ModelConstants.DNET_RELATION_RELCLASS).contains(r.getRelClass()));
|
||||||
|
assertTrue(vocabularies.getTerms(ModelConstants.DNET_RELATION_SUBRELTYPE).contains(r.getSubRelType()));
|
||||||
|
|
||||||
|
assertEquals("iis", r.getDataInfo().getProvenanceaction().getClassid());
|
||||||
|
assertEquals("Inferred by OpenAIRE", r.getDataInfo().getProvenanceaction().getClassname());
|
||||||
|
});
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
void testFilter_invisible_true() throws Exception {
|
||||||
|
|
||||||
|
assertNotNull(vocabularies);
|
||||||
|
assertNotNull(mapping);
|
||||||
|
|
||||||
|
String json = IOUtils
|
||||||
|
.toString(getClass().getResourceAsStream("/eu/dnetlib/dhp/oa/graph/clean/result_invisible.json"));
|
||||||
|
Publication p_in = MAPPER.readValue(json, Publication.class);
|
||||||
|
|
||||||
|
assertTrue(p_in instanceof Result);
|
||||||
|
assertTrue(p_in instanceof Publication);
|
||||||
|
|
||||||
|
assertEquals(true, GraphCleaningFunctions.filter(p_in));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
void testFilter_true_nothing_to_filter() throws Exception {
|
||||||
|
|
||||||
|
assertNotNull(vocabularies);
|
||||||
|
assertNotNull(mapping);
|
||||||
|
|
||||||
|
String json = IOUtils.toString(getClass().getResourceAsStream("/eu/dnetlib/dhp/oa/graph/clean/result.json"));
|
||||||
|
Publication p_in = MAPPER.readValue(json, Publication.class);
|
||||||
|
|
||||||
|
assertTrue(p_in instanceof Result);
|
||||||
|
assertTrue(p_in instanceof Publication);
|
||||||
|
|
||||||
|
assertEquals(true, GraphCleaningFunctions.filter(p_in));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
void testFilter_missing_invisible() throws Exception {
|
||||||
|
|
||||||
|
assertNotNull(vocabularies);
|
||||||
|
assertNotNull(mapping);
|
||||||
|
|
||||||
|
String json = IOUtils
|
||||||
|
.toString(getClass().getResourceAsStream("/eu/dnetlib/dhp/oa/graph/clean/result_missing_invisible.json"));
|
||||||
|
Publication p_in = MAPPER.readValue(json, Publication.class);
|
||||||
|
|
||||||
|
assertTrue(p_in instanceof Result);
|
||||||
|
assertTrue(p_in instanceof Publication);
|
||||||
|
|
||||||
|
assertEquals(true, GraphCleaningFunctions.filter(p_in));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
void testCleaning_publication() throws Exception {
|
||||||
|
|
||||||
|
spark.read()
|
||||||
|
.textFile(graphInputPath.toString() + "/publication")
|
||||||
|
.map(as(Publication.class), Encoders.bean(Publication.class))
|
||||||
|
.collectAsList()
|
||||||
|
.forEach(p -> {
|
||||||
|
assertNull(p.getBestaccessright());
|
||||||
|
assertTrue(p instanceof Result);
|
||||||
|
assertTrue(p instanceof Publication);
|
||||||
|
});
|
||||||
|
|
||||||
|
new CleanGraphSparkJob(
|
||||||
|
args("/eu/dnetlib/dhp/oa/graph/input_clean_graph_parameters.json",
|
||||||
|
new String[] {
|
||||||
|
"--inputPath", graphInputPath.toString() + "/publication",
|
||||||
|
"--outputPath", graphOutputPath.toString() + "/publication",
|
||||||
|
"--isLookupUrl", "lookupurl",
|
||||||
|
"--graphTableClassName", Publication.class.getCanonicalName(),
|
||||||
|
"--deepClean", "false"
|
||||||
|
})).run(false, isLookUpService);
|
||||||
|
|
||||||
|
Publication p = spark.read()
|
||||||
|
.textFile(graphOutputPath.toString() + "/publication")
|
||||||
|
.map(as(Publication.class), Encoders.bean(Publication.class))
|
||||||
|
.first();
|
||||||
|
|
||||||
|
assertNull(p.getPublisher());
|
||||||
|
|
||||||
|
assertEquals("und", p.getLanguage().getClassid());
|
||||||
|
assertEquals("Undetermined", p.getLanguage().getClassname());
|
||||||
|
|
||||||
|
assertEquals("DE", p.getCountry().get(0).getClassid());
|
||||||
|
assertEquals("Germany", p.getCountry().get(0).getClassname());
|
||||||
|
|
||||||
|
assertEquals("0018", p.getInstance().get(0).getInstancetype().getClassid());
|
||||||
|
assertEquals("Annotation", p.getInstance().get(0).getInstancetype().getClassname());
|
||||||
|
|
||||||
|
assertEquals("0027", p.getInstance().get(1).getInstancetype().getClassid());
|
||||||
|
assertEquals("Model", p.getInstance().get(1).getInstancetype().getClassname());
|
||||||
|
|
||||||
|
assertEquals("0038", p.getInstance().get(2).getInstancetype().getClassid());
|
||||||
|
assertEquals("Other literature type", p.getInstance().get(2).getInstancetype().getClassname());
|
||||||
|
|
||||||
|
assertEquals("CLOSED", p.getInstance().get(0).getAccessright().getClassid());
|
||||||
|
assertEquals("Closed Access", p.getInstance().get(0).getAccessright().getClassname());
|
||||||
|
|
||||||
|
Set<String> pidTerms = vocabularies.getTerms(ModelConstants.DNET_PID_TYPES);
|
||||||
|
assertTrue(
|
||||||
|
p
|
||||||
|
.getPid()
|
||||||
|
.stream()
|
||||||
|
.map(StructuredProperty::getQualifier)
|
||||||
|
.allMatch(q -> pidTerms.contains(q.getClassid())));
|
||||||
|
|
||||||
|
List<Instance> poi = p.getInstance();
|
||||||
|
assertNotNull(poi);
|
||||||
|
assertEquals(3, poi.size());
|
||||||
|
|
||||||
|
final Instance poii = poi.get(0);
|
||||||
|
assertNotNull(poii);
|
||||||
|
assertNotNull(poii.getPid());
|
||||||
|
|
||||||
|
assertEquals(2, poii.getPid().size());
|
||||||
|
|
||||||
|
assertTrue(
|
||||||
|
poii.getPid().stream().anyMatch(s -> s.getValue().equals("10.1007/s109090161569x")));
|
||||||
|
assertTrue(poii.getPid().stream().anyMatch(s -> s.getValue().equals("10.1008/abcd")));
|
||||||
|
|
||||||
|
assertNotNull(poii.getAlternateIdentifier());
|
||||||
|
assertEquals(1, poii.getAlternateIdentifier().size());
|
||||||
|
|
||||||
|
assertTrue(
|
||||||
|
poii
|
||||||
|
.getAlternateIdentifier()
|
||||||
|
.stream()
|
||||||
|
.anyMatch(s -> s.getValue().equals("10.1009/qwerty")));
|
||||||
|
|
||||||
|
assertEquals(3, p.getTitle().size());
|
||||||
|
|
||||||
|
|
||||||
|
List<String> titles = p
|
||||||
|
.getTitle()
|
||||||
|
.stream()
|
||||||
|
.map(StructuredProperty::getValue)
|
||||||
|
.collect(Collectors.toList());
|
||||||
|
assertTrue(titles.contains("omic"));
|
||||||
|
assertTrue(
|
||||||
|
titles.contains("Optical response of strained- and unstrained-silicon cold-electron bolometers test"));
|
||||||
|
assertTrue(titles.contains("「マキャベリ的知性と心の理論の進化論」 リチャード・バーン, アンドリュー・ホワイトゥン 編/藤田和生, 山下博志, 友永雅巳 監訳"));
|
||||||
|
|
||||||
|
assertEquals("CLOSED", p.getBestaccessright().getClassid());
|
||||||
|
assertNull(p.getPublisher());
|
||||||
|
|
||||||
|
assertEquals("1970-10-07", p.getDateofacceptance().getValue());
|
||||||
|
|
||||||
|
assertEquals("0038", p.getInstance().get(2).getInstancetype().getClassid());
|
||||||
|
assertEquals("Other literature type", p.getInstance().get(2).getInstancetype().getClassname());
|
||||||
|
|
||||||
|
final List<Instance> pci = p.getInstance();
|
||||||
|
assertNotNull(pci);
|
||||||
|
assertEquals(3, pci.size());
|
||||||
|
|
||||||
|
final Instance pcii = pci.get(0);
|
||||||
|
assertNotNull(pcii);
|
||||||
|
assertNotNull(pcii.getPid());
|
||||||
|
|
||||||
|
assertEquals(2, pcii.getPid().size());
|
||||||
|
|
||||||
|
assertTrue(
|
||||||
|
pcii.getPid().stream().anyMatch(s -> s.getValue().equals("10.1007/s109090161569x")));
|
||||||
|
assertTrue(pcii.getPid().stream().anyMatch(s -> s.getValue().equals("10.1008/abcd")));
|
||||||
|
|
||||||
|
assertNotNull(pcii.getAlternateIdentifier());
|
||||||
|
assertEquals(1, pcii.getAlternateIdentifier().size());
|
||||||
|
assertTrue(
|
||||||
|
pcii
|
||||||
|
.getAlternateIdentifier()
|
||||||
|
.stream()
|
||||||
|
.anyMatch(s -> s.getValue().equals("10.1009/qwerty")));
|
||||||
|
|
||||||
|
assertNotNull(p.getSubject());
|
||||||
|
|
||||||
|
List<Subject> fos_subjects = p
|
||||||
|
.getSubject()
|
||||||
|
.stream()
|
||||||
|
.filter(s -> ModelConstants.DNET_SUBJECT_FOS_CLASSID.equals(s.getQualifier().getClassid()))
|
||||||
|
.collect(Collectors.toList());
|
||||||
|
|
||||||
|
assertNotNull(fos_subjects);
|
||||||
|
assertEquals(2, fos_subjects.size());
|
||||||
|
|
||||||
|
assertTrue(
|
||||||
|
fos_subjects
|
||||||
|
.stream()
|
||||||
|
.anyMatch(
|
||||||
|
s -> "0101 mathematics".equals(s.getValue()) &
|
||||||
|
ModelConstants.DNET_SUBJECT_FOS_CLASSID.equals(s.getQualifier().getClassid()) &
|
||||||
|
"sysimport:crosswalk:datasetarchive"
|
||||||
|
.equals(s.getDataInfo().getProvenanceaction().getClassid())));
|
||||||
|
|
||||||
|
assertTrue(
|
||||||
|
fos_subjects
|
||||||
|
.stream()
|
||||||
|
.anyMatch(
|
||||||
|
s -> "0102 computer and information sciences".equals(s.getValue()) &
|
||||||
|
ModelConstants.DNET_SUBJECT_FOS_CLASSID.equals(s.getQualifier().getClassid())));
|
||||||
|
|
||||||
|
verify_keyword(p, "In Situ Hybridization");
|
||||||
|
verify_keyword(p, "Avicennia");
|
||||||
|
}
|
||||||
|
|
||||||
|
private List<String> vocs() throws IOException {
|
||||||
|
return IOUtils
|
||||||
|
.readLines(
|
||||||
|
GraphCleaningFunctionsTest.class.getResourceAsStream("/eu/dnetlib/dhp/oa/graph/clean/terms.txt"));
|
||||||
|
}
|
||||||
|
|
||||||
|
private List<String> synonyms() throws IOException {
|
||||||
|
return IOUtils
|
||||||
|
.readLines(
|
||||||
|
GraphCleaningFunctionsTest.class.getResourceAsStream("/eu/dnetlib/dhp/oa/graph/clean/synonyms.txt"));
|
||||||
|
}
|
||||||
|
|
||||||
|
private static <R> MapFunction<String, R> as(Class<R> clazz) {
|
||||||
|
return s -> MAPPER.readValue(s, clazz);
|
||||||
|
}
|
||||||
|
|
||||||
|
private static String classPathResourceAsString(String path) throws IOException {
|
||||||
|
return IOUtils
|
||||||
|
.toString(
|
||||||
|
CleanGraphSparkJobTest.class
|
||||||
|
.getResourceAsStream(path));
|
||||||
|
}
|
||||||
|
|
||||||
|
private ArgumentApplicationParser args(String paramSpecs, String[] args) throws IOException, ParseException {
|
||||||
|
ArgumentApplicationParser parser = new ArgumentApplicationParser(classPathResourceAsString(paramSpecs));
|
||||||
|
parser.parseArgument(args);
|
||||||
|
return parser;
|
||||||
|
}
|
||||||
|
|
||||||
|
private static void verify_keyword(Publication p_cleaned, String subject) {
|
||||||
|
Optional<Subject> s1 = p_cleaned
|
||||||
|
.getSubject()
|
||||||
|
.stream()
|
||||||
|
.filter(s -> s.getValue().equals(subject))
|
||||||
|
.findFirst();
|
||||||
|
|
||||||
|
assertTrue(s1.isPresent());
|
||||||
|
assertEquals(ModelConstants.DNET_SUBJECT_KEYWORD, s1.get().getQualifier().getClassid());
|
||||||
|
assertEquals(ModelConstants.DNET_SUBJECT_KEYWORD, s1.get().getQualifier().getClassname());
|
||||||
|
}
|
||||||
|
|
||||||
|
private Stream<StructuredProperty> getAuthorPids(Result pub) {
|
||||||
|
return pub
|
||||||
|
.getAuthor()
|
||||||
|
.stream()
|
||||||
|
.map(Author::getPid)
|
||||||
|
.flatMap(Collection::stream);
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -13,7 +13,6 @@ import java.util.stream.Collectors;
|
||||||
import java.util.stream.Stream;
|
import java.util.stream.Stream;
|
||||||
|
|
||||||
import org.apache.commons.io.IOUtils;
|
import org.apache.commons.io.IOUtils;
|
||||||
import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.MappableBlock;
|
|
||||||
import org.junit.jupiter.api.Assertions;
|
import org.junit.jupiter.api.Assertions;
|
||||||
import org.junit.jupiter.api.BeforeEach;
|
import org.junit.jupiter.api.BeforeEach;
|
||||||
import org.junit.jupiter.api.Test;
|
import org.junit.jupiter.api.Test;
|
||||||
|
@ -59,7 +58,7 @@ public class GraphCleaningFunctionsTest {
|
||||||
void testCleanRelations() throws Exception {
|
void testCleanRelations() throws Exception {
|
||||||
|
|
||||||
List<String> lines = IOUtils
|
List<String> lines = IOUtils
|
||||||
.readLines(getClass().getResourceAsStream("/eu/dnetlib/dhp/oa/graph/clean/relation.json"));
|
.readLines(getClass().getResourceAsStream("/eu/dnetlib/dhp/oa/graph/clean/graph/relation/relation.json"));
|
||||||
for (String json : lines) {
|
for (String json : lines) {
|
||||||
Relation r_in = MAPPER.readValue(json, Relation.class);
|
Relation r_in = MAPPER.readValue(json, Relation.class);
|
||||||
assertNotNull(r_in);
|
assertNotNull(r_in);
|
||||||
|
|
File diff suppressed because one or more lines are too long
|
@ -0,0 +1,11 @@
|
||||||
|
<configuration>
|
||||||
|
<appender name="STDOUT" class="ch.qos.logback.core.ConsoleAppender">
|
||||||
|
<encoder>
|
||||||
|
<pattern>%d{HH:mm:ss.SSS} %-5level %logger{36} - %msg%n</pattern>
|
||||||
|
</encoder>
|
||||||
|
</appender>
|
||||||
|
|
||||||
|
<root level="info">
|
||||||
|
<appender-ref ref="STDOUT" />
|
||||||
|
</root>
|
||||||
|
</configuration>
|
Loading…
Reference in New Issue