[stats wf] indicators across stats dbs & updates in the org ids #248

Closed
dimitris.pierrakos wants to merge 1742 commits from beta into beta2master_sept_2022
13 changed files with 243 additions and 129 deletions
Showing only changes of commit 58c05731f9 - Show all commits

View File

@ -47,8 +47,8 @@ public class CleanContextSparkJob implements Serializable {
String inputPath = parser.get("inputPath");
log.info("inputPath: {}", inputPath);
String workingPath = parser.get("workingPath");
log.info("workingPath: {}", workingPath);
String workingDir = parser.get("workingDir");
log.info("workingDir: {}", workingDir);
String contextId = parser.get("contextId");
log.info("contextId: {}", contextId);
@ -67,12 +67,12 @@ public class CleanContextSparkJob implements Serializable {
isSparkSessionManaged,
spark -> {
cleanContext(spark, contextId, verifyParam, inputPath, entityClazz, workingPath);
cleanContext(spark, contextId, verifyParam, inputPath, entityClazz, workingDir);
});
}
private static <T extends Result> void cleanContext(SparkSession spark, String contextId, String verifyParam,
String inputPath, Class<T> entityClazz, String workingPath) {
String inputPath, Class<T> entityClazz, String workingDir) {
Dataset<T> res = spark
.read()
.textFile(inputPath)
@ -106,11 +106,11 @@ public class CleanContextSparkJob implements Serializable {
.write()
.mode(SaveMode.Overwrite)
.option("compression", "gzip")
.json(workingPath);
.json(workingDir);
spark
.read()
.textFile(workingPath)
.textFile(workingDir)
.map(
(MapFunction<String, T>) value -> OBJECT_MAPPER.readValue(value, entityClazz),
Encoders.bean(entityClazz))

View File

@ -3,8 +3,11 @@ package eu.dnetlib.dhp.oa.graph.clean.cfhb;
import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession;
import java.util.Iterator;
import java.util.List;
import java.util.Objects;
import java.util.Optional;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import org.apache.commons.io.IOUtils;
@ -12,8 +15,10 @@ import org.apache.commons.lang3.StringUtils;
import org.apache.spark.SparkConf;
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.*;
import org.apache.spark.sql.expressions.Aggregator;
import org.jetbrains.annotations.NotNull;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -52,14 +57,14 @@ public class CleanCfHbSparkJob {
String inputPath = parser.get("inputPath");
log.info("inputPath: {}", inputPath);
String workingPath = parser.get("workingPath");
log.info("workingPath: {}", workingPath);
String resolvedPath = parser.get("resolvedPath");
log.info("resolvedPath: {}", resolvedPath);
String outputPath = parser.get("outputPath");
log.info("outputPath: {}", outputPath);
String masterDuplicatePath = parser.get("masterDuplicatePath");
log.info("masterDuplicatePath: {}", masterDuplicatePath);
String dsMasterDuplicatePath = parser.get("datasourceMasterDuplicate");
log.info("datasourceMasterDuplicate: {}", dsMasterDuplicatePath);
String graphTableClassName = parser.get("graphTableClassName");
log.info("graphTableClassName: {}", graphTableClassName);
@ -72,12 +77,12 @@ public class CleanCfHbSparkJob {
isSparkSessionManaged,
spark -> {
cleanCfHb(
spark, inputPath, entityClazz, workingPath, masterDuplicatePath, outputPath);
spark, inputPath, entityClazz, resolvedPath, dsMasterDuplicatePath, outputPath);
});
}
private static <T extends Result> void cleanCfHb(SparkSession spark, String inputPath, Class<T> entityClazz,
String workingPath, String masterDuplicatePath, String outputPath) {
String resolvedPath, String masterDuplicatePath, String outputPath) {
// read the master-duplicate tuples
Dataset<MasterDuplicate> md = spark
@ -85,116 +90,94 @@ public class CleanCfHbSparkJob {
.textFile(masterDuplicatePath)
.map(as(MasterDuplicate.class), Encoders.bean(MasterDuplicate.class));
// read the result table
Dataset<T> res = spark
.read()
.textFile(inputPath)
.map(as(entityClazz), Encoders.bean(entityClazz));
// prepare the resolved CF|HB references with the corresponding EMPTY master ID
Dataset<IdCfHbMapping> resolved = res
.flatMap(
(FlatMapFunction<T, IdCfHbMapping>) r -> Stream
.concat(
r.getCollectedfrom().stream().map(KeyValue::getKey),
Stream
Dataset<IdCfHbMapping> resolved = spark
.read()
.textFile(inputPath)
.map(as(entityClazz), Encoders.bean(entityClazz))
.flatMap(
(FlatMapFunction<T, IdCfHbMapping>) r -> {
final List<IdCfHbMapping> list = Stream
.concat(
r.getInstance().stream().map(Instance::getHostedby).map(KeyValue::getKey),
r.getInstance().stream().map(Instance::getCollectedfrom).map(KeyValue::getKey)))
.distinct()
.map(s -> asIdCfHbMapping(r.getId(), s))
.iterator(),
r.getCollectedfrom().stream().map(KeyValue::getKey),
Stream
.concat(
r.getInstance().stream().map(Instance::getHostedby).map(KeyValue::getKey),
r.getInstance().stream().map(Instance::getCollectedfrom).map(KeyValue::getKey)))
.distinct()
.map(s -> asIdCfHbMapping(r.getId(), s))
.collect(Collectors.toList());
return list.iterator();
},
Encoders.bean(IdCfHbMapping.class));
final String resolvedPath = workingPath + "/cfHbResolved";
// set the EMPTY master ID and save it aside
// set the EMPTY master ID/NAME and save it
resolved
.joinWith(md, resolved.col("cfhb").equalTo(md.col("duplicate")))
.joinWith(md, resolved.col("cfhb").equalTo(md.col("duplicateId")))
.map((MapFunction<Tuple2<IdCfHbMapping, MasterDuplicate>, IdCfHbMapping>) t -> {
t._1().setMasterId(t._2().getMasterId());
t._1().setMasterName(t._2().getMasterName());
return t._1();
}, Encoders.bean(IdCfHbMapping.class))
.write()
.mode(SaveMode.Overwrite)
.parquet(resolvedPath);
.json(resolvedPath);
// read again the resolved CF|HB mapping
Dataset<IdCfHbMapping> resolvedDS = spark
.read()
.load(resolvedPath)
.as(Encoders.bean(IdCfHbMapping.class));
.textFile(resolvedPath)
.map(as(IdCfHbMapping.class), Encoders.bean(IdCfHbMapping.class));
// read the result table
Dataset<T> res = spark
.read()
.textFile(inputPath)
.map(as(entityClazz), Encoders.bean(entityClazz));
// Join the results with the resolved CF|HB mapping, apply the mapping and save it
res
.joinWith(resolvedDS, res.col("id").equalTo(resolved.col("resultId")), "left")
.joinWith(resolvedDS, res.col("id").equalTo(resolvedDS.col("resultId")), "left")
.groupByKey((MapFunction<Tuple2<T, IdCfHbMapping>, String>) t -> t._1().getId(), Encoders.STRING())
.agg(new IdCfHbMappingAggregator(entityClazz).toColumn())
.mapGroups(getMapGroupsFunction(), Encoders.bean(entityClazz))
//.agg(new IdCfHbMappingAggregator(entityClazz).toColumn())
.write()
.mode(SaveMode.Overwrite)
.option("compression", "gzip")
.json(outputPath);
}
public static class IdCfHbMappingAggregator<T extends Result> extends Aggregator<IdCfHbMapping, T, T> {
@NotNull
private static <T extends Result> 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) throws Exception {
final Tuple2<T, IdCfHbMapping> first = values.next();
final T res = first._1();
private final Class<T> entityClazz;
public IdCfHbMappingAggregator(Class<T> entityClazz) {
this.entityClazz = entityClazz;
}
@Override
public T zero() {
try {
return entityClazz.newInstance();
} catch (InstantiationException | IllegalAccessException e) {
throw new RuntimeException(e);
updateResult(res, first._2());
values.forEachRemaining(t -> updateResult(res, t._2()));
return res;
}
}
@Override
public T reduce(T r, IdCfHbMapping a) {
if (Objects.isNull(a) && StringUtils.isBlank(a.getMasterId())) {
return r;
private void updateResult(T res, IdCfHbMapping m) {
if (Objects.nonNull(m)) {
res.getCollectedfrom().forEach(kv -> updateKeyValue(kv, m));
res.getInstance().forEach(i -> {
updateKeyValue(i.getHostedby(), m);
updateKeyValue(i.getCollectedfrom(), m);
});
}
}
r.getCollectedfrom().forEach(kv -> updateKeyValue(kv, a));
r.getInstance().forEach(i -> {
updateKeyValue(i.getHostedby(), a);
updateKeyValue(i.getCollectedfrom(), a);
});
return r;
}
@Override
public T merge(T b1, T b2) {
if (Objects.isNull(b1.getId())) {
return b2;
private void updateKeyValue(final KeyValue kv, final IdCfHbMapping a) {
if (kv.getKey().equals(a.getCfhb())) {
kv.setKey(a.getMasterId());
kv.setValue(a.getMasterName());
}
}
return b1;
}
@Override
public T finish(T r) {
return r;
}
private void updateKeyValue(final KeyValue kv, final IdCfHbMapping a) {
if (kv.getKey().equals(a.getCfhb())) {
kv.setKey(a.getMasterId());
kv.setValue(a.getMasterName());
}
}
@Override
public Encoder<T> bufferEncoder() {
return Encoders.bean(entityClazz);
}
@Override
public Encoder<T> outputEncoder() {
return Encoders.bean(entityClazz);
}
};
}
private static IdCfHbMapping asIdCfHbMapping(String resultId, String cfHb) {

View File

@ -5,7 +5,7 @@ import java.io.Serializable;
public class IdCfHbMapping implements Serializable {
private String resultid;
private String resultId;
private String cfhb;
@ -17,15 +17,15 @@ public class IdCfHbMapping implements Serializable {
}
public IdCfHbMapping(String id) {
this.resultid = id;
this.resultId = id;
}
public String getResultid() {
return resultid;
public String getResultId() {
return resultId;
}
public void setResultid(String resultid) {
this.resultid = resultid;
public void setResultId(String resultId) {
this.resultId = resultId;
}
public String getCfhb() {

View File

@ -58,8 +58,8 @@ public class CleanCountrySparkJob implements Serializable {
String inputPath = parser.get("inputPath");
log.info("inputPath: {}", inputPath);
String workingPath = parser.get("workingPath");
log.info("workingPath: {}", workingPath);
String workingDir = parser.get("workingDir");
log.info("workingDir: {}", workingDir);
String datasourcePath = parser.get("hostedBy");
log.info("datasourcePath: {}", datasourcePath);
@ -85,12 +85,12 @@ public class CleanCountrySparkJob implements Serializable {
spark -> {
cleanCountry(
spark, country, verifyParam, inputPath, entityClazz, workingPath, collectedfrom, datasourcePath);
spark, country, verifyParam, inputPath, entityClazz, workingDir, collectedfrom, datasourcePath);
});
}
private static <T extends Result> void cleanCountry(SparkSession spark, String country, String[] verifyParam,
String inputPath, Class<T> entityClazz, String workingPath, String collectedfrom, String datasourcePath) {
String inputPath, Class<T> entityClazz, String workingDir, String collectedfrom, String datasourcePath) {
List<String> hostedBy = spark
.read()
@ -134,11 +134,11 @@ public class CleanCountrySparkJob implements Serializable {
.write()
.mode(SaveMode.Overwrite)
.option("compression", "gzip")
.json(workingPath);
.json(workingDir);
spark
.read()
.textFile(workingPath)
.textFile(workingDir)
.map(
(MapFunction<String, T>) value -> OBJECT_MAPPER.readValue(value, entityClazz),
Encoders.bean(entityClazz))

View File

@ -54,8 +54,8 @@ public class GetDatasourceFromCountry implements Serializable {
String inputPath = parser.get("inputPath");
log.info("inputPath: {}", inputPath);
String workingPath = parser.get("workingPath");
log.info("workingPath: {}", workingPath);
String workingPath = parser.get("workingDir");
log.info("workingDir: {}", workingPath);
String country = parser.get("country");
log.info("country: {}", country);
@ -70,7 +70,7 @@ public class GetDatasourceFromCountry implements Serializable {
}
private static void getDatasourceFromCountry(SparkSession spark, String country, String inputPath,
String workingPath) {
String workingDir) {
Dataset<Organization> organization = spark
.read()
@ -100,7 +100,7 @@ public class GetDatasourceFromCountry implements Serializable {
.write()
.mode(SaveMode.Overwrite)
.option("compression", "gzip")
.json(workingPath);
.json(workingDir);
}
}

View File

@ -343,7 +343,7 @@
</spark-opts>
<arg>--inputPath</arg><arg>${graphOutputPath}/publication</arg>
<arg>--graphTableClassName</arg><arg>eu.dnetlib.dhp.schema.oaf.Publication</arg>
<arg>--workingPath</arg><arg>${workingDir}/working/publication</arg>
<arg>--workingDir</arg><arg>${workingDir}/working/publication</arg>
<arg>--contextId</arg><arg>${contextId}</arg>
<arg>--verifyParam</arg><arg>${verifyParam}</arg>
</spark>
@ -370,7 +370,7 @@
</spark-opts>
<arg>--inputPath</arg><arg>${graphOutputPath}/dataset</arg>
<arg>--graphTableClassName</arg><arg>eu.dnetlib.dhp.schema.oaf.Dataset</arg>
<arg>--workingPath</arg><arg>${workingDir}/working/dataset</arg>
<arg>--workingDir</arg><arg>${workingDir}/working/dataset</arg>
<arg>--contextId</arg><arg>${contextId}</arg>
<arg>--verifyParam</arg><arg>${verifyParam}</arg>
</spark>
@ -397,7 +397,7 @@
</spark-opts>
<arg>--inputPath</arg><arg>${graphOutputPath}/otherresearchproduct</arg>
<arg>--graphTableClassName</arg><arg>eu.dnetlib.dhp.schema.oaf.OtherResearchProduct</arg>
<arg>--workingPath</arg><arg>${workingDir}/working/otherresearchproduct</arg>
<arg>--workingDir</arg><arg>${workingDir}/working/otherresearchproduct</arg>
<arg>--contextId</arg><arg>${contextId}</arg>
<arg>--verifyParam</arg><arg>${verifyParam}</arg>
</spark>
@ -424,7 +424,7 @@
</spark-opts>
<arg>--inputPath</arg><arg>${graphOutputPath}/software</arg>
<arg>--graphTableClassName</arg><arg>eu.dnetlib.dhp.schema.oaf.Software</arg>
<arg>--workingPath</arg><arg>${workingDir}/working/software</arg>
<arg>--workingDir</arg><arg>${workingDir}/working/software</arg>
<arg>--contextId</arg><arg>${contextId}</arg>
<arg>--verifyParam</arg><arg>${verifyParam}</arg>
</spark>
@ -452,7 +452,7 @@
--conf spark.sql.shuffle.partitions=7680
</spark-opts>
<arg>--inputPath</arg><arg>${graphOutputPath}</arg>
<arg>--workingPath</arg><arg>${workingDir}/working/hostedby</arg>
<arg>--workingDir</arg><arg>${workingDir}/working/hostedby</arg>
<arg>--country</arg><arg>${country}</arg>
</spark>
<ok to="fork_clean_country"/>
@ -485,7 +485,7 @@
</spark-opts>
<arg>--inputPath</arg><arg>${graphOutputPath}/publication</arg>
<arg>--graphTableClassName</arg><arg>eu.dnetlib.dhp.schema.oaf.Publication</arg>
<arg>--workingPath</arg><arg>${workingDir}/working/publication</arg>
<arg>--workingDir</arg><arg>${workingDir}/working/publication</arg>
<arg>--country</arg><arg>${country}</arg>
<arg>--verifyParam</arg><arg>${verifyCountryParam}</arg>
<arg>--hostedBy</arg><arg>${workingDir}/working/hostedby</arg>
@ -514,7 +514,7 @@
</spark-opts>
<arg>--inputPath</arg><arg>${graphOutputPath}/dataset</arg>
<arg>--graphTableClassName</arg><arg>eu.dnetlib.dhp.schema.oaf.Dataset</arg>
<arg>--workingPath</arg><arg>${workingDir}/working/dataset</arg>
<arg>--workingDir</arg><arg>${workingDir}/working/dataset</arg>
<arg>--country</arg><arg>${country}</arg>
<arg>--verifyParam</arg><arg>${verifyCountryParam}</arg>
<arg>--hostedBy</arg><arg>${workingDir}/working/hostedby</arg>
@ -543,7 +543,7 @@
</spark-opts>
<arg>--inputPath</arg><arg>${graphOutputPath}/otherresearchproduct</arg>
<arg>--graphTableClassName</arg><arg>eu.dnetlib.dhp.schema.oaf.OtherResearchProduct</arg>
<arg>--workingPath</arg><arg>${workingDir}/working/otherresearchproduct</arg>
<arg>--workingDir</arg><arg>${workingDir}/working/otherresearchproduct</arg>
<arg>--country</arg><arg>${country}</arg>
<arg>--verifyParam</arg><arg>${verifyCountryParam}</arg>
<arg>--hostedBy</arg><arg>${workingDir}/working/hostedby</arg>
@ -572,7 +572,7 @@
</spark-opts>
<arg>--inputPath</arg><arg>${graphOutputPath}/software</arg>
<arg>--graphTableClassName</arg><arg>eu.dnetlib.dhp.schema.oaf.Software</arg>
<arg>--workingPath</arg><arg>${workingDir}/working/software</arg>
<arg>--workingDir</arg><arg>${workingDir}/working/software</arg>
<arg>--country</arg><arg>${country}</arg>
<arg>--verifyParam</arg><arg>${verifyCountryParam}</arg>
<arg>--hostedBy</arg><arg>${workingDir}/working/hostedby</arg>
@ -629,9 +629,9 @@
--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>${workingPath}/cfHbPatched/publication</arg>
<arg>--graphTableClassName</arg><arg>eu.dnetlib.dhp.schema.oaf.Publication</arg>
<arg>--workingDir</arg><arg>${workingDir}/working/publication</arg>
<arg>--masterDuplicatePath</arg><arg>${workingDir}/masterduplicate</arg>
</spark>
<ok to="wait_clean_cfhb"/>
@ -656,9 +656,9 @@
--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>${workingPath}/cfHbPatched/dataset</arg>
<arg>--graphTableClassName</arg><arg>eu.dnetlib.dhp.schema.oaf.Dataset</arg>
<arg>--workingDir</arg><arg>${workingDir}/working/dataset</arg>
<arg>--masterDuplicatePath</arg><arg>${workingDir}/masterduplicate</arg>
</spark>
<ok to="wait_clean_cfhb"/>
@ -683,9 +683,9 @@
--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>${workingPath}/cfHbPatched/otherresearchproduct</arg>
<arg>--graphTableClassName</arg><arg>eu.dnetlib.dhp.schema.oaf.OtherResearchProduct</arg>
<arg>--workingDir</arg><arg>${workingDir}/working/otherresearchproduct</arg>
<arg>--masterDuplicatePath</arg><arg>${workingDir}/masterduplicate</arg>
</spark>
<ok to="wait_clean_cfhb"/>
@ -710,9 +710,9 @@
--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>${workingPath}/cfHbPatched/software</arg>
<arg>--graphTableClassName</arg><arg>eu.dnetlib.dhp.schema.oaf.Software</arg>
<arg>--workingDir</arg><arg>${workingDir}/working/software</arg>
<arg>--masterDuplicatePath</arg><arg>${workingDir}/masterduplicate</arg>
</spark>
<ok to="wait_clean_cfhb"/>
@ -733,7 +733,7 @@
<prepare>
<delete path="${graphOutputPath}/publication"/>
</prepare>
<arg>${workingPath}/cfHbPatched/publication</arg>
<arg>${workingDir}/cfHbPatched/publication</arg>
<arg>${graphOutputPath}/publication</arg>
</distcp>
<ok to="copy_wait"/>
@ -745,7 +745,7 @@
<prepare>
<delete path="${graphOutputPath}/dataset"/>
</prepare>
<arg>${workingPath}/cfHbPatched/dataset</arg>
<arg>${workingDir}/cfHbPatched/dataset</arg>
<arg>${graphOutputPath}/dataset</arg>
</distcp>
<ok to="copy_wait"/>
@ -757,7 +757,7 @@
<prepare>
<delete path="${graphOutputPath}/otherresearchproduct"/>
</prepare>
<arg>${workingPath}/cfHbPatched/otherresearchproduct</arg>
<arg>${workingDir}/cfHbPatched/otherresearchproduct</arg>
<arg>${graphOutputPath}/otherresearchproduct</arg>
</distcp>
<ok to="copy_wait"/>
@ -769,7 +769,7 @@
<prepare>
<delete path="${graphOutputPath}/software"/>
</prepare>
<arg>${workingPath}/cfHbPatched/software</arg>
<arg>${workingDir}/cfHbPatched/software</arg>
<arg>${graphOutputPath}/software</arg>
</distcp>
<ok to="copy_wait"/>

View File

@ -11,6 +11,12 @@
"paramDescription": "the path to the graph data dump to read",
"paramRequired": true
},
{
"paramName": "rp",
"paramLongName": "resolvedPath",
"paramDescription": "the path to store the resolved records",
"paramRequired": true
},
{
"paramName": "out",
"paramLongName": "outputPath",

View File

@ -12,8 +12,8 @@
"paramRequired": true
},
{
"paramName": "wp",
"paramLongName": "workingPath",
"paramName": "wd",
"paramLongName": "workingDir",
"paramDescription": "the path to store the output graph",
"paramRequired": true
},

View File

@ -12,8 +12,8 @@
"paramRequired": true
},
{
"paramName": "wp",
"paramLongName": "workingPath",
"paramName": "wd",
"paramLongName": "workingDir",
"paramDescription": "the path to store the output graph",
"paramRequired": true
},

View File

@ -12,8 +12,8 @@
"paramRequired": true
},
{
"paramName": "wp",
"paramLongName": "workingPath",
"paramName": "wd",
"paramLongName": "workingDir",
"paramDescription": "the path to store the output graph",
"paramRequired": true
},

View File

@ -0,0 +1,118 @@
package eu.dnetlib.dhp.oa.graph.clean.cfhb;
import com.fasterxml.jackson.databind.ObjectMapper;
import eu.dnetlib.dhp.schema.oaf.Dataset;
import eu.dnetlib.dhp.schema.oaf.Publication;
import org.apache.commons.io.FileUtils;
import org.apache.spark.SparkConf;
import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.api.java.function.MapFunction;
import org.apache.spark.sql.Encoders;
import org.apache.spark.sql.SparkSession;
import org.junit.jupiter.api.*;
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;
public class CleanCfHbSparkJobTest {
private static final Logger log = LoggerFactory.getLogger(CleanCfHbSparkJobTest.class);
private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
private static SparkSession spark;
private static Path testBaseTmpPath;
private static String resolvedPath;
private static String graphInputPath;
private static String graphOutputPath;
private static String dsMasterDuplicatePath;
@BeforeAll
public static void beforeAll() throws IOException, URISyntaxException {
testBaseTmpPath = Files.createTempDirectory(CleanCfHbSparkJobTest.class.getSimpleName());
log.info("using test base path {}", testBaseTmpPath);
final File entitiesSources = Paths
.get(CleanCfHbSparkJobTest.class.getResource("/eu/dnetlib/dhp/oa/graph/clean/cfhb/entities").toURI())
.toFile();
FileUtils
.copyDirectory(
entitiesSources,
testBaseTmpPath.resolve("input").resolve("entities").toFile());
FileUtils
.copyFileToDirectory(
Paths
.get(CleanCfHbSparkJobTest.class.getResource("/eu/dnetlib/dhp/oa/graph/clean/cfhb/masterduplicate.json").toURI())
.toFile(),
testBaseTmpPath.resolve("workingDir").resolve("masterduplicate").toFile());
graphInputPath = testBaseTmpPath.resolve("input").resolve("entities").toString();
resolvedPath = testBaseTmpPath.resolve("workingDir").resolve("cfHbResolved").toString();
graphOutputPath = testBaseTmpPath.resolve("workingDir").resolve("cfHbPatched").toString();
dsMasterDuplicatePath = testBaseTmpPath.resolve("workingDir").resolve("masterduplicate").toString();
SparkConf conf = new SparkConf();
conf.setAppName(CleanCfHbSparkJobTest.class.getSimpleName());
conf.setMaster("local[*]");
conf.set("spark.driver.host", "localhost");
conf.set("spark.ui.enabled", "false");
spark = SparkSession
.builder()
.appName(CleanCfHbSparkJobTest.class.getSimpleName())
.config(conf)
.getOrCreate();
}
@AfterAll
public static void afterAll() throws IOException {
FileUtils.deleteDirectory(testBaseTmpPath.toFile());
spark.stop();
}
@Test
void testCleanCfHbSparkJob() throws Exception {
final String outputPath = graphOutputPath + "/dataset";
CleanCfHbSparkJob
.main(
new String[] {
"--isSparkSessionManaged", Boolean.FALSE.toString(),
"--inputPath", graphInputPath + "/dataset",
"--outputPath", outputPath,
"--resolvedPath", resolvedPath + "/dataset",
"--graphTableClassName", Dataset.class.getCanonicalName(),
"--datasourceMasterDuplicate", dsMasterDuplicatePath
});
//final JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext());
Assertions.assertTrue(Files.exists(Paths.get(graphOutputPath, "dataset")));
final org.apache.spark.sql.Dataset<Dataset> d = spark
.read()
.textFile(outputPath)
.map(as(Dataset.class), Encoders.bean(Dataset.class));
Assertions.assertEquals(3, d.count());
}
private static <R> MapFunction<String, R> as(Class<R> clazz) {
return s -> OBJECT_MAPPER.readValue(s, clazz);
}
}

View File

@ -0,0 +1,4 @@
{ "duplicateId" : "10|re3data_____::4c4416659cb74c2e0e891a883a047cbc", "masterId" : "10|fairsharing_::a29d1598024f9e87beab4b98411d48ce", "masterName" : "Bacterial Protein Interaction Database" }
{ "duplicateId" : "10|opendoar____::788b4ac1e172d8e520c2b9461c0a3d35", "masterId" : "10|re3data_____::fc1db64b3964826913b1e9eafe830490", "masterName" : "FULIR Data" }
{ "duplicateId" : "10|re3data_____::6ffd7bc058f762912dc494cd9c175341", "masterId" : "10|fairsharing_::3f647cadf56541fb9513cb63ec370187", "masterName" : "depositar" }
{ "duplicateId" : "10|scindeksserb::07022f78a8cc6d1171092454ecdbb47c", "masterId" : "10|doajarticles::07022f78a8cc6d1171092454ecdbb47c", "masterName" : "Artefact" }