Merge branch 'master' into deduptesting

This commit is contained in:
Claudio Atzori 2020-04-18 12:06:58 +02:00
commit 9374ff03ea
24 changed files with 599 additions and 593 deletions

View File

@ -22,6 +22,21 @@ public class HdfsSupport {
private HdfsSupport() {
}
/**
* Checks a path (file or dir) exists on HDFS.
*
* @param path Path to be checked
* @param configuration Configuration of hadoop env
*/
public static boolean exists(String path, Configuration configuration) {
logger.info("Removing path: {}", path);
return rethrowAsRuntimeException(() -> {
Path f = new Path(path);
FileSystem fileSystem = FileSystem.get(configuration);
return fileSystem.exists(f);
});
}
/**
* Removes a path (file or dir) from HDFS.
*

View File

@ -2,7 +2,7 @@ package eu.dnetlib.dhp.schema.oaf;
import java.io.Serializable;
public class StructuredProperty implements Serializable {
public class StructuredProperty implements Serializable {
private String value;

View File

@ -99,6 +99,8 @@ public class PartitionActionSetsByPayloadTypeJob {
List<String> inputActionSetPaths,
String outputPath) {
inputActionSetPaths
.stream()
.filter(path -> HdfsSupport.exists(path, spark.sparkContext().hadoopConfiguration()))
.forEach(inputActionSetPath -> {
Dataset<Row> actionDS = readActionSetFromPath(spark, inputActionSetPath);
saveActions(actionDS, outputPath);

View File

@ -1,12 +1,17 @@
package eu.dnetlib.dhp.oa.dedup;
import com.fasterxml.jackson.databind.DeserializationFeature;
import com.fasterxml.jackson.databind.ObjectMapper;
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
import eu.dnetlib.dhp.common.HdfsSupport;
import eu.dnetlib.dhp.schema.common.ModelSupport;
import eu.dnetlib.dhp.schema.oaf.*;
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException;
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService;
import eu.dnetlib.pace.config.DedupConfig;
import org.apache.spark.SparkConf;
import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.SaveMode;
import org.apache.spark.sql.SparkSession;
import org.dom4j.Document;
import org.dom4j.DocumentException;
@ -21,6 +26,9 @@ import java.util.List;
abstract class AbstractSparkAction implements Serializable {
protected static final ObjectMapper OBJECT_MAPPER = new ObjectMapper()
.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
public ArgumentApplicationParser parser; //parameters for the spark action
public SparkSession spark; //the spark session
@ -68,44 +76,22 @@ abstract class AbstractSparkAction implements Serializable {
abstract void run(ISLookUpService isLookUpService) throws DocumentException, IOException, ISLookUpException;
protected static SparkSession getSparkSession(ArgumentApplicationParser parser) {
SparkConf conf = new SparkConf();
conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer");
conf.registerKryoClasses(new Class[] {
Author.class,
Context.class,
Country.class,
DataInfo.class,
Dataset.class,
Datasource.class,
ExternalReference.class,
ExtraInfo.class,
Field.class,
GeoLocation.class,
Instance.class,
Journal.class,
KeyValue.class,
Oaf.class,
OafEntity.class,
OAIProvenance.class,
Organization.class,
OriginDescription.class,
OtherResearchProduct.class,
Project.class,
Publication.class,
Qualifier.class,
Relation.class,
Result.class,
Software.class,
StructuredProperty.class
});
protected static SparkSession getSparkSession(SparkConf conf) {
return SparkSession
.builder()
.appName(SparkCreateSimRels.class.getSimpleName())
.master(parser.get("master"))
.config(conf)
.getOrCreate();
}
protected static <T> void save(Dataset<T> dataset, String outPath, SaveMode mode) {
dataset
.write()
.option("compression", "gzip")
.mode(mode)
.json(outPath);
}
protected static void removeOutputDir(SparkSession spark, String path) {
HdfsSupport.remove(path, spark.sparkContext().hadoopConfiguration());
}
}

View File

@ -1,274 +1,95 @@
package eu.dnetlib.dhp.oa.dedup;
import com.fasterxml.jackson.databind.DeserializationFeature;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.collect.Lists;
import eu.dnetlib.dhp.schema.common.EntityType;
import eu.dnetlib.dhp.schema.common.ModelSupport;
import eu.dnetlib.dhp.schema.oaf.*;
import eu.dnetlib.pace.config.DedupConfig;
import eu.dnetlib.pace.util.MapDocumentUtil;
import org.apache.spark.api.java.JavaPairRDD;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.api.java.function.FilterFunction;
import org.apache.spark.api.java.function.MapFunction;
import org.apache.spark.api.java.function.MapGroupsFunction;
import org.apache.spark.api.java.function.PairFunction;
import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Encoders;
import org.apache.spark.sql.SparkSession;
import org.codehaus.jackson.map.ObjectMapper;
import scala.Tuple2;
import java.util.Collection;
import java.util.Iterator;
public class DedupRecordFactory {
public static JavaRDD<OafEntity> createDedupRecord(final JavaSparkContext sc, final SparkSession spark, final String mergeRelsInputPath, final String entitiesInputPath, final OafEntityType entityType, final DedupConfig dedupConf) {
protected static final ObjectMapper OBJECT_MAPPER = new com.fasterxml.jackson.databind.ObjectMapper()
.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
public static <T extends OafEntity> Dataset<T> createDedupRecord(
final SparkSession spark, final String mergeRelsInputPath, final String entitiesInputPath, final Class<T> clazz, final DedupConfig dedupConf) {
long ts = System.currentTimeMillis();
//<id, json_entity>
final JavaPairRDD<String, String> inputJsonEntities = sc.textFile(entitiesInputPath)
.mapToPair((PairFunction<String, String, String>) it ->
new Tuple2<String, String>(MapDocumentUtil.getJPathString(dedupConf.getWf().getIdPath(), it), it)
);
Dataset<Tuple2<String, T>> entities = spark.read()
.textFile(entitiesInputPath)
.map((MapFunction<String, Tuple2<String, T>>) it -> {
T entity = OBJECT_MAPPER.readValue(it, clazz);
return new Tuple2<>(entity.getId(), entity);
}, Encoders.tuple(Encoders.STRING(), Encoders.kryo(clazz)));
//<source, target>: source is the dedup_id, target is the id of the mergedIn
JavaPairRDD<String, String> mergeRels = spark
.read().load(mergeRelsInputPath).as(Encoders.bean(Relation.class))
.where("relClass=='merges'")
.javaRDD()
.mapToPair(
(PairFunction<Relation, String, String>) r ->
new Tuple2<String, String>(r.getTarget(), r.getSource())
);
Dataset<Tuple2<String, String>> mergeRels = spark
.read()
.load(mergeRelsInputPath)
.as(Encoders.bean(Relation.class))
.where("relClass == 'merges'")
.map((MapFunction<Relation, Tuple2<String, String>>)
r -> new Tuple2<>(r.getSource(), r.getTarget()), Encoders.tuple(Encoders.STRING(), Encoders.STRING()));
//<dedup_id, json_entity_merged>
final JavaPairRDD<String, String> joinResult = mergeRels.join(inputJsonEntities).mapToPair((PairFunction<Tuple2<String, Tuple2<String, String>>, String, String>) Tuple2::_2);
JavaPairRDD<String, Iterable<String>> sortedJoinResult = joinResult.groupByKey();
switch (entityType) {
case publication:
return sortedJoinResult.map(p -> DedupRecordFactory.publicationMerger(p, ts));
case dataset:
return sortedJoinResult.map(d -> DedupRecordFactory.datasetMerger(d, ts));
case project:
return sortedJoinResult.map(p -> DedupRecordFactory.projectMerger(p, ts));
case software:
return sortedJoinResult.map(s -> DedupRecordFactory.softwareMerger(s, ts));
case datasource:
return sortedJoinResult.map(d -> DedupRecordFactory.datasourceMerger(d, ts));
case organization:
return sortedJoinResult.map(o -> DedupRecordFactory.organizationMerger(o, ts));
case otherresearchproduct:
return sortedJoinResult.map(o -> DedupRecordFactory.otherresearchproductMerger(o, ts));
default:
return null;
}
return mergeRels.joinWith(entities, mergeRels.col("_1").equalTo(entities.col("_1")), "left_outer")
.filter((FilterFunction<Tuple2<Tuple2<String, String>, Tuple2<String, T>>>) value -> value._2() != null)
.map((MapFunction<Tuple2<Tuple2<String, String>, Tuple2<String, T>>, T>)
value -> value._2()._2(), Encoders.kryo(clazz))
.groupByKey((MapFunction<T, String>) value -> value.getId(), Encoders.STRING())
.mapGroups((MapGroupsFunction<String, T, T>)
(key, values) -> entityMerger(key, values, ts, clazz), Encoders.bean(clazz));
}
private static Publication publicationMerger(Tuple2<String, Iterable<String>> e, final long ts) {
private static <T extends OafEntity> T entityMerger(String id, Iterator<T> entities, final long ts, Class<T> clazz) {
try {
T entity = clazz.newInstance();
entity.setId(id);
if (entity.getDataInfo() == null) {
entity.setDataInfo(new DataInfo());
}
entity.getDataInfo().setTrust("0.9");
entity.setLastupdatetimestamp(ts);
Publication p = new Publication(); //the result of the merge, to be returned at the end
final Collection<String> dates = Lists.newArrayList();
entities.forEachRemaining(e -> {
entity.mergeFrom(e);
if (ModelSupport.isSubClass(e, Result.class)) {
Result r1 = (Result) e;
Result er = (Result) entity;
er.setAuthor(DedupUtility.mergeAuthor(er.getAuthor(), r1.getAuthor()));
p.setId(e._1());
final ObjectMapper mapper = new ObjectMapper();
final Collection<String> dateofacceptance = Lists.newArrayList();
if (e._2() != null)
e._2().forEach(pub -> {
try {
Publication publication = mapper.readValue(pub, Publication.class);
p.mergeFrom(publication);
p.setAuthor(DedupUtility.mergeAuthor(p.getAuthor(), publication.getAuthor()));
//add to the list if they are not null
if (publication.getDateofacceptance() != null)
dateofacceptance.add(publication.getDateofacceptance().getValue());
} catch (Exception exc) {
throw new RuntimeException(exc);
}
});
p.setDateofacceptance(DatePicker.pick(dateofacceptance));
if (p.getDataInfo() == null)
p.setDataInfo(new DataInfo());
p.getDataInfo().setTrust("0.9");
p.setLastupdatetimestamp(ts);
return p;
}
private static Dataset datasetMerger(Tuple2<String, Iterable<String>> e, final long ts) {
Dataset d = new Dataset(); //the result of the merge, to be returned at the end
d.setId(e._1());
final ObjectMapper mapper = new ObjectMapper();
final Collection<String> dateofacceptance = Lists.newArrayList();
if (e._2() != null)
e._2().forEach(dat -> {
try {
Dataset dataset = mapper.readValue(dat, Dataset.class);
d.mergeFrom(dataset);
d.setAuthor(DedupUtility.mergeAuthor(d.getAuthor(), dataset.getAuthor()));
//add to the list if they are not null
if (dataset.getDateofacceptance() != null)
dateofacceptance.add(dataset.getDateofacceptance().getValue());
} catch (Exception exc) {
throw new RuntimeException(exc);
}
});
d.setDateofacceptance(DatePicker.pick(dateofacceptance));
if (d.getDataInfo() == null)
d.setDataInfo(new DataInfo());
d.getDataInfo().setTrust("0.9");
d.setLastupdatetimestamp(ts);
return d;
}
private static Project projectMerger(Tuple2<String, Iterable<String>> e, final long ts) {
Project p = new Project(); //the result of the merge, to be returned at the end
p.setId(e._1());
final ObjectMapper mapper = new ObjectMapper();
if (e._2() != null)
e._2().forEach(proj -> {
try {
Project project = mapper.readValue(proj, Project.class);
p.mergeFrom(project);
} catch (Exception exc) {
throw new RuntimeException(exc);
}
});
if (p.getDataInfo() == null)
p.setDataInfo(new DataInfo());
p.getDataInfo().setTrust("0.9");
p.setLastupdatetimestamp(ts);
return p;
}
private static Software softwareMerger(Tuple2<String, Iterable<String>> e, final long ts) {
Software s = new Software(); //the result of the merge, to be returned at the end
s.setId(e._1());
final ObjectMapper mapper = new ObjectMapper();
final Collection<String> dateofacceptance = Lists.newArrayList();
if (e._2() != null)
e._2().forEach(soft -> {
try {
Software software = mapper.readValue(soft, Software.class);
s.mergeFrom(software);
s.setAuthor(DedupUtility.mergeAuthor(s.getAuthor(), software.getAuthor()));
//add to the list if they are not null
if (software.getDateofacceptance() != null)
dateofacceptance.add(software.getDateofacceptance().getValue());
} catch (Exception exc) {
throw new RuntimeException(exc);
}
});
s.setDateofacceptance(DatePicker.pick(dateofacceptance));
if (s.getDataInfo() == null)
s.setDataInfo(new DataInfo());
s.getDataInfo().setTrust("0.9");
s.setLastupdatetimestamp(ts);
return s;
}
private static Datasource datasourceMerger(Tuple2<String, Iterable<String>> e, final long ts) {
Datasource d = new Datasource(); //the result of the merge, to be returned at the end
d.setId(e._1());
final ObjectMapper mapper = new ObjectMapper();
if (e._2() != null)
e._2().forEach(dat -> {
try {
Datasource datasource = mapper.readValue(dat, Datasource.class);
d.mergeFrom(datasource);
} catch (Exception exc) {
throw new RuntimeException(exc);
}
});
if (d.getDataInfo() == null)
d.setDataInfo(new DataInfo());
d.getDataInfo().setTrust("0.9");
d.setLastupdatetimestamp(ts);
return d;
}
private static Organization organizationMerger(Tuple2<String, Iterable<String>> e, final long ts) {
Organization o = new Organization(); //the result of the merge, to be returned at the end
o.setId(e._1());
final ObjectMapper mapper = new ObjectMapper();
StringBuilder trust = new StringBuilder("0.0");
if (e._2() != null)
e._2().forEach(pub -> {
try {
Organization organization = mapper.readValue(pub, Organization.class);
final String currentTrust = organization.getDataInfo().getTrust();
if (!"1.0".equals(currentTrust)) {
trust.setLength(0);
trust.append(currentTrust);
if (er.getDateofacceptance() != null) {
dates.add(r1.getDateofacceptance().getValue());
}
o.mergeFrom(organization);
} catch (Exception exc) {
throw new RuntimeException(exc);
}
});
if (o.getDataInfo() == null)
{
o.setDataInfo(new DataInfo());
if (ModelSupport.isSubClass(entity, Result.class)) {
((Result) entity).setDateofacceptance(DatePicker.pick(dates));
}
return entity;
} catch (IllegalAccessException | InstantiationException e) {
throw new RuntimeException(e);
}
if (o.getDataInfo() == null)
o.setDataInfo(new DataInfo());
o.getDataInfo().setTrust("0.9");
o.setLastupdatetimestamp(ts);
return o;
}
private static OtherResearchProduct otherresearchproductMerger(Tuple2<String, Iterable<String>> e, final long ts) {
OtherResearchProduct o = new OtherResearchProduct(); //the result of the merge, to be returned at the end
o.setId(e._1());
final ObjectMapper mapper = new ObjectMapper();
final Collection<String> dateofacceptance = Lists.newArrayList();
if (e._2() != null)
e._2().forEach(orp -> {
try {
OtherResearchProduct otherResearchProduct = mapper.readValue(orp, OtherResearchProduct.class);
o.mergeFrom(otherResearchProduct);
o.setAuthor(DedupUtility.mergeAuthor(o.getAuthor(), otherResearchProduct.getAuthor()));
//add to the list if they are not null
if (otherResearchProduct.getDateofacceptance() != null)
dateofacceptance.add(otherResearchProduct.getDateofacceptance().getValue());
} catch (Exception exc) {
throw new RuntimeException(exc);
}
});
if (o.getDataInfo() == null)
o.setDataInfo(new DataInfo());
o.setDateofacceptance(DatePicker.pick(dateofacceptance));
o.getDataInfo().setTrust("0.9");
o.setLastupdatetimestamp(ts);
return o;
}
}

View File

@ -1,74 +1,50 @@
package eu.dnetlib.dhp.oa.dedup;
import eu.dnetlib.dhp.oa.dedup.model.Block;
import eu.dnetlib.pace.config.DedupConfig;
import eu.dnetlib.pace.model.MapDocument;
import eu.dnetlib.pace.util.BlockProcessor;
import eu.dnetlib.pace.util.MapDocumentUtil;
import org.apache.spark.api.java.JavaPairRDD;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.api.java.function.Function2;
import org.apache.spark.api.java.function.PairFlatMapFunction;
import org.apache.spark.api.java.function.PairFunction;
import org.apache.spark.util.LongAccumulator;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import scala.Serializable;
import scala.Tuple2;
import java.util.ArrayList;
import java.util.Comparator;
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;
public class Deduper implements Serializable {
private static final Logger log = LoggerFactory.getLogger(Deduper.class);
public static JavaPairRDD<String, String> computeRelations(JavaSparkContext context, JavaPairRDD<String, List<MapDocument>> blocks, DedupConfig config) {
public static JavaPairRDD<String, String> computeRelations(JavaSparkContext context, JavaPairRDD<String, Block> blocks, DedupConfig config) {
Map<String, LongAccumulator> accumulators = DedupUtility.constructAccumulator(config, context.sc());
return blocks.flatMapToPair((PairFlatMapFunction<Tuple2<String, List<MapDocument>>, String, String>) it -> {
try {
final SparkReporter reporter = new SparkReporter(accumulators);
new BlockProcessor(config).processSortedBlock(it._1(), it._2(), reporter);
return reporter.getRelations().iterator();
} catch (Exception e) {
throw new RuntimeException(it._2().get(0).getIdentifier(), e);
}
}).mapToPair(
(PairFunction<Tuple2<String, String>, String, Tuple2<String, String>>) item ->
new Tuple2<String, Tuple2<String, String>>(item._1() + item._2(), item))
return blocks
.flatMapToPair(it -> {
final SparkReporter reporter = new SparkReporter(accumulators);
new BlockProcessor(config).processSortedBlock(it._1(), it._2().getDocuments(), reporter);
return reporter.getRelations().iterator();
})
.mapToPair(it -> new Tuple2<>(it._1() + it._2(), it))
.reduceByKey((a, b) -> a)
.mapToPair((PairFunction<Tuple2<String, Tuple2<String, String>>, String, String>) Tuple2::_2);
.mapToPair(Tuple2::_2);
}
public static JavaPairRDD<String, List<MapDocument>> createSortedBlocks(JavaSparkContext context, JavaPairRDD<String, MapDocument> mapDocs, DedupConfig config) {
public static JavaPairRDD<String, Block> createSortedBlocks(JavaPairRDD<String, MapDocument> mapDocs, DedupConfig config) {
final String of = config.getWf().getOrderField();
final int maxQueueSize = config.getWf().getGroupMaxSize();
return mapDocs
//the reduce is just to be sure that we haven't document with same id
.reduceByKey((a, b) -> a)
.map(Tuple2::_2)
//Clustering: from <id, doc> to List<groupkey,doc>
.flatMapToPair((PairFlatMapFunction<MapDocument, String, List<MapDocument>>) a ->
DedupUtility.getGroupingKeys(config, a)
.flatMap(a -> DedupUtility.getGroupingKeys(config, a)
.stream()
.map(it -> {
List<MapDocument> tmp = new ArrayList<>();
tmp.add(a);
return new Tuple2<>(it, tmp);
}
)
.map(it -> Block.from(it, a))
.collect(Collectors.toList())
.iterator())
.reduceByKey((Function2<List<MapDocument>, List<MapDocument>, List<MapDocument>>) (v1, v2) -> {
v1.addAll(v2);
v1.sort(Comparator.comparing(a -> a.getFieldMap().get(of).stringValue()));
if (v1.size() > maxQueueSize)
return new ArrayList<>(v1.subList(0, maxQueueSize));
return v1;
});
.mapToPair(block -> new Tuple2<>(block.getKey(), block))
.reduceByKey((b1, b2) -> Block.from(b1, b2, of, maxQueueSize));
}
}

View File

@ -1,15 +0,0 @@
package eu.dnetlib.dhp.oa.dedup;
public enum OafEntityType {
datasource,
organization,
project,
dataset,
otherresearchproduct,
software,
publication
}

View File

@ -1,15 +1,21 @@
package eu.dnetlib.dhp.oa.dedup;
import com.fasterxml.jackson.databind.ObjectMapper;
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
import eu.dnetlib.dhp.schema.common.EntityType;
import eu.dnetlib.dhp.schema.common.ModelSupport;
import eu.dnetlib.dhp.schema.oaf.OafEntity;
import eu.dnetlib.dhp.utils.ISLookupClientFactory;
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException;
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService;
import eu.dnetlib.pace.config.DedupConfig;
import org.apache.commons.io.IOUtils;
import org.apache.spark.SparkConf;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.api.java.function.MapFunction;
import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Encoders;
import org.apache.spark.sql.SaveMode;
import org.apache.spark.sql.SparkSession;
import org.dom4j.DocumentException;
import org.slf4j.Logger;
@ -31,7 +37,12 @@ public class SparkCreateDedupRecord extends AbstractSparkAction {
SparkCreateSimRels.class.getResourceAsStream("/eu/dnetlib/dhp/oa/dedup/createDedupRecord_parameters.json")));
parser.parseArgument(args);
new SparkCreateDedupRecord(parser, getSparkSession(parser)).run(ISLookupClientFactory.getLookUpService(parser.get("isLookUpUrl")));
SparkConf conf = new SparkConf();
conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer");
conf.registerKryoClasses(ModelSupport.getOafModelClasses());
new SparkCreateDedupRecord(parser, getSparkSession(conf))
.run(ISLookupClientFactory.getLookUpService(parser.get("isLookUpUrl")));
}
@Override
@ -42,26 +53,28 @@ public class SparkCreateDedupRecord extends AbstractSparkAction {
final String actionSetId = parser.get("actionSetId");
final String workingPath = parser.get("workingPath");
System.out.println(String.format("graphBasePath: '%s'", graphBasePath));
System.out.println(String.format("isLookUpUrl: '%s'", isLookUpUrl));
System.out.println(String.format("actionSetId: '%s'", actionSetId));
System.out.println(String.format("workingPath: '%s'", workingPath));
final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext());
log.info("graphBasePath: '{}'", graphBasePath);
log.info("isLookUpUrl: '{}'", isLookUpUrl);
log.info("actionSetId: '{}'", actionSetId);
log.info("workingPath: '{}'", workingPath);
for (DedupConfig dedupConf: getConfigurations(isLookUpService, actionSetId)) {
String subEntity = dedupConf.getWf().getSubEntityValue();
System.out.println(String.format("Creating deduprecords for: '%s'", subEntity));
log.info("Creating deduprecords for: '{}'", subEntity);
final String outputPath = DedupUtility.createDedupRecordPath(workingPath, actionSetId, subEntity);
removeOutputDir(spark, outputPath);
final String mergeRelPath = DedupUtility.createMergeRelPath(workingPath, actionSetId, subEntity);
final String entityPath = DedupUtility.createEntityPath(graphBasePath, subEntity);
final OafEntityType entityType = OafEntityType.valueOf(subEntity);
final JavaRDD<OafEntity> dedupRecord =
DedupRecordFactory.createDedupRecord(sc, spark, mergeRelPath, entityPath, entityType, dedupConf);
dedupRecord.map(r -> {
ObjectMapper mapper = new ObjectMapper();
return mapper.writeValueAsString(r);
}).saveAsTextFile(DedupUtility.createDedupRecordPath(workingPath, actionSetId, subEntity));
Class<OafEntity> clazz = ModelSupport.entityTypes.get(EntityType.valueOf(subEntity));
DedupRecordFactory.createDedupRecord(spark, mergeRelPath, entityPath, clazz, dedupConf)
.map((MapFunction<OafEntity, String>) value -> OBJECT_MAPPER.writeValueAsString(value), Encoders.STRING())
.write()
.mode(SaveMode.Overwrite)
.json(outputPath);
}
}

View File

@ -4,6 +4,9 @@ import com.google.common.hash.Hashing;
import eu.dnetlib.dhp.oa.dedup.graph.ConnectedComponent;
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
import eu.dnetlib.dhp.oa.dedup.graph.GraphProcessor;
import eu.dnetlib.dhp.schema.common.ModelSupport;
import eu.dnetlib.dhp.schema.oaf.DataInfo;
import eu.dnetlib.dhp.schema.oaf.Qualifier;
import eu.dnetlib.dhp.schema.oaf.Relation;
import eu.dnetlib.dhp.utils.ISLookupClientFactory;
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException;
@ -11,14 +14,15 @@ import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService;
import eu.dnetlib.pace.config.DedupConfig;
import eu.dnetlib.pace.util.MapDocumentUtil;
import org.apache.commons.io.IOUtils;
import org.apache.spark.SparkConf;
import org.apache.spark.api.java.JavaPairRDD;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.api.java.function.PairFunction;
import org.apache.spark.graphx.Edge;
import org.apache.spark.rdd.RDD;
import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Encoders;
import org.apache.spark.sql.SaveMode;
import org.apache.spark.sql.SparkSession;
import org.dom4j.DocumentException;
import scala.Tuple2;
@ -32,7 +36,9 @@ import java.util.List;
public class SparkCreateMergeRels extends AbstractSparkAction {
public static final String PROVENANCE_ACTION_CLASS = "sysimport:dedup";
private static final Logger log = LoggerFactory.getLogger(SparkCreateMergeRels.class);
public static final String DNET_PROVENANCE_ACTIONS = "dnet:provenanceActions";
public SparkCreateMergeRels(ArgumentApplicationParser parser, SparkSession spark) {
super(parser, spark);
@ -44,7 +50,14 @@ public class SparkCreateMergeRels extends AbstractSparkAction {
SparkCreateSimRels.class.getResourceAsStream("/eu/dnetlib/dhp/oa/dedup/createCC_parameters.json")));
parser.parseArgument(args);
new SparkCreateMergeRels(parser, getSparkSession(parser)).run(ISLookupClientFactory.getLookUpService(parser.get("isLookUpUrl")));
final String isLookUpUrl = parser.get("isLookUpUrl");
log.info("isLookupUrl {}", isLookUpUrl);
SparkConf conf = new SparkConf();
conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer");
conf.registerKryoClasses(ModelSupport.getOafModelClasses());
new SparkCreateMergeRels(parser, getSparkSession(conf)).run(ISLookupClientFactory.getLookUpService(isLookUpUrl));
}
@Override
@ -55,56 +68,90 @@ public class SparkCreateMergeRels extends AbstractSparkAction {
final String isLookUpUrl = parser.get("isLookUpUrl");
final String actionSetId = parser.get("actionSetId");
System.out.println(String.format("graphBasePath: '%s'", graphBasePath));
System.out.println(String.format("isLookUpUrl: '%s'", isLookUpUrl));
System.out.println(String.format("actionSetId: '%s'", actionSetId));
System.out.println(String.format("workingPath: '%s'", workingPath));
log.info("graphBasePath: '{}'", graphBasePath);
log.info("isLookUpUrl: '{}'", isLookUpUrl);
log.info("actionSetId: '{}'", actionSetId);
log.info("workingPath: '{}'", workingPath);
final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext());
final JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext());
for (DedupConfig dedupConf: getConfigurations(isLookUpService, actionSetId)) {
final String subEntity = dedupConf.getWf().getSubEntityValue();
System.out.println(String.format("Creating mergerels for: '%s'", subEntity));
log.info("Creating mergerels for: '{}'", subEntity);
final int maxIterations = dedupConf.getWf().getMaxIterations();
log.info("Max iterations {}", maxIterations);
final String mergeRelPath = DedupUtility.createMergeRelPath(workingPath, actionSetId, subEntity);
final JavaPairRDD<Object, String> vertexes = sc.textFile(graphBasePath + "/" + subEntity)
.map(s -> MapDocumentUtil.getJPathString(dedupConf.getWf().getIdPath(), s))
.mapToPair((PairFunction<String, Object, String>)
s -> new Tuple2<Object, String>(getHashcode(s), s)
);
.mapToPair((PairFunction<String, Object, String>) s -> new Tuple2<>(hash(s), s));
final Dataset<Relation> similarityRelations = spark.read().load(DedupUtility.createSimRelPath(workingPath, actionSetId, subEntity)).as(Encoders.bean(Relation.class));
final RDD<Edge<String>> edgeRdd = similarityRelations.javaRDD().map(it -> new Edge<>(getHashcode(it.getSource()), getHashcode(it.getTarget()), it.getRelClass())).rdd();
final JavaRDD<ConnectedComponent> cc = GraphProcessor.findCCs(vertexes.rdd(), edgeRdd, dedupConf.getWf().getMaxIterations()).toJavaRDD();
final Dataset<Relation> mergeRelation = spark.createDataset(cc.filter(k -> k.getDocIds().size() > 1)
.flatMap(this::ccToMergeRel).rdd(), Encoders.bean(Relation.class));
final RDD<Edge<String>> edgeRdd = spark
.read()
.load(DedupUtility.createSimRelPath(workingPath, actionSetId, subEntity))
.as(Encoders.bean(Relation.class))
.javaRDD()
.map(it -> new Edge<>(hash(it.getSource()), hash(it.getTarget()), it.getRelClass()))
.rdd();
mergeRelation
.write().mode("overwrite")
.save(DedupUtility.createMergeRelPath(workingPath, actionSetId, subEntity));
final Dataset<Relation> mergeRels = spark
.createDataset(GraphProcessor.findCCs(vertexes.rdd(), edgeRdd, maxIterations)
.toJavaRDD()
.filter(k -> k.getDocIds().size() > 1)
.flatMap(cc -> ccToMergeRel(cc, dedupConf))
.rdd(), Encoders.bean(Relation.class));
mergeRels
.write()
.mode(SaveMode.Append)
.parquet(mergeRelPath);
}
}
public Iterator<Relation> ccToMergeRel(ConnectedComponent cc){
public Iterator<Relation> ccToMergeRel(ConnectedComponent cc, DedupConfig dedupConf){
return cc.getDocIds()
.stream()
.flatMap(id -> {
List<Relation> tmp = new ArrayList<>();
Relation r = new Relation();
r.setSource(cc.getCcId());
r.setTarget(id);
r.setRelClass("merges");
tmp.add(r);
r = new Relation();
r.setTarget(cc.getCcId());
r.setSource(id);
r.setRelClass("isMergedIn");
tmp.add(r);
tmp.add(rel(cc.getCcId(), id, "merges", dedupConf));
tmp.add(rel(id, cc.getCcId(), "isMergedIn", dedupConf));
return tmp.stream();
}).iterator();
}
public static long getHashcode(final String id) {
private Relation rel(String source, String target, String relClass, DedupConfig dedupConf) {
Relation r = new Relation();
r.setSource(source);
r.setTarget(target);
r.setRelClass(relClass);
r.setSubRelType("dedup");
DataInfo info = new DataInfo();
info.setDeletedbyinference(false);
info.setInferred(true);
info.setInvisible(false);
info.setInferenceprovenance(dedupConf.getWf().getConfigurationId());
Qualifier provenanceAction = new Qualifier();
provenanceAction.setClassid(PROVENANCE_ACTION_CLASS);
provenanceAction.setClassname(PROVENANCE_ACTION_CLASS);
provenanceAction.setSchemeid(DNET_PROVENANCE_ACTIONS);
provenanceAction.setSchemename(DNET_PROVENANCE_ACTIONS);
info.setProvenanceaction(provenanceAction);
//TODO calculate the trust value based on the similarity score of the elements in the CC
//info.setTrust();
r.setDataInfo(info);
return r;
}
public static long hash(final String id) {
return Hashing.murmur3_128().hashString(id).asLong();
}

View File

@ -1,23 +1,28 @@
package eu.dnetlib.dhp.oa.dedup;
import com.fasterxml.jackson.core.JsonProcessingException;
import com.fasterxml.jackson.databind.ObjectMapper;
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
import eu.dnetlib.dhp.schema.action.AtomicAction;
import eu.dnetlib.dhp.oa.dedup.model.Block;
import eu.dnetlib.dhp.schema.common.ModelSupport;
import eu.dnetlib.dhp.schema.oaf.DataInfo;
import eu.dnetlib.dhp.schema.oaf.Relation;
import eu.dnetlib.dhp.utils.ISLookupClientFactory;
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException;
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService;
import eu.dnetlib.pace.config.DedupConfig;
import eu.dnetlib.pace.model.FieldListImpl;
import eu.dnetlib.pace.model.FieldValueImpl;
import eu.dnetlib.pace.model.MapDocument;
import eu.dnetlib.pace.util.MapDocumentUtil;
import org.apache.commons.io.IOUtils;
import org.apache.hadoop.io.Text;
import org.apache.spark.SparkConf;
import org.apache.spark.api.java.JavaPairRDD;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.api.java.function.MapFunction;
import org.apache.spark.api.java.function.PairFunction;
import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Encoders;
import org.apache.spark.sql.SaveMode;
import org.apache.spark.sql.SparkSession;
import org.dom4j.DocumentException;
import org.slf4j.Logger;
@ -25,7 +30,6 @@ import org.slf4j.LoggerFactory;
import scala.Tuple2;
import java.io.IOException;
import java.util.List;
public class SparkCreateSimRels extends AbstractSparkAction {
@ -41,7 +45,17 @@ public class SparkCreateSimRels extends AbstractSparkAction {
SparkCreateSimRels.class.getResourceAsStream("/eu/dnetlib/dhp/oa/dedup/createSimRels_parameters.json")));
parser.parseArgument(args);
new SparkCreateSimRels(parser, getSparkSession(parser)).run(ISLookupClientFactory.getLookUpService(parser.get("isLookUpUrl")));
SparkConf conf = new SparkConf();
conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer");
conf.registerKryoClasses(new Class[] {
MapDocument.class,
FieldListImpl.class,
FieldValueImpl.class,
Block.class
});
new SparkCreateSimRels(parser, getSparkSession(conf))
.run(ISLookupClientFactory.getLookUpService(parser.get("isLookUpUrl")));
}
@Override
@ -53,77 +67,63 @@ public class SparkCreateSimRels extends AbstractSparkAction {
final String actionSetId = parser.get("actionSetId");
final String workingPath = parser.get("workingPath");
System.out.println(String.format("graphBasePath: '%s'", graphBasePath));
System.out.println(String.format("isLookUpUrl: '%s'", isLookUpUrl));
System.out.println(String.format("actionSetId: '%s'", actionSetId));
System.out.println(String.format("workingPath: '%s'", workingPath));
final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext());
log.info("graphBasePath: '{}'", graphBasePath);
log.info("isLookUpUrl: '{}'", isLookUpUrl);
log.info("actionSetId: '{}'", actionSetId);
log.info("workingPath: '{}'", workingPath);
//for each dedup configuration
for (DedupConfig dedupConf: getConfigurations(isLookUpService, actionSetId)) {
final String entity = dedupConf.getWf().getEntityType();
final String subEntity = dedupConf.getWf().getSubEntityValue();
System.out.println(String.format("Creating simrels for: '%s'", subEntity));
log.info("Creating simrels for: '{}'", subEntity);
JavaPairRDD<String, MapDocument> mapDocument = sc.textFile(DedupUtility.createEntityPath(graphBasePath, subEntity))
.mapToPair((PairFunction<String, String, MapDocument>) s -> {
final String outputPath = DedupUtility.createSimRelPath(workingPath, actionSetId, subEntity);
removeOutputDir(spark, outputPath);
JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext());
JavaPairRDD<String, MapDocument> mapDocuments = sc.textFile(DedupUtility.createEntityPath(graphBasePath, subEntity))
.mapToPair((PairFunction<String, String, MapDocument>) s -> {
MapDocument d = MapDocumentUtil.asMapDocumentWithJPath(dedupConf, s);
return new Tuple2<String, MapDocument>(d.getIdentifier(), d);
return new Tuple2<>(d.getIdentifier(), d);
});
//create blocks for deduplication
JavaPairRDD<String, List<MapDocument>> blocks = Deduper.createSortedBlocks(sc, mapDocument, dedupConf);
JavaPairRDD<String, Block> blocks = Deduper.createSortedBlocks(mapDocuments, dedupConf);
//create relations by comparing only elements in the same group
final JavaPairRDD<String, String> dedupRels = Deduper.computeRelations(sc, blocks, dedupConf);
JavaRDD<Relation> relationsRDD = dedupRels.map(r -> createSimRel(r._1(), r._2(), entity));
JavaRDD<Relation> relations = Deduper.computeRelations(sc, blocks, dedupConf)
.map(t -> createSimRel(t._1(), t._2(), entity));
//save the simrel in the workingdir
spark.createDataset(relationsRDD.rdd(), Encoders.bean(Relation.class))
spark.createDataset(relations.rdd(), Encoders.bean(Relation.class))
.write()
.mode("overwrite")
.save(DedupUtility.createSimRelPath(workingPath, actionSetId, subEntity));
.mode(SaveMode.Append)
.save(outputPath);
}
}
/**
* Utility method used to create an atomic action from a Relation object
* @param relation input relation
* @return A tuple2 with [id, json serialization of the atomic action]
* @throws JsonProcessingException
*/
public Tuple2<Text, Text> createSequenceFileRow(Relation relation) throws JsonProcessingException {
ObjectMapper mapper = new ObjectMapper();
String id = relation.getSource() + "@" + relation.getRelClass() + "@" + relation.getTarget();
AtomicAction<Relation> aa = new AtomicAction<>(Relation.class, relation);
return new Tuple2<>(
new Text(id),
new Text(mapper.writeValueAsString(aa))
);
}
public Relation createSimRel(String source, String target, String entity){
private Relation createSimRel(String source, String target, String entity) {
final Relation r = new Relation();
r.setSource(source);
r.setTarget(target);
r.setSubRelType("dedupSimilarity");
r.setRelClass("isSimilarTo");
r.setDataInfo(new DataInfo());
switch(entity){
switch(entity) {
case "result":
r.setRelClass("resultResult_dedupSimilarity_isSimilarTo");
r.setRelType("resultResult");
break;
case "organization":
r.setRelClass("organizationOrganization_dedupSimilarity_isSimilarTo");
r.setRelType("organizationOrganization");
break;
default:
r.setRelClass("isSimilarTo");
break;
throw new IllegalArgumentException("unmanaged entity type: " + entity);
}
return r;
}
}

View File

@ -3,6 +3,7 @@ package eu.dnetlib.dhp.oa.dedup;
import com.fasterxml.jackson.databind.DeserializationFeature;
import com.fasterxml.jackson.databind.ObjectMapper;
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
import eu.dnetlib.dhp.schema.common.ModelSupport;
import eu.dnetlib.dhp.schema.oaf.DataInfo;
import eu.dnetlib.dhp.schema.oaf.Relation;
import eu.dnetlib.dhp.utils.ISLookupClientFactory;
@ -10,11 +11,9 @@ import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService;
import org.apache.commons.io.IOUtils;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.spark.SparkConf;
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.Row;
import org.apache.spark.sql.SparkSession;
import org.apache.spark.sql.*;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import scala.Tuple2;
@ -27,9 +26,6 @@ public class SparkPropagateRelation extends AbstractSparkAction {
private static final Logger log = LoggerFactory.getLogger(SparkPropagateRelation.class);
private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper()
.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
enum FieldType {
SOURCE,
TARGET
@ -46,7 +42,11 @@ public class SparkPropagateRelation extends AbstractSparkAction {
parser.parseArgument(args);
new SparkPropagateRelation(parser, getSparkSession(parser))
SparkConf conf = new SparkConf();
conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer");
conf.registerKryoClasses(ModelSupport.getOafModelClasses());
new SparkPropagateRelation(parser, getSparkSession(conf))
.run(ISLookupClientFactory.getLookUpService(parser.get("isLookUpUrl")));
}
@ -62,7 +62,7 @@ public class SparkPropagateRelation extends AbstractSparkAction {
log.info("dedupGraphPath: '{}'", dedupGraphPath);
final String outputRelationPath = DedupUtility.createEntityPath(dedupGraphPath, "relation");
deletePath(outputRelationPath);
removeOutputDir(spark, outputRelationPath);
Dataset<Relation> mergeRels = spark.read()
.load(DedupUtility.createMergeRelPath(workingPath, "*", "*"))
@ -95,7 +95,7 @@ public class SparkPropagateRelation extends AbstractSparkAction {
processDataset(rels, mergedIds, FieldType.SOURCE, getDeletedFn()),
mergedIds, FieldType.TARGET, getDeletedFn());
save(newRels.union(updated), outputRelationPath);
save(newRels.union(updated), outputRelationPath, SaveMode.Overwrite);
}
@ -169,26 +169,6 @@ public class SparkPropagateRelation extends AbstractSparkAction {
};
}
private void deletePath(String path) {
try {
Path p = new Path(path);
FileSystem fs = FileSystem.get(spark.sparkContext().hadoopConfiguration());
if (fs.exists(p)) {
fs.delete(p, true);
}
} catch (IOException e) {
throw new RuntimeException(e);
}
}
private static void save(Dataset<Relation> dataset, String outPath) {
dataset
.write()
.option("compression", "gzip")
.json(outPath);
}
private static boolean containsDedup(final Relation r) {
return r.getSource().toLowerCase().contains("dedup") || r.getTarget().toLowerCase().contains("dedup");
}

View File

@ -13,9 +13,9 @@ import java.util.Map;
public class SparkReporter implements Serializable, Reporter {
final List<Tuple2<String, String>> relations = new ArrayList<>();
private static final Log log = LogFactory.getLog(SparkReporter.class);
Map<String, LongAccumulator> accumulators;
private final List<Tuple2<String, String>> relations = new ArrayList<>();
private Map<String, LongAccumulator> accumulators;
public SparkReporter(Map<String, LongAccumulator> accumulators){
this.accumulators = accumulators;

View File

@ -1,17 +1,23 @@
package eu.dnetlib.dhp.oa.dedup;
import com.fasterxml.jackson.databind.DeserializationFeature;
import com.fasterxml.jackson.databind.ObjectMapper;
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
import eu.dnetlib.dhp.schema.oaf.*;
import eu.dnetlib.dhp.oa.dedup.model.Block;
import eu.dnetlib.dhp.schema.common.ModelSupport;
import eu.dnetlib.dhp.schema.oaf.DataInfo;
import eu.dnetlib.dhp.schema.oaf.Oaf;
import eu.dnetlib.dhp.schema.oaf.OafEntity;
import eu.dnetlib.dhp.schema.oaf.Relation;
import eu.dnetlib.dhp.utils.ISLookupClientFactory;
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService;
import eu.dnetlib.pace.model.FieldListImpl;
import eu.dnetlib.pace.model.FieldValueImpl;
import eu.dnetlib.pace.model.MapDocument;
import eu.dnetlib.pace.util.MapDocumentUtil;
import org.apache.commons.io.IOUtils;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.*;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.compress.GzipCodec;
import org.apache.spark.SparkConf;
import org.apache.spark.api.java.JavaPairRDD;
@ -27,7 +33,6 @@ import org.slf4j.LoggerFactory;
import scala.Tuple2;
import java.io.IOException;
import java.io.Serializable;
public class SparkUpdateEntity extends AbstractSparkAction {
@ -42,27 +47,16 @@ public class SparkUpdateEntity extends AbstractSparkAction {
public static void main(String[] args) throws Exception {
ArgumentApplicationParser parser = new ArgumentApplicationParser(
IOUtils.toString(
SparkUpdateEntity.class.getResourceAsStream("/eu/dnetlib/dhp/oa/dedup/updateEntity_parameters.json")));
SparkUpdateEntity.class.getResourceAsStream(
"/eu/dnetlib/dhp/oa/dedup/updateEntity_parameters.json")));
parser.parseArgument(args);
new SparkUpdateEntity(parser, getSparkSession(parser)).run(ISLookupClientFactory.getLookUpService(parser.get("isLookUpUrl")));
}
SparkConf conf = new SparkConf();
conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer");
conf.registerKryoClasses(ModelSupport.getOafModelClasses());
public boolean mergeRelExists(String basePath, String entity) throws IOException {
boolean result = false;
FileSystem fileSystem = FileSystem.get(new Configuration());
FileStatus[] fileStatuses = fileSystem.listStatus(new Path(basePath));
for (FileStatus fs : fileStatuses) {
if (fs.isDirectory())
if (fileSystem.exists(new Path(DedupUtility.createMergeRelPath(basePath, fs.getPath().getName(), entity))))
result = true;
}
return result;
new SparkUpdateEntity(parser, getSparkSession(conf))
.run(ISLookupClientFactory.getLookUpService(parser.get("isLookUpUrl")));
}
public void run(ISLookUpService isLookUpService) throws IOException {
@ -71,75 +65,81 @@ public class SparkUpdateEntity extends AbstractSparkAction {
final String workingPath = parser.get("workingPath");
final String dedupGraphPath = parser.get("dedupGraphPath");
System.out.println(String.format("graphBasePath: '%s'", graphBasePath));
System.out.println(String.format("workingPath: '%s'", workingPath));
System.out.println(String.format("dedupGraphPath: '%s'", dedupGraphPath));
log.info("graphBasePath: '{}'", graphBasePath);
log.info("workingPath: '{}'", workingPath);
log.info("dedupGraphPath: '{}'", dedupGraphPath);
final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext());
//for each entity
for (OafEntityType entity: OafEntityType.values()) {
ModelSupport.entityTypes.forEach((entity, clazz) -> {
final String outputPath = dedupGraphPath + "/" + entity;
removeOutputDir(spark, outputPath);
JavaRDD<String> sourceEntity = sc.textFile(DedupUtility.createEntityPath(graphBasePath, entity.toString()));
if (mergeRelExists(workingPath, entity.toString())) {
final Dataset<Relation> rel = spark.read().load(DedupUtility.createMergeRelPath(workingPath, "*", entity.toString())).as(Encoders.bean(Relation.class));
final String mergeRelPath = DedupUtility.createMergeRelPath(workingPath, "*", entity.toString());
final String dedupRecordPath = DedupUtility.createDedupRecordPath(workingPath, "*", entity.toString());
final Dataset<Relation> rel = spark.read()
.load(mergeRelPath)
.as(Encoders.bean(Relation.class));
final JavaPairRDD<String, String> mergedIds = rel
.where("relClass == 'merges'")
.select(rel.col("target"))
.distinct()
.toJavaRDD()
.mapToPair((PairFunction<Row, String, String>) r -> new Tuple2<String, String>(r.getString(0), "d"));
.mapToPair((PairFunction<Row, String, String>) r -> new Tuple2<>(r.getString(0), "d"));
final JavaRDD<String> dedupEntity = sc.textFile(DedupUtility.createDedupRecordPath(workingPath, "*", entity.toString()));
JavaPairRDD<String, String> entitiesWithId = sourceEntity
.mapToPair((PairFunction<String, String, String>) s -> new Tuple2<>(MapDocumentUtil.getJPathString(IDJSONPATH, s), s));
JavaPairRDD<String, String> entitiesWithId = sourceEntity.mapToPair((PairFunction<String, String, String>) s -> new Tuple2<String, String>(MapDocumentUtil.getJPathString(IDJSONPATH, s), s));
JavaRDD<String> map = entitiesWithId.leftOuterJoin(mergedIds).map(k -> k._2()._2().isPresent() ? updateDeletedByInference(k._2()._1(), getOafClass(entity)) : k._2()._1());
sourceEntity = map.union(dedupEntity);
JavaRDD<String> map = entitiesWithId
.leftOuterJoin(mergedIds)
.map(k -> k._2()._2().isPresent() ? updateDeletedByInference(k._2()._1(), clazz) : k._2()._1());
sourceEntity = map.union(sc.textFile(dedupRecordPath));
}
sourceEntity.saveAsTextFile(dedupGraphPath + "/" + entity, GzipCodec.class);
sourceEntity.saveAsTextFile(outputPath, GzipCodec.class);
}
});
}
public Class<? extends Oaf> getOafClass(OafEntityType className) {
switch (className.toString()) {
case "publication":
return Publication.class;
case "dataset":
return eu.dnetlib.dhp.schema.oaf.Dataset.class;
case "datasource":
return Datasource.class;
case "software":
return Software.class;
case "organization":
return Organization.class;
case "otherresearchproduct":
return OtherResearchProduct.class;
case "project":
return Project.class;
default:
throw new IllegalArgumentException("Illegal type " + className);
}
}
public boolean mergeRelExists(String basePath, String entity) {
private static <T extends Oaf> String updateDeletedByInference(final String json, final Class<T> clazz) {
final ObjectMapper mapper = new ObjectMapper();
mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
boolean result = false;
try {
Oaf entity = mapper.readValue(json, clazz);
FileSystem fileSystem = FileSystem.get(new Configuration());
FileStatus[] fileStatuses = fileSystem.listStatus(new Path(basePath));
for (FileStatus fs : fileStatuses) {
if (fs.isDirectory())
if (fileSystem.exists(new Path(DedupUtility.createMergeRelPath(basePath, fs.getPath().getName(), entity))))
result = true;
}
return result;
} catch (IOException e) {
throw new RuntimeException(e);
}
}
private static <T extends OafEntity> String updateDeletedByInference(final String json, final Class<T> clazz) {
try {
Oaf entity = OBJECT_MAPPER.readValue(json, clazz);
if (entity.getDataInfo()== null)
entity.setDataInfo(new DataInfo());
entity.getDataInfo().setDeletedbyinference(true);
return mapper.writeValueAsString(entity);
return OBJECT_MAPPER.writeValueAsString(entity);
} catch (IOException e) {
throw new RuntimeException("Unable to convert json", e);
}
}
}

View File

@ -0,0 +1,76 @@
package eu.dnetlib.dhp.oa.dedup.model;
import com.google.common.collect.Lists;
import eu.dnetlib.pace.model.MapDocument;
import java.io.Serializable;
import java.util.ArrayList;
import java.util.Comparator;
import java.util.Iterator;
import java.util.List;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import java.util.stream.StreamSupport;
public class Block implements Serializable {
private String key;
private List<MapDocument> documents;
public Block() {
super();
}
public static Block from(String key, MapDocument doc) {
Block block = new Block();
block.setKey(key);
block.setDocuments(Lists.newArrayList(doc));
return block;
}
public static Block from(String key, Iterator<Block> blocks, String orderField, int maxSize) {
Block block = new Block();
block.setKey(key);
Iterable<Block> it = () -> blocks;
block.setDocuments(
StreamSupport.stream(it.spliterator(), false)
.flatMap(b -> b.getDocuments().stream())
.sorted(Comparator.comparing(a -> a.getFieldMap().get(orderField).stringValue()))
.limit(maxSize)
.collect(Collectors.toCollection(ArrayList::new)));
return block;
}
public static Block from(Block b1, Block b2, String orderField, int maxSize) {
Block block = new Block();
block.setKey(b1.getKey());
block.setDocuments(
Stream.concat(
b1.getDocuments().stream(),
b2.getDocuments().stream())
.sorted(Comparator.comparing(a -> a.getFieldMap().get(orderField).stringValue()))
.limit(maxSize)
.collect(Collectors.toCollection(ArrayList::new)));
return block;
}
public String getKey() {
return key;
}
public void setKey(String key) {
this.key = key;
}
public List<MapDocument> getDocuments() {
return documents;
}
public void setDocuments(List<MapDocument> documents) {
this.documents = documents;
}
}

View File

@ -68,38 +68,12 @@
</configuration>
</global>
<start to="UpdateEntity"/>
<start to="PropagateRelation"/>
<kill name="Kill">
<message>Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}]</message>
</kill>
<action name="UpdateEntity">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn</master>
<mode>cluster</mode>
<name>Update Entity</name>
<class>eu.dnetlib.dhp.oa.dedup.SparkUpdateEntity</class>
<jar>dhp-dedup-openaire-${projectVersion}.jar</jar>
<spark-opts>
--executor-memory=${sparkExecutorMemory}
--executor-cores=${sparkExecutorCores}
--driver-memory=${sparkDriverMemory}
--conf spark.extraListeners=${spark2ExtraListeners}
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
--conf spark.sql.shuffle.partitions=3840
</spark-opts>
<arg>-mt</arg><arg>yarn</arg>
<arg>--i</arg><arg>${graphBasePath}</arg>
<arg>--w</arg><arg>${workingPath}</arg>
<arg>--o</arg><arg>${dedupGraphPath}</arg>
</spark>
<ok to="PropagateRelation"/>
<error to="Kill"/>
</action>
<action name="PropagateRelation">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn</master>
@ -117,14 +91,116 @@
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
--conf spark.sql.shuffle.partitions=7680
</spark-opts>
<arg>-mt</arg><arg>yarn</arg>
<arg>--i</arg><arg>${graphBasePath}</arg>
<arg>--o</arg><arg>${dedupGraphPath}</arg>
<arg>--w</arg><arg>${workingPath}</arg>
</spark>
<ok to="End"/>
<ok to="fork_copy_entities"/>
<error to="Kill"/>
</action>
<fork name="fork_copy_entities">
<path start="copy_datasource"/>
<path start="copy_project"/>
<path start="copy_organization"/>
<path start="copy_publication"/>
<path start="copy_dataset"/>
<path start="copy_software"/>
<path start="copy_otherresearchproduct"/>
</fork>
<action name="copy_datasource">
<distcp xmlns="uri:oozie:distcp-action:0.2">
<prepare>
<delete path="${dedupGraphPath}/datasource"/>
</prepare>
<arg>-pb</arg>
<arg>${graphBasePath}/datasource</arg>
<arg>${dedupGraphPath}/datasource</arg>
</distcp>
<ok to="wait_copy"/>
<error to="Kill"/>
</action>
<action name="copy_project">
<distcp xmlns="uri:oozie:distcp-action:0.2">
<prepare>
<delete path="${dedupGraphPath}/project"/>
</prepare>
<arg>-pb</arg>
<arg>${graphBasePath}/project</arg>
<arg>${dedupGraphPath}/project</arg>
</distcp>
<ok to="wait_copy"/>
<error to="Kill"/>
</action>
<action name="copy_organization">
<distcp xmlns="uri:oozie:distcp-action:0.2">
<prepare>
<delete path="${dedupGraphPath}/organization"/>
</prepare>
<arg>-pb</arg>
<arg>${graphBasePath}/organization</arg>
<arg>${dedupGraphPath}/organization</arg>
</distcp>
<ok to="wait_copy"/>
<error to="Kill"/>
</action>
<action name="copy_publication">
<distcp xmlns="uri:oozie:distcp-action:0.2">
<prepare>
<delete path="${dedupGraphPath}/publication"/>
</prepare>
<arg>-pb</arg>
<arg>${graphBasePath}/publication</arg>
<arg>${dedupGraphPath}/publication</arg>
</distcp>
<ok to="wait_copy"/>
<error to="Kill"/>
</action>
<action name="copy_dataset">
<distcp xmlns="uri:oozie:distcp-action:0.2">
<prepare>
<delete path="${dedupGraphPath}/dataset"/>
</prepare>
<arg>-pb</arg>
<arg>${graphBasePath}/dataset</arg>
<arg>${dedupGraphPath}/dataset</arg>
</distcp>
<ok to="wait_copy"/>
<error to="Kill"/>
</action>
<action name="copy_software">
<distcp xmlns="uri:oozie:distcp-action:0.2">
<prepare>
<delete path="${dedupGraphPath}/software"/>
</prepare>
<arg>-pb</arg>
<arg>${graphBasePath}/software</arg>
<arg>${dedupGraphPath}/software</arg>
</distcp>
<ok to="wait_copy"/>
<error to="Kill"/>
</action>
<action name="copy_otherresearchproduct">
<distcp xmlns="uri:oozie:distcp-action:0.2">
<prepare>
<delete path="${dedupGraphPath}/otherresearchproduct"/>
</prepare>
<arg>-pb</arg>
<arg>${graphBasePath}/otherresearchproduct</arg>
<arg>${dedupGraphPath}/otherresearchproduct</arg>
</distcp>
<ok to="wait_copy"/>
<error to="Kill"/>
</action>
<join name="wait_copy" to="End"/>
<end name="End"/>
</workflow-app>

View File

@ -1,10 +1,4 @@
[
{
"paramName": "mt",
"paramLongName": "master",
"paramDescription": "should be local or yarn",
"paramRequired": true
},
{
"paramName": "asi",
"paramLongName": "actionSetId",

View File

@ -1,10 +1,4 @@
[
{
"paramName": "mt",
"paramLongName": "master",
"paramDescription": "should be local or yarn",
"paramRequired": true
},
{
"paramName": "i",
"paramLongName": "graphBasePath",

View File

@ -1,10 +1,4 @@
[
{
"paramName": "mt",
"paramLongName": "master",
"paramDescription": "should be local or yarn",
"paramRequired": true
},
{
"paramName": "la",
"paramLongName": "isLookUpUrl",

View File

@ -1,10 +1,4 @@
[
{
"paramName": "mt",
"paramLongName": "master",
"paramDescription": "should be local or yarn",
"paramRequired": true
},
{
"paramName": "i",
"paramLongName": "graphBasePath",

View File

@ -16,6 +16,10 @@
<name>workingPath</name>
<description>path for the working directory</description>
</property>
<property>
<name>dedupGraphPath</name>
<description>path for the output graph</description>
</property>
<property>
<name>sparkDriverMemory</name>
<description>memory for driver process</description>
@ -94,7 +98,6 @@
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
--conf spark.sql.shuffle.partitions=3840
</spark-opts>
<arg>-mt</arg><arg>yarn</arg>
<arg>--i</arg><arg>${graphBasePath}</arg>
<arg>--la</arg><arg>${isLookUpUrl}</arg>
<arg>--asi</arg><arg>${actionSetId}</arg>
@ -121,7 +124,6 @@
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
--conf spark.sql.shuffle.partitions=3840
</spark-opts>
<arg>-mt</arg><arg>yarn</arg>
<arg>--i</arg><arg>${graphBasePath}</arg>
<arg>--w</arg><arg>${workingPath}</arg>
<arg>--la</arg><arg>${isLookUpUrl}</arg>
@ -146,13 +148,51 @@
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
--conf spark.sql.shuffle.partitions=3840
</spark-opts>
<arg>-mt</arg><arg>yarn</arg>
<arg>--i</arg><arg>${graphBasePath}</arg>
<arg>--w</arg><arg>${workingPath}</arg>
<arg>--la</arg><arg>${isLookUpUrl}</arg>
<arg>--asi</arg><arg>${actionSetId}</arg>
</spark>
<ok to="UpdateEntity"/>
<error to="Kill"/>
</action>
<action name="UpdateEntity">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn</master>
<mode>cluster</mode>
<name>Update Entity</name>
<class>eu.dnetlib.dhp.oa.dedup.SparkUpdateEntity</class>
<jar>dhp-dedup-openaire-${projectVersion}.jar</jar>
<spark-opts>
--executor-memory=${sparkExecutorMemory}
--executor-cores=${sparkExecutorCores}
--driver-memory=${sparkDriverMemory}
--conf spark.extraListeners=${spark2ExtraListeners}
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
--conf spark.sql.shuffle.partitions=3840
</spark-opts>
<arg>--i</arg><arg>${graphBasePath}</arg>
<arg>--w</arg><arg>${workingPath}</arg>
<arg>--o</arg><arg>${dedupGraphPath}</arg>
</spark>
<ok to="copyRelations"/>
<error to="Kill"/>
</action>
<action name="copyRelations">
<distcp xmlns="uri:oozie:distcp-action:0.2">
<prepare>
<delete path="${dedupGraphPath}/relation"/>
</prepare>
<arg>-pb</arg>
<arg>${graphBasePath}/relation</arg>
<arg>${dedupGraphPath}/relation</arg>
</distcp>
<ok to="End"/>
<error to="Kill"/>
</action>

View File

@ -1,10 +1,4 @@
[
{
"paramName": "mt",
"paramLongName": "master",
"paramDescription": "should be local or yarn",
"paramRequired": true
},
{
"paramName": "i",
"paramLongName": "graphBasePath",

View File

@ -34,8 +34,13 @@ public class Datacite2Scholix {
ScholixResource resource = generateDataciteScholixResource(dJson);
return relIds.stream().flatMap(s-> {
final List<Scholix> result = generateScholix(resource, s.get("relatedIdentifier"), s.get("relatedIdentifierType"), s.get("relationType"), updated);
return result.stream();
try {
final List<Scholix> result = generateScholix(resource, ""+s.get("relatedIdentifier"), s.get("relatedIdentifierType"), s.get("relationType"), updated);
return result.stream();
} catch (Throwable e)
{
return new ArrayList<Scholix>().stream();
}
}).collect(Collectors.toList());
}
@ -48,6 +53,7 @@ public class Datacite2Scholix {
}
private List<Scholix> generateScholix(ScholixResource source, final String pid, final String pidtype, final String relType, final String updated) {
if ("doi".equalsIgnoreCase(pidtype)) {
ScholixResource target = new ScholixResource();
target.setIdentifier(Collections.singletonList(new ScholixIdentifier(pid, pidtype)));

View File

@ -7,4 +7,8 @@
<name>oozie.action.sharelib.for.spark</name>
<value>spark2</value>
</property>
<property>
<name>oozie.launcher.mapreduce.user.classpath.first</name>
<value>true</value>
</property>
</configuration>

View File

@ -3,16 +3,12 @@ package eu.dnetlib.dhp.provision;
import com.fasterxml.jackson.databind.ObjectMapper;
import eu.dnetlib.dhp.provision.scholix.Scholix;
import eu.dnetlib.dhp.provision.scholix.ScholixResource;
import eu.dnetlib.dhp.provision.update.CrossrefClient;
import eu.dnetlib.dhp.provision.update.Datacite2Scholix;
import eu.dnetlib.dhp.provision.update.DataciteClient;
import eu.dnetlib.dhp.provision.update.DataciteClientIterator;
import eu.dnetlib.dhp.provision.update.*;
import eu.dnetlib.scholexplorer.relation.RelationMapper;
import org.apache.commons.io.IOUtils;
import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.Test;
import java.io.IOException;
import java.util.List;
@ -27,6 +23,19 @@ public class DataciteClientTest {
System.out.println(new ObjectMapper().writeValueAsString(s));
}
// public void testS() throws Exception {
// RetrieveUpdateFromDatacite.main(new String[]{
// "-n", "file:///data/new_s2.txt",
// "-t", "/data/new_s2.txt",
// "-ts", "1586974078",
// "-ih", "ip-90-147-167-25.ct1.garrservices.it",
// "-in", "datacite",
// });
//
// }
public void testResolveDataset() throws Exception {
DataciteClient dc = new DataciteClient("ip-90-147-167-25.ct1.garrservices.it");
ScholixResource datasetByDOI = dc.getDatasetByDOI("10.17182/hepdata.15392.v1/t5");