[graph cleaning] WIP: refactoring of the cleaning stages, unit tests

This commit is contained in:
Claudio Atzori 2023-03-21 14:41:20 +01:00
parent 6d3d18d8b5
commit 4f5ba0ed52
9 changed files with 718 additions and 252 deletions

View File

@ -3,15 +3,18 @@ package eu.dnetlib.dhp.oa.graph.clean;
import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession;
import java.util.List;
import java.util.Optional;
import java.util.Set;
import java.util.*;
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.lang3.StringUtils;
import org.apache.spark.SparkConf;
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.MapGroupsFunction;
import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Encoders;
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.common.HdfsSupport;
import eu.dnetlib.dhp.common.action.model.MasterDuplicate;
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.ModelSupport;
import eu.dnetlib.dhp.schema.oaf.KeyValue;
import eu.dnetlib.dhp.schema.oaf.Oaf;
import eu.dnetlib.dhp.schema.oaf.OafEntity;
import eu.dnetlib.dhp.schema.oaf.Result;
import eu.dnetlib.dhp.schema.oaf.utils.GraphCleaningFunctions;
import eu.dnetlib.dhp.utils.ISLookupClientFactory;
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService;
import scala.Tuple2;
public class CleanGraphSparkJob {
@ -40,31 +47,43 @@ public class CleanGraphSparkJob {
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
.toString(
CleanGraphSparkJob.class
.getResourceAsStream(
"/eu/dnetlib/dhp/oa/graph/input_clean_graph_parameters.json"));
.toString(
CleanGraphSparkJob.class
.getResourceAsStream(
"/eu/dnetlib/dhp/oa/graph/input_clean_graph_parameters.json"));
final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration);
parser.parseArgument(args);
Boolean isSparkSessionManaged = Optional
.ofNullable(parser.get("isSparkSessionManaged"))
.map(Boolean::valueOf)
.orElse(Boolean.TRUE);
.ofNullable(parser.get("isSparkSessionManaged"))
.map(Boolean::valueOf)
.orElse(Boolean.TRUE);
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");
log.info("inputPath: {}", inputPath);
String outputPath = parser.get("outputPath");
log.info("outputPath: {}", outputPath);
String isLookupUrl = parser.get("isLookupUrl");
log.info("isLookupUrl: {}", isLookupUrl);
String graphTableClassName = parser.get("graphTableClassName");
log.info("graphTableClassName: {}", graphTableClassName);
@ -80,27 +99,38 @@ public class CleanGraphSparkJob {
String country = parser.get("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);
String collectedfrom = parser.get("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);
final ISLookUpService isLookupService = ISLookupClientFactory.getLookUpService(isLookupUrl);
final VocabularyGroup vocs = VocabularyGroup.loadVocsFromIS(isLookupService);
final VocabularyGroup vocs = VocabularyGroup.loadVocsFromIS(isLookUpService);
SparkConf conf = new SparkConf();
conf.setAppName(CleanGraphSparkJob.class.getSimpleName() + "#" + entityClazz.getSimpleName());
runWithSparkSession(
conf,
isSparkSessionManaged,
spark -> {
HdfsSupport.remove(outputPath, spark.sparkContext().hadoopConfiguration());
cleanGraphTable(
spark, vocs, inputPath, entityClazz, outputPath, contextId, verifyParam, datasourcePath, country,
verifyCountryParam, collectedfrom);
});
conf,
isSparkSessionManaged,
spark -> {
HdfsSupport.remove(outputPath, spark.sparkContext().hadoopConfiguration());
cleanGraphTable(
spark, vocs, inputPath, entityClazz, outputPath, contextId, verifyParam, datasourcePath, country,
verifyCountryParam, collectedfrom, dsMasterDuplicatePath, deepClean);
});
}
private static <T extends Oaf> void cleanGraphTable(
@ -109,33 +139,74 @@ public class CleanGraphSparkJob {
String inputPath,
Class<T> clazz,
String outputPath, String contextId, String verifyParam, String datasourcePath, String country,
String[] verifyCountryParam, String collectedfrom) {
Set<String> hostedBy = Sets
.newHashSet(
spark
.read()
.textFile(datasourcePath)
.collectAsList());
String[] verifyCountryParam, String collectedfrom, String dsMasterDuplicatePath,
Boolean deepClean) {
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>) value -> OafCleaner.apply(value, mapping), Encoders.bean(clazz))
.map((MapFunction<T, T>) value -> GraphCleaningFunctions.cleanup(value, vocs), Encoders.bean(clazz))
.filter((FilterFunction<T>) GraphCleaningFunctions::filter)
.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))
.write()
.mode(SaveMode.Overwrite)
.option("compression", "gzip")
.json(outputPath);
.filter((FilterFunction<T>) GraphCleaningFunctions::filter);
if (Boolean.FALSE.equals(deepClean)) {
cleaned_basic
.write()
.mode(SaveMode.Overwrite)
.option("compression", "gzip")
.json(outputPath);
} else if (Boolean.TRUE.equals(ModelSupport.isSubClass(clazz, Result.class))) {
// 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(
@ -145,9 +216,98 @@ public class CleanGraphSparkJob {
return spark
.read()
.textFile(inputEntityPath)
.map(
(MapFunction<String, T>) value -> OBJECT_MAPPER.readValue(value, clazz),
Encoders.bean(clazz));
.map(as(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());
}
}
};
}
}

