Added First Implementation of Spark Test
This commit is contained in:
parent
e65ecfcc3a
commit
d0edb7b773
|
@ -15,15 +15,76 @@
|
|||
<version>1.0-SNAPSHOT</version>
|
||||
|
||||
<dependencies>
|
||||
<!--<dependency>-->
|
||||
<!--<groupId>eu.dnetlib</groupId>-->
|
||||
<!--<artifactId>dnet-openaireplus-mapping-utils</artifactId>-->
|
||||
<!--<version>6.2.9</version>-->
|
||||
<!--<exclusions>-->
|
||||
<!--<exclusion>-->
|
||||
<!--<groupId>org.apache.hadoop</groupId>-->
|
||||
<!--<artifactId>hadoop-commons</artifactId>-->
|
||||
<!--</exclusion>-->
|
||||
<!--<exclusion>-->
|
||||
<!--<groupId>org.apache.hadoop</groupId>-->
|
||||
<!--<artifactId>hadoop-hdfs</artifactId>-->
|
||||
<!--</exclusion>-->
|
||||
<!--<exclusion>-->
|
||||
<!--<groupId>org.apache.hadoop</groupId>-->
|
||||
<!--<artifactId>hadoop-core</artifactId>-->
|
||||
<!--</exclusion>-->
|
||||
|
||||
<!--<exclusion>-->
|
||||
<!--<groupId>com.google.guava</groupId>-->
|
||||
<!--<artifactId>guava</artifactId>-->
|
||||
<!--</exclusion>-->
|
||||
|
||||
<!--</exclusions>-->
|
||||
<!--</dependency>-->
|
||||
|
||||
<dependency>
|
||||
<groupId>eu.dnetlib</groupId>
|
||||
<artifactId>dnet-openaire-data-protos</artifactId>
|
||||
<version>3.9.3-proto250</version>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>eu.dnetlib</groupId>
|
||||
<artifactId>dnet-pace-core</artifactId>
|
||||
<version>2.6.8-SNAPSHOT</version>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>org.apache.spark</groupId>
|
||||
<artifactId>spark-core_2.11</artifactId>
|
||||
<version>${spark.version}</version>
|
||||
</dependency>
|
||||
|
||||
|
||||
<!--<dependency>-->
|
||||
<!--<groupId>org.mongodb.spark</groupId>-->
|
||||
<!--<artifactId>mongo-spark-connector_2.11</artifactId>-->
|
||||
<!--<version>2.0.0</version>-->
|
||||
<!--</dependency>-->
|
||||
<!--<dependency>-->
|
||||
<!--<groupId>org.apache.spark</groupId>-->
|
||||
<!--<artifactId>spark-sql_2.11</artifactId>-->
|
||||
<!--<version>${spark.version}</version>-->
|
||||
<!--<!–<scope>provided</scope>–>-->
|
||||
<!--</dependency>-->
|
||||
<!--<dependency>-->
|
||||
<!--<groupId>org.apache.spark</groupId>-->
|
||||
<!--<artifactId>spark-mllib_2.11</artifactId>-->
|
||||
<!--<version>${spark.version}</version>-->
|
||||
<!--<!–<scope>provided</scope>–>-->
|
||||
<!--</dependency>-->
|
||||
|
||||
|
||||
|
||||
</dependencies>
|
||||
|
||||
<properties>
|
||||
<spark.version>2.2.0</spark.version>
|
||||
</properties>
|
||||
|
||||
|
||||
</project>
|
|
@ -0,0 +1,190 @@
|
|||
package eu.dnetlib;
|
||||
|
||||
import com.google.common.collect.Lists;
|
||||
import eu.dnetlib.pace.clustering.NGramUtils;
|
||||
import eu.dnetlib.pace.config.DedupConfig;
|
||||
import eu.dnetlib.pace.config.WfConfig;
|
||||
import eu.dnetlib.pace.distance.PaceDocumentDistance;
|
||||
import eu.dnetlib.pace.distance.eval.ScoreResult;
|
||||
import eu.dnetlib.pace.model.Field;
|
||||
import eu.dnetlib.pace.model.MapDocument;
|
||||
import eu.dnetlib.pace.model.MapDocumentComparator;
|
||||
import org.apache.commons.lang.StringUtils;
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import scala.Tuple2;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.*;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
public class BlockProcessor {
|
||||
|
||||
private static final Log log = LogFactory.getLog(BlockProcessor.class);
|
||||
|
||||
private DedupConfig dedupConf;
|
||||
|
||||
public BlockProcessor(DedupConfig dedupConf) {
|
||||
this.dedupConf = dedupConf;
|
||||
}
|
||||
|
||||
public List<Tuple2<String, String>> process(final String key, final Stream<MapDocument> documents, final Reporter context) throws IOException, InterruptedException {
|
||||
|
||||
final Queue<MapDocument> q = prepare(documents);
|
||||
|
||||
if (q.size() > 1) {
|
||||
log.info("reducing key: '" + key + "' records: " + q.size());
|
||||
//process(q, context);
|
||||
return process(simplifyQueue(q, key, context), context);
|
||||
} else {
|
||||
context.incrementCounter(dedupConf.getWf().getEntityType(), "records per hash key = 1", 1);
|
||||
return new ArrayList<>();
|
||||
}
|
||||
}
|
||||
|
||||
private Queue<MapDocument> prepare(final Stream<MapDocument> documents) {
|
||||
final Queue<MapDocument> queue = new PriorityQueue<MapDocument>(100, new MapDocumentComparator(dedupConf.getWf().getOrderField()));
|
||||
|
||||
final Set<String> seen = new HashSet<String>();
|
||||
final int queueMaxSize = dedupConf.getWf().getQueueMaxSize();
|
||||
|
||||
documents.forEach(doc -> {
|
||||
if (queue.size() <= queueMaxSize) {
|
||||
final String id = doc.getIdentifier();
|
||||
|
||||
if (!seen.contains(id)) {
|
||||
seen.add(id);
|
||||
queue.add(doc);
|
||||
}
|
||||
}
|
||||
});
|
||||
|
||||
return queue;
|
||||
}
|
||||
|
||||
private Queue<MapDocument> simplifyQueue(final Queue<MapDocument> queue, final String ngram, final Reporter context) {
|
||||
final Queue<MapDocument> q = new LinkedList<MapDocument>();
|
||||
|
||||
String fieldRef = "";
|
||||
final List<MapDocument> tempResults = Lists.newArrayList();
|
||||
|
||||
while (!queue.isEmpty()) {
|
||||
final MapDocument result = queue.remove();
|
||||
|
||||
final String orderFieldName = dedupConf.getWf().getOrderField();
|
||||
final Field orderFieldValue = result.values(orderFieldName);
|
||||
if (!orderFieldValue.isEmpty()) {
|
||||
final String field = NGramUtils.cleanupForOrdering(orderFieldValue.stringValue());
|
||||
if (field.equals(fieldRef)) {
|
||||
tempResults.add(result);
|
||||
} else {
|
||||
populateSimplifiedQueue(q, tempResults, context, fieldRef, ngram);
|
||||
tempResults.clear();
|
||||
tempResults.add(result);
|
||||
fieldRef = field;
|
||||
}
|
||||
} else {
|
||||
context.incrementCounter(dedupConf.getWf().getEntityType(), "missing " + dedupConf.getWf().getOrderField(), 1);
|
||||
}
|
||||
}
|
||||
populateSimplifiedQueue(q, tempResults, context, fieldRef, ngram);
|
||||
|
||||
return q;
|
||||
}
|
||||
|
||||
private void populateSimplifiedQueue(final Queue<MapDocument> q,
|
||||
final List<MapDocument> tempResults,
|
||||
final Reporter context,
|
||||
final String fieldRef,
|
||||
final String ngram) {
|
||||
WfConfig wf = dedupConf.getWf();
|
||||
if (tempResults.size() < wf.getGroupMaxSize()) {
|
||||
q.addAll(tempResults);
|
||||
} else {
|
||||
context.incrementCounter(wf.getEntityType(), String.format("Skipped records for count(%s) >= %s", wf.getOrderField(), wf.getGroupMaxSize()), tempResults.size());
|
||||
log.info("Skipped field: " + fieldRef + " - size: " + tempResults.size() + " - ngram: " + ngram);
|
||||
}
|
||||
}
|
||||
|
||||
private List<Tuple2<String, String>> process(final Queue<MapDocument> queue, final Reporter context) throws IOException, InterruptedException {
|
||||
|
||||
final PaceDocumentDistance algo = new PaceDocumentDistance();
|
||||
List<Tuple2<String, String>> resultEmit = new ArrayList<>();
|
||||
|
||||
while (!queue.isEmpty()) {
|
||||
|
||||
final MapDocument pivot = queue.remove();
|
||||
final String idPivot = pivot.getIdentifier();
|
||||
|
||||
WfConfig wf = dedupConf.getWf();
|
||||
final Field fieldsPivot = pivot.values(wf.getOrderField());
|
||||
final String fieldPivot = (fieldsPivot == null) || fieldsPivot.isEmpty() ? null : fieldsPivot.stringValue();
|
||||
|
||||
if (fieldPivot != null) {
|
||||
// System.out.println(idPivot + " --> " + fieldPivot);
|
||||
|
||||
int i = 0;
|
||||
for (final MapDocument curr : queue) {
|
||||
final String idCurr = curr.getIdentifier();
|
||||
|
||||
if (mustSkip(idCurr)) {
|
||||
|
||||
context.incrementCounter(wf.getEntityType(), "skip list", 1);
|
||||
|
||||
break;
|
||||
}
|
||||
|
||||
if (i > wf.getSlidingWindowSize()) {
|
||||
break;
|
||||
}
|
||||
|
||||
final Field fieldsCurr = curr.values(wf.getOrderField());
|
||||
final String fieldCurr = (fieldsCurr == null) || fieldsCurr.isEmpty() ? null : fieldsCurr.stringValue();
|
||||
|
||||
if (!idCurr.equals(idPivot) && (fieldCurr != null)) {
|
||||
|
||||
final ScoreResult sr = similarity(algo, pivot, curr);
|
||||
emitOutput(sr, idPivot, idCurr,context, resultEmit);
|
||||
i++;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
return resultEmit;
|
||||
}
|
||||
|
||||
private void emitOutput(final ScoreResult sr, final String idPivot, final String idCurr, final Reporter context,List<Tuple2<String, String>> emitResult) throws IOException, InterruptedException {
|
||||
final double d = sr.getScore();
|
||||
|
||||
if (d >= dedupConf.getWf().getThreshold()) {
|
||||
|
||||
writeSimilarity(idPivot, idCurr, emitResult);
|
||||
context.incrementCounter(dedupConf.getWf().getEntityType(), "dedupSimilarity (x2)", 1);
|
||||
} else {
|
||||
context.incrementCounter(dedupConf.getWf().getEntityType(), "d < " + dedupConf.getWf().getThreshold(), 1);
|
||||
}
|
||||
}
|
||||
|
||||
private ScoreResult similarity(final PaceDocumentDistance algo, final MapDocument a, final MapDocument b) {
|
||||
try {
|
||||
return algo.between(a, b, dedupConf);
|
||||
} catch(Throwable e) {
|
||||
log.error(String.format("\nA: %s\n----------------------\nB: %s", a, b), e);
|
||||
throw new IllegalArgumentException(e);
|
||||
}
|
||||
}
|
||||
|
||||
private boolean mustSkip(final String idPivot) {
|
||||
return dedupConf.getWf().getSkipList().contains(getNsPrefix(idPivot));
|
||||
}
|
||||
|
||||
private String getNsPrefix(final String id) {
|
||||
return StringUtils.substringBetween(id, "|", "::");
|
||||
}
|
||||
|
||||
private void writeSimilarity( final String from, final String to, List<Tuple2<String, String>> emitResult){
|
||||
emitResult.add(new Tuple2<>(from, to));
|
||||
emitResult.add(new Tuple2<>( to, from));
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,28 @@
|
|||
package eu.dnetlib;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
|
||||
import java.io.Serializable;
|
||||
import java.util.HashMap;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
|
||||
class Counters extends HashMap<String, HashMap<String, AtomicLong>> implements Serializable {
|
||||
|
||||
public AtomicLong get(String counterGroup, String counterName) {
|
||||
if (!super.containsKey(counterGroup)) {
|
||||
super.put(counterGroup, new HashMap<>());
|
||||
}
|
||||
if (!super.get(counterGroup).containsKey(counterName)) {
|
||||
super.get(counterGroup).put(counterName, new AtomicLong(0));
|
||||
}
|
||||
return super.get(counterGroup).get(counterName);
|
||||
}
|
||||
|
||||
public void print(final Log log) {
|
||||
entrySet().forEach(cg -> {
|
||||
cg.getValue().entrySet().forEach(cn -> {
|
||||
log.info(cg.getKey() + " " + cn.getKey() + " " + cn.getValue());
|
||||
});
|
||||
});
|
||||
}
|
||||
}
|
|
@ -0,0 +1,13 @@
|
|||
package eu.dnetlib;
|
||||
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.Serializable;
|
||||
|
||||
public interface Reporter extends Serializable {
|
||||
|
||||
void incrementCounter(String counterGroup, String counterName, long delta);
|
||||
|
||||
void emit(final String type, final String from, final String to) throws IOException, InterruptedException;
|
||||
|
||||
}
|
|
@ -0,0 +1,108 @@
|
|||
package eu.dnetlib;
|
||||
|
||||
import com.google.common.collect.Sets;
|
||||
import eu.dnetlib.pace.clustering.BlacklistAwareClusteringCombiner;
|
||||
import eu.dnetlib.pace.config.DedupConfig;
|
||||
import eu.dnetlib.pace.model.MapDocument;
|
||||
import eu.dnetlib.pace.utils.PaceUtils;
|
||||
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 scala.Tuple2;
|
||||
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.StringWriter;
|
||||
import java.util.*;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.StreamSupport;
|
||||
|
||||
|
||||
public class SparkTest {
|
||||
|
||||
class Results extends HashMap<String, Set<String>> {
|
||||
public Results(Set<String> keys) {
|
||||
super(keys.size());
|
||||
keys.forEach(k -> put(k, new HashSet<>()));
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
public static void main(String[] args) {
|
||||
final JavaSparkContext context = new JavaSparkContext(new SparkConf().setAppName("Hello World").setMaster("local[*]"));
|
||||
final JavaRDD<String> dataRDD = context.textFile("file:///Users/sandro/Downloads/organizations.json");
|
||||
|
||||
final Counters c = new Counters();
|
||||
|
||||
long count = dataRDD.mapToPair(it -> {
|
||||
final DedupConfig config = DedupConfig.load(readFromClasspath("/eu/dnetlib/pace/organization.pace.conf"));
|
||||
MapDocument mapDocument = PaceUtils.asMapDocument(config, it);
|
||||
return new Tuple2<>(mapDocument.getIdentifier(), mapDocument);
|
||||
}).reduceByKey((a, b) -> a).flatMapToPair(a -> {
|
||||
final MapDocument currentDocument = a._2();
|
||||
final DedupConfig config = DedupConfig.load(readFromClasspath("/eu/dnetlib/pace/organization.pace.conf"));
|
||||
return getGroupingKeys(config, currentDocument).stream()
|
||||
.map(it -> new Tuple2<>(it, currentDocument)).collect(Collectors.toList()).iterator();
|
||||
}).groupByKey().flatMapToPair(it -> {
|
||||
final DedupConfig config = DedupConfig.load(readFromClasspath("/eu/dnetlib/pace/organization.pace.conf"));
|
||||
return process(config, it, c).iterator();
|
||||
}).count();
|
||||
|
||||
|
||||
System.out.println("total Element = " + count);
|
||||
|
||||
// final MapDocument resA = result(config, "A", "Recent results from CDF");
|
||||
// final MapDocument resB = result(config, "B", "Recent results from CDF");
|
||||
//
|
||||
// final ScoreResult sr = new PaceDocumentDistance().between(resA, resB, config);
|
||||
// final double d = sr.getScore();
|
||||
// System.out.println(String.format(" d ---> %s", d));
|
||||
|
||||
}
|
||||
|
||||
|
||||
static String readFromClasspath(final String filename) {
|
||||
final StringWriter sw = new StringWriter();
|
||||
try {
|
||||
IOUtils.copy(SparkTest.class.getResourceAsStream(filename), sw);
|
||||
return sw.toString();
|
||||
} catch (final IOException e) {
|
||||
throw new RuntimeException("cannot load resource from classpath: " + filename);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
static Set<String> getGroupingKeys(DedupConfig conf, MapDocument doc) {
|
||||
return Sets.newHashSet(BlacklistAwareClusteringCombiner.filterAndCombine(doc, conf));
|
||||
}
|
||||
|
||||
|
||||
static List<Tuple2<String, String>> process(DedupConfig conf, Tuple2<String, Iterable<MapDocument>> entry, Counters c) {
|
||||
try {
|
||||
return new BlockProcessor(conf).process(entry._1(), StreamSupport.stream(entry._2().spliterator(),false), new Reporter() {
|
||||
@Override
|
||||
public void incrementCounter(String counterGroup, String counterName, long delta) {
|
||||
c.get(counterGroup, counterName).addAndGet(delta);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void emit(String type, String from, String to) {
|
||||
|
||||
}
|
||||
});
|
||||
} catch (IOException | InterruptedException e) {
|
||||
e.printStackTrace();
|
||||
return new ArrayList<>();
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
}
|
|
@ -0,0 +1,150 @@
|
|||
package eu.dnetlib.data.transform;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.commons.lang.StringUtils;
|
||||
|
||||
import com.google.common.base.Splitter;
|
||||
import com.google.common.collect.Iterables;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.protobuf.Descriptors.EnumValueDescriptor;
|
||||
import com.google.protobuf.Descriptors.FieldDescriptor;
|
||||
import com.google.protobuf.GeneratedMessage;
|
||||
import com.google.protobuf.Message;
|
||||
import com.googlecode.protobuf.format.JsonFormat;
|
||||
|
||||
import eu.dnetlib.pace.config.Type;
|
||||
|
||||
/**
|
||||
* AbstractProtoMapper provide common navigation methods on the protocolbuffers Messages.
|
||||
*
|
||||
* @author claudio
|
||||
*/
|
||||
public abstract class AbstractProtoMapper {
|
||||
|
||||
private static final String COND_WRAPPER = "\\{|\\}";
|
||||
private static final String COND_SEPARATOR = "#";
|
||||
/** The Constant PATH_SEPARATOR. */
|
||||
private static final String PATH_SEPARATOR = "/";
|
||||
|
||||
/**
|
||||
* Process multi path.
|
||||
*
|
||||
* @param proto
|
||||
* the proto
|
||||
* @param paths
|
||||
* the paths
|
||||
* @return the list
|
||||
*/
|
||||
protected List<Object> processMultiPath(final GeneratedMessage proto, final List<String> paths, final Type type) {
|
||||
final List<Object> response = Lists.newArrayList();
|
||||
for (final String pathElements : paths) {
|
||||
response.addAll(processPath(proto, pathElements, type));
|
||||
}
|
||||
return response;
|
||||
}
|
||||
|
||||
/**
|
||||
* Process path.
|
||||
*
|
||||
* @param proto
|
||||
* the proto
|
||||
* @param path
|
||||
* the path
|
||||
* @return the list
|
||||
*/
|
||||
protected List<Object> processPath(final GeneratedMessage proto, final String path, final Type type) {
|
||||
return processPath(proto, Lists.newLinkedList(Splitter.on(PATH_SEPARATOR).trimResults().split(path)), type);
|
||||
}
|
||||
|
||||
/**
|
||||
* Process path.
|
||||
*
|
||||
* @param proto
|
||||
* the proto
|
||||
* @param pathElements
|
||||
* the list
|
||||
* @return the list
|
||||
*/
|
||||
protected List<Object> processPath(final GeneratedMessage proto, final List<String> pathElements, final Type type) {
|
||||
|
||||
final List<Object> response = Lists.newArrayList();
|
||||
|
||||
if (pathElements.isEmpty()) throw new RuntimeException("ProtoBuf navigation path is empty");
|
||||
|
||||
final String fieldPathCond = pathElements.get(0);
|
||||
|
||||
final String fieldPath = StringUtils.substringBefore(fieldPathCond, "[");
|
||||
final String cond = getCondition(fieldPathCond);
|
||||
|
||||
final FieldDescriptor fd = proto.getDescriptorForType().findFieldByName(fieldPath);
|
||||
if ((fd != null)) {
|
||||
if (fd.isRepeated()) {
|
||||
final int count = proto.getRepeatedFieldCount(fd);
|
||||
for (int i = 0; i < count; i++) {
|
||||
final Object field = proto.getRepeatedField(fd, i);
|
||||
response.addAll(generateFields(fd, field, pathElements, cond, type));
|
||||
}
|
||||
} else {
|
||||
final Object field = proto.getField(fd);
|
||||
response.addAll(generateFields(fd, field, pathElements, cond, type));
|
||||
}
|
||||
} else throw new IllegalArgumentException("Invalid protobuf path (field not found): " + StringUtils.join(pathElements, ">") + "\nMessage:\n" + proto);
|
||||
|
||||
return response;
|
||||
}
|
||||
|
||||
/**
|
||||
* Generate fields.
|
||||
*
|
||||
* @param fd
|
||||
* the fd
|
||||
* @param field
|
||||
* the field
|
||||
* @param list
|
||||
* the list
|
||||
* @return the list
|
||||
*/
|
||||
private List<Object> generateFields(final FieldDescriptor fd, final Object field, final List<String> list, final String cond, final Type type) {
|
||||
|
||||
final List<Object> res = Lists.newArrayList();
|
||||
if (field instanceof GeneratedMessage) {
|
||||
if (list.size() > 1) {
|
||||
|
||||
if (StringUtils.isBlank(cond)) return processPath((GeneratedMessage) field, list.subList(1, list.size()), type);
|
||||
else {
|
||||
|
||||
final List<String> condPath =
|
||||
Lists.newLinkedList(Splitter.on(COND_SEPARATOR).trimResults().split(StringUtils.substringBefore(cond, "=")));
|
||||
|
||||
final String val = (String) Iterables.getOnlyElement(processPath((GeneratedMessage) field, condPath, type));
|
||||
final String condVal = StringUtils.substringAfter(cond, "=").replaceAll(COND_WRAPPER, "").trim();
|
||||
|
||||
return val.equals(condVal) ? processPath((GeneratedMessage) field, list.subList(1, list.size()), type) : res;
|
||||
}
|
||||
}
|
||||
else if (Type.JSON.equals(type)) {
|
||||
res.add(JsonFormat.printToString((Message) field));
|
||||
return res;
|
||||
} else throw new RuntimeException("No primitive type found");
|
||||
} else {
|
||||
if (list.size() == 1) {
|
||||
|
||||
switch (fd.getType()) {
|
||||
case ENUM:
|
||||
res.add(((EnumValueDescriptor) field).getName());
|
||||
break;
|
||||
default:
|
||||
res.add(field);
|
||||
break;
|
||||
}
|
||||
return res;
|
||||
}
|
||||
else throw new RuntimeException("Found a primitive type before the path end");
|
||||
}
|
||||
}
|
||||
|
||||
private String getCondition(final String fieldPathCond) {
|
||||
return fieldPathCond.contains("[") ? StringUtils.substringAfter(fieldPathCond, "[").replace("]", "") : "";
|
||||
}
|
||||
}
|
|
@ -0,0 +1,36 @@
|
|||
package eu.dnetlib.pace.model;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.protobuf.GeneratedMessage;
|
||||
|
||||
import eu.dnetlib.data.transform.AbstractProtoMapper;
|
||||
|
||||
public class ProtoDocumentBuilder extends AbstractProtoMapper {
|
||||
|
||||
public static MapDocument newInstance(final String id, final GeneratedMessage proto, final List<FieldDef> fields) {
|
||||
final Map<String, Field> fieldMap = new ProtoDocumentBuilder().generateFieldMap(proto, fields);
|
||||
return new MapDocument(id, fieldMap);
|
||||
}
|
||||
|
||||
private Map<String, Field> generateFieldMap(final GeneratedMessage proto, final List<FieldDef> fields) {
|
||||
final Map<String, Field> fieldMap = Maps.newHashMap();
|
||||
|
||||
for (final FieldDef fd : fields) {
|
||||
|
||||
final FieldList fl = new FieldListImpl(fd.getName(), fd.getType());
|
||||
|
||||
for (final Object o : processPath(proto, fd.getPathList(), fd.getType())) {
|
||||
|
||||
fl.add(new FieldValueImpl(fd.getType(), fd.getName(), o));
|
||||
}
|
||||
|
||||
fieldMap.put(fd.getName(), fl);
|
||||
}
|
||||
|
||||
return fieldMap;
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,90 @@
|
|||
package eu.dnetlib.pace.utils;
|
||||
|
||||
import com.google.common.collect.Lists;
|
||||
import com.googlecode.protobuf.format.JsonFormat;
|
||||
import eu.dnetlib.data.proto.OafProtos;
|
||||
import eu.dnetlib.data.proto.ResultProtos;
|
||||
import eu.dnetlib.pace.config.Config;
|
||||
import eu.dnetlib.pace.config.DedupConfig;
|
||||
import eu.dnetlib.pace.model.MapDocument;
|
||||
import eu.dnetlib.pace.model.ProtoDocumentBuilder;
|
||||
import org.apache.commons.lang3.RandomStringUtils;
|
||||
import org.apache.commons.lang3.StringUtils;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.LinkedList;
|
||||
import java.util.List;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.IntStream;
|
||||
|
||||
import static eu.dnetlib.proto.utils.OAFProtoUtils.*;
|
||||
import static eu.dnetlib.proto.utils.OAFProtoUtils.author;
|
||||
import static eu.dnetlib.proto.utils.OAFProtoUtils.sp;
|
||||
|
||||
public class PaceUtils {
|
||||
|
||||
public static MapDocument result(final Config config, final String id, final String title) {
|
||||
return result(config, id, title, null, new ArrayList<>(), null);
|
||||
}
|
||||
|
||||
public static MapDocument result(final Config config, final String id, final String title, final String date) {
|
||||
return result(config, id, title, date, new ArrayList<>(), null);
|
||||
}
|
||||
|
||||
public static MapDocument result(final Config config, final String id, final String title, final String date, final List<String> pid) {
|
||||
return result(config, id, title, date, pid, null);
|
||||
}
|
||||
|
||||
public static MapDocument result(final Config config, final String id, final String title, final String date, final String pid) {
|
||||
return result(config, id, title, date, pid, null);
|
||||
}
|
||||
|
||||
public static MapDocument result(final Config config, final String id, final String title, final String date, final String pid, final List<String> authors) {
|
||||
return result(config, id, title, date, Lists.newArrayList(pid), authors);
|
||||
}
|
||||
|
||||
public static MapDocument result(final Config config, final String id, final String title, final String date, final List<String> pid, final List<String> authors) {
|
||||
final ResultProtos.Result.Metadata.Builder metadata = ResultProtos.Result.Metadata.newBuilder();
|
||||
if (!StringUtils.isBlank(title)) {
|
||||
metadata.addTitle(getStruct(title, getQualifier("main title", "dnet:titles")));
|
||||
metadata.addTitle(getStruct(RandomStringUtils.randomAlphabetic(10), getQualifier("alternative title", "dnet:titles")));
|
||||
}
|
||||
if (!StringUtils.isBlank(date)) {
|
||||
metadata.setDateofacceptance(sf(date));
|
||||
}
|
||||
|
||||
final OafProtos.OafEntity.Builder entity = oafEntity(id, eu.dnetlib.data.proto.TypeProtos.Type.result);
|
||||
final ResultProtos.Result.Builder result = ResultProtos.Result.newBuilder().setMetadata(metadata);
|
||||
|
||||
if (authors != null) {
|
||||
result.getMetadataBuilder().addAllAuthor(
|
||||
IntStream.range(0, authors.size())
|
||||
.mapToObj(i -> author(authors.get(i), i))
|
||||
.collect(Collectors.toCollection(LinkedList::new)));
|
||||
}
|
||||
|
||||
entity.setResult(result);
|
||||
|
||||
if (pid != null) {
|
||||
for (String p : pid) {
|
||||
if (!StringUtils.isBlank(p)) {
|
||||
entity.addPid(sp(p, "doi"));
|
||||
//entity.addPid(sp(RandomStringUtils.randomAlphabetic(10), "oai"));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
final OafProtos.OafEntity build = entity.build();
|
||||
return ProtoDocumentBuilder.newInstance(id, build, config.model());
|
||||
}
|
||||
|
||||
public static MapDocument asMapDocument(DedupConfig conf, final String json) {
|
||||
OafProtos.OafEntity.Builder b = OafProtos.OafEntity.newBuilder();
|
||||
try {
|
||||
JsonFormat.merge(json, b);
|
||||
} catch (JsonFormat.ParseException e) {
|
||||
throw new IllegalArgumentException(e);
|
||||
}
|
||||
return ProtoDocumentBuilder.newInstance(b.getId(), b.build(), conf.getPace().getModel());
|
||||
}
|
||||
}
|
|
@ -0,0 +1,44 @@
|
|||
package eu.dnetlib.proto.utils;
|
||||
|
||||
import eu.dnetlib.data.proto.FieldTypeProtos;
|
||||
import eu.dnetlib.data.proto.OafProtos;
|
||||
|
||||
public class OAFProtoUtils {
|
||||
|
||||
|
||||
public static FieldTypeProtos.Author author(final String s, int rank) {
|
||||
final eu.dnetlib.pace.model.Person p = new eu.dnetlib.pace.model.Person(s, false);
|
||||
final FieldTypeProtos.Author.Builder author = FieldTypeProtos.Author.newBuilder();
|
||||
if (p.isAccurate()) {
|
||||
author.setName(p.getNormalisedFirstName());
|
||||
author.setSurname(p.getNormalisedSurname());
|
||||
}
|
||||
author.setFullname(p.getNormalisedFullname());
|
||||
author.setRank(rank);
|
||||
|
||||
return author.build();
|
||||
}
|
||||
|
||||
public static FieldTypeProtos.StructuredProperty sp(final String pid, final String type) {
|
||||
FieldTypeProtos.StructuredProperty.Builder pidSp = FieldTypeProtos.StructuredProperty.newBuilder().setValue(pid)
|
||||
.setQualifier(FieldTypeProtos.Qualifier.newBuilder().setClassid(type).setClassname(type).setSchemeid("dnet:pid_types").setSchemename("dnet:pid_types"));
|
||||
return pidSp.build();
|
||||
}
|
||||
|
||||
public static FieldTypeProtos.StringField.Builder sf(final String s) { return FieldTypeProtos.StringField.newBuilder().setValue(s); }
|
||||
|
||||
public static FieldTypeProtos.StructuredProperty.Builder getStruct(final String value, final FieldTypeProtos.Qualifier.Builder qualifier) {
|
||||
return FieldTypeProtos.StructuredProperty.newBuilder().setValue(value).setQualifier(qualifier);
|
||||
}
|
||||
|
||||
public static FieldTypeProtos.Qualifier.Builder getQualifier(final String classname, final String schemename) {
|
||||
return
|
||||
FieldTypeProtos.Qualifier.newBuilder().setClassid(classname).setClassname(classname).setSchemeid(schemename).setSchemename(schemename);
|
||||
}
|
||||
|
||||
public static OafProtos.OafEntity.Builder oafEntity(final String id, final eu.dnetlib.data.proto.TypeProtos.Type type) {
|
||||
final OafProtos.OafEntity.Builder entity = OafProtos.OafEntity.newBuilder().setId(id).setType(type);
|
||||
return entity;
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,34 @@
|
|||
{
|
||||
"wf" : {
|
||||
"threshold" : "0.85",
|
||||
"dedupRun" : "001",
|
||||
"entityType" : "organization",
|
||||
"orderField" : "legalname",
|
||||
"queueMaxSize" : "20000",
|
||||
"groupMaxSize" : "20",
|
||||
"slidingWindowSize" : "400",
|
||||
"rootBuilder" : [ "organization", "projectOrganization_participation_isParticipant", "datasourceOrganization_provision_isProvidedBy" ],
|
||||
"includeChildren" : "true"
|
||||
},
|
||||
"pace" : {
|
||||
"clustering" : [
|
||||
{ "name" : "ngrampairs", "fields" : [ "legalname" ], "params" : { "max" : 1, "ngramLen" : "3" } },
|
||||
{ "name" : "suffixprefix", "fields" : [ "legalname" ], "params" : { "max" : 1, "len" : "3" } },
|
||||
{ "name" : "immutablefieldvalue", "fields" : [ "country" ], "params" : { } },
|
||||
{ "name" : "spacetrimmingfieldvalue", "fields" : [ "legalshortname" ], "params" : { "randomLength" : "5" } },
|
||||
{ "name" : "urlclustering", "fields" : [ "websiteurl" ], "params" : { } }
|
||||
],
|
||||
"conditions" : [
|
||||
{ "name" : "exactMatch", "fields" : [ "country" ] },
|
||||
{ "name" : "mustBeDifferent", "fields" : [ "gridid" ] }
|
||||
],
|
||||
"model" : [
|
||||
{ "name" : "legalname", "algo" : "LevensteinTitle", "type" : "String", "weight" : "0.2", "ignoreMissing" : "false", "path" : "organization/metadata/legalname/value" },
|
||||
{ "name" : "legalshortname", "algo" : "LevensteinTitle", "type" : "String", "weight" : "0.2", "ignoreMissing" : "true", "path" : "organization/metadata/legalshortname/value" },
|
||||
{ "name" : "websiteurl", "algo" : "urlMatcher", "type" : "URL", "weight" : "0.6", "ignoreMissing" : "true", "path" : "organization/metadata/websiteurl/value", "params" : { "host" : 0.5, "path" : 0.5 } },
|
||||
{ "name" : "country", "algo" : "Null", "type" : "String", "weight" : "0.0", "ignoreMissing" : "true", "path" : "organization/metadata/country/classid" },
|
||||
{ "name" : "gridid", "algo" : "Null", "type" : "String", "weight" : "0.0", "ignoreMissing" : "true", "path" : "pid[qualifier#classid = {grid}]/value" }
|
||||
],
|
||||
"blacklists" : { }
|
||||
}
|
||||
}
|
|
@ -0,0 +1,51 @@
|
|||
{
|
||||
"wf" : {
|
||||
"threshold" : "0.99",
|
||||
"run" : "001",
|
||||
"entityType" : "result",
|
||||
"orderField" : "title",
|
||||
"queueMaxSize" : "2000",
|
||||
"groupMaxSize" : "10",
|
||||
"slidingWindowSize" : "200",
|
||||
"rootBuilder" : [ "result" ],
|
||||
"includeChildren" : "true"
|
||||
},
|
||||
"pace" : {
|
||||
"clustering" : [
|
||||
{ "name" : "acronyms", "fields" : [ "title" ], "params" : { "max" : "1", "minLen" : "2", "maxLen" : "4"} },
|
||||
{ "name" : "ngrampairs", "fields" : [ "title" ], "params" : { "max" : "1", "ngramLen" : "3"} },
|
||||
{ "name" : "suffixprefix", "fields" : [ "title" ], "params" : { "max" : "1", "len" : "3" } }
|
||||
],
|
||||
"conditions" : [
|
||||
{ "name" : "yearMatch", "fields" : [ "dateofacceptance" ] },
|
||||
{ "name" : "titleVersionMatch", "fields" : [ "title" ] },
|
||||
{ "name" : "sizeMatch", "fields" : [ "authors" ] } ,
|
||||
{ "name" : "pidMatch", "fields" : [ "pid" ] }
|
||||
],
|
||||
"model" : [
|
||||
{ "name" : "pid", "algo" : "Null", "type" : "JSON", "weight" : "0.0", "ignoreMissing" : "true", "path" : "pid", "overrideMatch" : "true" },
|
||||
{ "name" : "title", "algo" : "JaroWinkler", "type" : "String", "weight" : "1.0", "ignoreMissing" : "false", "path" : "result/metadata/title[qualifier#classid = {main title}]/value" },
|
||||
{ "name" : "dateofacceptance", "algo" : "Null", "type" : "String", "weight" : "0.0", "ignoreMissing" : "true", "path" : "result/metadata/dateofacceptance/value" } ,
|
||||
{ "name" : "authors", "algo" : "Null", "type" : "String", "weight" : "0.0", "ignoreMissing" : "true", "path" : "result/metadata/author/fullname" }
|
||||
],
|
||||
"blacklists" : {
|
||||
"title" : [
|
||||
"^(Corpus Oral Dialectal \\(COD\\)\\.).*$",
|
||||
"^(Kiri Karl Morgensternile).*$",
|
||||
"^(\\[Eksliibris Aleksandr).*\\]$",
|
||||
"^(\\[Eksliibris Aleksandr).*$",
|
||||
"^(Eksliibris Aleksandr).*$",
|
||||
"^(Kiri A\\. de Vignolles).*$",
|
||||
"^(2 kirja Karl Morgensternile).*$",
|
||||
"^(Pirita kloostri idaosa arheoloogilised).*$",
|
||||
"^(Kiri tundmatule).*$",
|
||||
"^(Kiri Jenaer Allgemeine Literaturzeitung toimetusele).*$",
|
||||
"^(Eksliibris Nikolai Birukovile).*$",
|
||||
"^(Eksliibris Nikolai Issakovile).*$",
|
||||
"^(WHP Cruise Summary Information of section).*$",
|
||||
"^(Measurement of the top quark\\-pair production cross section with ATLAS in pp collisions at).*$",
|
||||
"^(Measurement of the spin\\-dependent structure function).*"
|
||||
] }
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,21 @@
|
|||
{
|
||||
"wf" : {
|
||||
"threshold" : "0.99",
|
||||
"run" : "001",
|
||||
"entityType" : "result",
|
||||
"orderField" : "title",
|
||||
"queueMaxSize" : "2000",
|
||||
"groupMaxSize" : "10",
|
||||
"slidingWindowSize" : "200",
|
||||
"rootBuilder" : [ "result" ],
|
||||
"includeChildren" : "true"
|
||||
},
|
||||
"pace" : {
|
||||
"conditions" : [ ],
|
||||
"model" : [
|
||||
{ "name" : "title", "algo" : "JaroWinkler", "type" : "String", "weight" : "1.0", "ignoreMissing" : "false", "path" : "result/metadata/title[qualifier#classid = {main title}]/value" }
|
||||
],
|
||||
"blacklists" : { }
|
||||
}
|
||||
|
||||
}
|
|
@ -23,7 +23,7 @@
|
|||
<dependency>
|
||||
<groupId>com.google.guava</groupId>
|
||||
<artifactId>guava</artifactId>
|
||||
<version>${google.guava.version}</version>
|
||||
<version>15.0</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.google.code.gson</groupId>
|
||||
|
|
|
@ -22,9 +22,12 @@ public class BlacklistAwareClusteringCombiner extends ClusteringCombiner {
|
|||
|
||||
private static final Log log = LogFactory.getLog(BlacklistAwareClusteringCombiner.class);
|
||||
|
||||
public static Collection<String> filterAndCombine(final MapDocument a, final Config conf, final Map<String, List<String>> blacklists) {
|
||||
|
||||
final Document filtered = new BlacklistAwareClusteringCombiner().filter(a, blacklists);
|
||||
|
||||
|
||||
public static Collection<String> filterAndCombine(final MapDocument a, final Config conf) {
|
||||
|
||||
final Document filtered = new BlacklistAwareClusteringCombiner().filter(a, conf.blacklists());
|
||||
return combine(filtered, conf);
|
||||
}
|
||||
|
||||
|
|
|
@ -1,6 +1,7 @@
|
|||
package eu.dnetlib.pace.config;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.Serializable;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
@ -19,7 +20,7 @@ import eu.dnetlib.pace.model.FieldDef;
|
|||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
|
||||
public class DedupConfig implements Config {
|
||||
public class DedupConfig implements Config, Serializable {
|
||||
|
||||
private static final Log log = LogFactory.getLog(DedupConfig.class);
|
||||
|
||||
|
|
|
@ -2,10 +2,12 @@ package eu.dnetlib.pace.model;
|
|||
|
||||
import eu.dnetlib.pace.config.Type;
|
||||
|
||||
import java.io.Serializable;
|
||||
|
||||
/**
|
||||
* The Interface Field.
|
||||
*/
|
||||
public interface Field extends Iterable<Field> {
|
||||
public interface Field extends Iterable<Field>, Serializable {
|
||||
|
||||
/**
|
||||
* Gets the name.
|
||||
|
|
|
@ -1,5 +1,6 @@
|
|||
package eu.dnetlib.pace.model;
|
||||
|
||||
import java.io.Serializable;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
|
@ -10,7 +11,7 @@ import com.google.common.collect.Maps;
|
|||
/**
|
||||
* The Class MapDocument.
|
||||
*/
|
||||
public class MapDocument implements Document {
|
||||
public class MapDocument implements Document, Serializable {
|
||||
|
||||
/** The identifier. */
|
||||
private String identifier;
|
||||
|
|
Loading…
Reference in New Issue