View File

@ -83,12 +83,17 @@
</property>
</parameters>
<start to="select_datasourceId_from_country"/>
<start to="prepare_info"/>
<kill name="Kill">
<message>Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}]</message>
</kill>
<fork name="prepare_info">
<path start="select_datasourceId_from_country"/>
<path start="get_ds_master_duplicate"/>
</fork>
<action name="select_datasourceId_from_country">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn</master>
@ -110,10 +115,25 @@
<arg>--workingDir</arg><arg>${workingDir}/working/hostedby</arg>
<arg>--country</arg><arg>${country}</arg>
</spark>
<ok to="fork_clean_graph"/>
<ok to="wait_prepare"/>
<error to="Kill"/>
</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">
<path start="clean_publication"/>
<path start="clean_dataset"/>
@ -152,6 +172,8 @@
<arg>--verifyCountryParam</arg><arg>${verifyCountryParam}</arg>
<arg>--hostedBy</arg><arg>${workingDir}/working/hostedby</arg>
<arg>--collectedfrom</arg><arg>${collectedfrom}</arg>
<arg>--masterDuplicatePath</arg><arg>${workingDir}/masterduplicate</arg>
<arg>--deepClean</arg><arg>${shouldClean}</arg>
</spark>
<ok to="wait_clean"/>
<error to="Kill"/>
@ -184,6 +206,8 @@
<arg>--verifyCountryParam</arg><arg>${verifyCountryParam}</arg>
<arg>--hostedBy</arg><arg>${workingDir}/working/hostedby</arg>
<arg>--collectedfrom</arg><arg>${collectedfrom}</arg>
<arg>--masterDuplicatePath</arg><arg>${workingDir}/masterduplicate</arg>
<arg>--deepClean</arg><arg>${shouldClean}</arg>
</spark>
<ok to="wait_clean"/>
<error to="Kill"/>
@ -216,6 +240,8 @@
<arg>--verifyCountryParam</arg><arg>${verifyCountryParam}</arg>
<arg>--hostedBy</arg><arg>${workingDir}/working/hostedby</arg>
<arg>--collectedfrom</arg><arg>${collectedfrom}</arg>
<arg>--masterDuplicatePath</arg><arg>${workingDir}/masterduplicate</arg>
<arg>--deepClean</arg><arg>${shouldClean}</arg>
</spark>
<ok to="wait_clean"/>
<error to="Kill"/>
@ -248,6 +274,8 @@
<arg>--verifyCountryParam</arg><arg>${verifyCountryParam}</arg>
<arg>--hostedBy</arg><arg>${workingDir}/working/hostedby</arg>
<arg>--collectedfrom</arg><arg>${collectedfrom}</arg>
<arg>--masterDuplicatePath</arg><arg>${workingDir}/masterduplicate</arg>
<arg>--deepClean</arg><arg>${shouldClean}</arg>
</spark>
<ok to="wait_clean"/>
<error to="Kill"/>
@ -280,6 +308,8 @@
<arg>--verifyCountryParam</arg><arg>${verifyCountryParam}</arg>
<arg>--hostedBy</arg><arg>${workingDir}/working/hostedby</arg>
<arg>--collectedfrom</arg><arg>${collectedfrom}</arg>
<arg>--masterDuplicatePath</arg><arg>${workingDir}/masterduplicate</arg>
<arg>--deepClean</arg><arg>${shouldClean}</arg>
</spark>
<ok to="wait_clean"/>
<error to="Kill"/>
@ -312,6 +342,8 @@
<arg>--verifyCountryParam</arg><arg>${verifyCountryParam}</arg>
<arg>--hostedBy</arg><arg>${workingDir}/working/hostedby</arg>
<arg>--collectedfrom</arg><arg>${collectedfrom}</arg>
<arg>--masterDuplicatePath</arg><arg>${workingDir}/masterduplicate</arg>
<arg>--deepClean</arg><arg>${shouldClean}</arg>
</spark>
<ok to="wait_clean"/>
<error to="Kill"/>
@ -344,6 +376,8 @@
<arg>--verifyCountryParam</arg><arg>${verifyCountryParam}</arg>
<arg>--hostedBy</arg><arg>${workingDir}/working/hostedby</arg>
<arg>--collectedfrom</arg><arg>${collectedfrom}</arg>
<arg>--masterDuplicatePath</arg><arg>${workingDir}/masterduplicate</arg>
<arg>--deepClean</arg><arg>${shouldClean}</arg>
</spark>
<ok to="wait_clean"/>
<error to="Kill"/>
@ -376,206 +410,14 @@
<arg>--verifyCountryParam</arg><arg>${verifyCountryParam}</arg>
<arg>--hostedBy</arg><arg>${workingDir}/working/hostedby</arg>
<arg>--collectedfrom</arg><arg>${collectedfrom}</arg>
<arg>--masterDuplicatePath</arg><arg>${workingDir}/masterduplicate</arg>
<arg>--deepClean</arg><arg>${shouldClean}</arg>
</spark>
<ok to="wait_clean"/>
<error to="Kill"/>
</action>
<join name="wait_clean" to="should_patch_datasource_ids"/>
<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"/>
<join name="wait_clean" to="End"/>
<end name="End"/>

View File

@ -33,30 +33,48 @@
"paramName": "ci",
"paramLongName": "contextId",
"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",
"paramLongName": "country",
"paramDescription": "the id of the context to be removed",
"paramRequired": true
"paramRequired": false
},
{
"paramName": "vfc",
"paramLongName": "verifyCountryParam",
"paramDescription": "the parameter to be verified to remove the country",
"paramRequired": true
"paramRequired": false
},
{
"paramName": "cf",
"paramLongName": "collectedfrom",
"paramDescription": "the collectedfrom value for which we should apply the cleaning",
"paramRequired": true
"paramRequired": false
},
{
"paramName": "hb",
"paramLongName": "hostedBy",
"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
}
]

View File

@ -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);
}
}

View File

@ -13,7 +13,6 @@ import java.util.stream.Collectors;
import java.util.stream.Stream;
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.BeforeEach;
import org.junit.jupiter.api.Test;
@ -59,7 +58,7 @@ public class GraphCleaningFunctionsTest {
void testCleanRelations() throws Exception {
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) {
Relation r_in = MAPPER.readValue(json, Relation.class);
assertNotNull(r_in);

View File

@ -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>