forked from D-Net/dnet-hadoop
refactoring of the id generation, sparkcreatemergerels collects entities to create root id after a join
This commit is contained in:
parent
176a5e493d
commit
f64e57c112
|
@ -36,7 +36,6 @@ public class ModelConstants {
|
|||
public static final String DNET_COUNTRY_TYPE = "dnet:countries";
|
||||
public static final String DNET_REVIEW_LEVELS = "dnet:review_levels";
|
||||
public static final String DNET_PROGRAMMING_LANGUAGES = "dnet:programming_languages";
|
||||
public static final String DNET_PROVENANCEACTIONS = "dnet:provenanceActions";
|
||||
public static final String DNET_EXTERNAL_REF_TYPES = "dnet:externalReference_typologies";
|
||||
|
||||
public static final String SYSIMPORT_CROSSWALK_REPOSITORY = "sysimport:crosswalk:repository";
|
||||
|
@ -50,7 +49,7 @@ public class ModelConstants {
|
|||
public static final String PROVENANCE_DEDUP = "sysimport:dedup";
|
||||
|
||||
public static final Qualifier PROVENANCE_ACTION_SET_QUALIFIER = qualifier(
|
||||
SYSIMPORT_ACTIONSET, SYSIMPORT_ACTIONSET, DNET_PROVENANCEACTIONS, DNET_PROVENANCEACTIONS);
|
||||
SYSIMPORT_ACTIONSET, SYSIMPORT_ACTIONSET, DNET_PROVENANCE_ACTIONS, DNET_PROVENANCE_ACTIONS);
|
||||
|
||||
public static final String DATASET_RESULTTYPE_CLASSID = "dataset";
|
||||
public static final String PUBLICATION_RESULTTYPE_CLASSID = "publication";
|
||||
|
|
|
@ -82,16 +82,12 @@ public class DedupRecordFactory {
|
|||
|
||||
final Collection<String> dates = Lists.newArrayList();
|
||||
final List<List<Author>> authors = Lists.newArrayList();
|
||||
// final List<Identifier<T>> bestPids = Lists.newArrayList(); // best pids list
|
||||
|
||||
entities
|
||||
.forEachRemaining(
|
||||
t -> {
|
||||
T duplicate = t._2();
|
||||
|
||||
// prepare the list of pids to be used for the id generation
|
||||
// bestPids.add(Identifier.newInstance(duplicate));
|
||||
|
||||
entity.mergeFrom(duplicate);
|
||||
if (ModelSupport.isSubClass(duplicate, Result.class)) {
|
||||
Result r1 = (Result) duplicate;
|
||||
|
@ -109,7 +105,6 @@ public class DedupRecordFactory {
|
|||
((Result) entity).setAuthor(AuthorMerger.merge(authors));
|
||||
}
|
||||
|
||||
// entity.setId(IdGenerator.generate(bestPids, id));
|
||||
entity.setId(id);
|
||||
|
||||
entity.setLastupdatetimestamp(ts);
|
||||
|
|
|
@ -1,6 +1,9 @@
|
|||
|
||||
package eu.dnetlib.dhp.oa.dedup;
|
||||
|
||||
import static eu.dnetlib.dhp.schema.common.ModelConstants.DNET_PROVENANCE_ACTIONS;
|
||||
import static eu.dnetlib.dhp.schema.common.ModelConstants.PROVENANCE_DEDUP;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.commons.io.IOUtils;
|
||||
|
@ -27,8 +30,6 @@ public class SparkCreateDedupRecord extends AbstractSparkAction {
|
|||
private static final Logger log = LoggerFactory.getLogger(SparkCreateDedupRecord.class);
|
||||
|
||||
public static final String ROOT_TRUST = "0.8";
|
||||
public static final String PROVENANCE_ACTION_CLASS = "sysimport:dedup";
|
||||
public static final String PROVENANCE_ACTIONS = "dnet:provenanceActions";
|
||||
|
||||
public SparkCreateDedupRecord(ArgumentApplicationParser parser, SparkSession spark) {
|
||||
super(parser, spark);
|
||||
|
@ -94,10 +95,10 @@ public class SparkCreateDedupRecord extends AbstractSparkAction {
|
|||
info.setTrust(ROOT_TRUST);
|
||||
info.setInferenceprovenance(dedupConf.getWf().getConfigurationId());
|
||||
Qualifier provenance = new Qualifier();
|
||||
provenance.setClassid(PROVENANCE_ACTION_CLASS);
|
||||
provenance.setClassname(PROVENANCE_ACTION_CLASS);
|
||||
provenance.setSchemeid(PROVENANCE_ACTIONS);
|
||||
provenance.setSchemename(PROVENANCE_ACTIONS);
|
||||
provenance.setClassid(PROVENANCE_DEDUP);
|
||||
provenance.setClassname(PROVENANCE_DEDUP);
|
||||
provenance.setSchemeid(DNET_PROVENANCE_ACTIONS);
|
||||
provenance.setSchemename(DNET_PROVENANCE_ACTIONS);
|
||||
info.setProvenanceaction(provenance);
|
||||
return info;
|
||||
}
|
||||
|
|
|
@ -1,20 +1,20 @@
|
|||
|
||||
package eu.dnetlib.dhp.oa.dedup;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Optional;
|
||||
import static eu.dnetlib.dhp.schema.common.ModelConstants.DNET_PROVENANCE_ACTIONS;
|
||||
import static eu.dnetlib.dhp.schema.common.ModelConstants.PROVENANCE_DEDUP;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.*;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import eu.dnetlib.dhp.schema.common.EntityType;
|
||||
import eu.dnetlib.dhp.schema.oaf.OafEntity;
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.spark.SparkConf;
|
||||
import org.apache.spark.api.java.JavaPairRDD;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.apache.spark.api.java.function.FlatMapFunction;
|
||||
import org.apache.spark.api.java.function.MapFunction;
|
||||
import org.apache.spark.api.java.function.PairFunction;
|
||||
import org.apache.spark.api.java.function.MapGroupsFunction;
|
||||
import org.apache.spark.graphx.Edge;
|
||||
import org.apache.spark.rdd.RDD;
|
||||
import org.apache.spark.sql.Dataset;
|
||||
|
@ -25,14 +25,18 @@ import org.dom4j.DocumentException;
|
|||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.hash.Hashing;
|
||||
|
||||
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
|
||||
import eu.dnetlib.dhp.oa.dedup.graph.ConnectedComponent;
|
||||
import eu.dnetlib.dhp.oa.dedup.graph.GraphProcessor;
|
||||
import eu.dnetlib.dhp.oa.dedup.model.Identifier;
|
||||
import eu.dnetlib.dhp.schema.common.EntityType;
|
||||
import eu.dnetlib.dhp.schema.common.ModelConstants;
|
||||
import eu.dnetlib.dhp.schema.common.ModelSupport;
|
||||
import eu.dnetlib.dhp.schema.oaf.DataInfo;
|
||||
import eu.dnetlib.dhp.schema.oaf.OafEntity;
|
||||
import eu.dnetlib.dhp.schema.oaf.Qualifier;
|
||||
import eu.dnetlib.dhp.schema.oaf.Relation;
|
||||
import eu.dnetlib.dhp.utils.ISLookupClientFactory;
|
||||
|
@ -44,9 +48,7 @@ import scala.Tuple2;
|
|||
|
||||
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);
|
||||
|
@ -94,6 +96,7 @@ public class SparkCreateMergeRels extends AbstractSparkAction {
|
|||
|
||||
for (DedupConfig dedupConf : getConfigurations(isLookUpService, actionSetId)) {
|
||||
final String subEntity = dedupConf.getWf().getSubEntityValue();
|
||||
final Class<OafEntity> clazz = ModelSupport.entityTypes.get(EntityType.valueOf(subEntity));
|
||||
|
||||
log.info("Creating mergerels for: '{}'", subEntity);
|
||||
|
||||
|
@ -102,7 +105,7 @@ public class SparkCreateMergeRels extends AbstractSparkAction {
|
|||
|
||||
final String mergeRelPath = DedupUtility.createMergeRelPath(workingPath, actionSetId, subEntity);
|
||||
|
||||
//<hash(id), json>
|
||||
// <hash(id), id>
|
||||
JavaPairRDD<Object, String> vertexes = createVertexes(sc, graphBasePath, subEntity, dedupConf);
|
||||
|
||||
final RDD<Edge<String>> edgeRdd = spark
|
||||
|
@ -113,14 +116,42 @@ public class SparkCreateMergeRels extends AbstractSparkAction {
|
|||
.map(it -> new Edge<>(hash(it.getSource()), hash(it.getTarget()), it.getRelClass()))
|
||||
.rdd();
|
||||
|
||||
final Dataset<Relation> mergeRels = spark
|
||||
Dataset<Tuple2<String, String>> rawMergeRels = spark
|
||||
.createDataset(
|
||||
GraphProcessor
|
||||
.findCCs(vertexes.rdd(), edgeRdd, subEntity, maxIterations, cut)
|
||||
.findCCs(vertexes.rdd(), edgeRdd, maxIterations, cut)
|
||||
.toJavaRDD()
|
||||
.filter(k -> k.getEntities().size() > 1)
|
||||
.flatMap(cc -> ccToMergeRel(cc, dedupConf))
|
||||
.filter(k -> k.getIds().size() > 1)
|
||||
.flatMap(this::ccToRels)
|
||||
.rdd(),
|
||||
Encoders.tuple(Encoders.STRING(), Encoders.STRING()));
|
||||
|
||||
Dataset<Tuple2<String, OafEntity>> entities = spark
|
||||
.read()
|
||||
.textFile(DedupUtility.createEntityPath(graphBasePath, subEntity))
|
||||
.map(
|
||||
(MapFunction<String, Tuple2<String, OafEntity>>) it -> {
|
||||
OafEntity entity = OBJECT_MAPPER.readValue(it, clazz);
|
||||
return new Tuple2<>(entity.getId(), entity);
|
||||
},
|
||||
Encoders.tuple(Encoders.STRING(), Encoders.kryo(clazz)));
|
||||
|
||||
Dataset<Relation> mergeRels = rawMergeRels
|
||||
.joinWith(entities, rawMergeRels.col("_2").equalTo(entities.col("_1")), "inner")
|
||||
// <tmp_source,target>,<target,entity>
|
||||
.map(
|
||||
(MapFunction<Tuple2<Tuple2<String, String>, Tuple2<String, OafEntity>>, Tuple2<String, OafEntity>>) value -> new Tuple2<>(
|
||||
value._1()._1(), value._2()._2()),
|
||||
Encoders.tuple(Encoders.STRING(), Encoders.kryo(clazz)))
|
||||
// <tmp_source,entity>
|
||||
.groupByKey(
|
||||
(MapFunction<Tuple2<String, OafEntity>, String>) Tuple2::_1, Encoders.STRING())
|
||||
.mapGroups(
|
||||
(MapGroupsFunction<String, Tuple2<String, OafEntity>, ConnectedComponent>) this::generateID,
|
||||
Encoders.bean(ConnectedComponent.class))
|
||||
// <root_id, list(target)>
|
||||
.flatMap(
|
||||
(FlatMapFunction<ConnectedComponent, Relation>) cc -> ccToMergeRel(cc, dedupConf),
|
||||
Encoders.bean(Relation.class));
|
||||
|
||||
mergeRels.write().mode(SaveMode.Append).parquet(mergeRelPath);
|
||||
|
@ -128,26 +159,52 @@ public class SparkCreateMergeRels extends AbstractSparkAction {
|
|||
}
|
||||
}
|
||||
|
||||
private JavaPairRDD<Object, String> createVertexes(JavaSparkContext sc, String graphBasePath, String subEntity, DedupConfig dedupConf) throws IOException {
|
||||
private <T extends OafEntity> ConnectedComponent generateID(String key, Iterator<Tuple2<String, T>> values) {
|
||||
|
||||
List<Identifier<T>> identifiers = Lists.newArrayList(values).stream().map(v -> {
|
||||
T entity = v._2();
|
||||
Identifier<T> identifier = Identifier.newInstance(entity);
|
||||
return identifier;
|
||||
}).collect(Collectors.toList());
|
||||
|
||||
String rootID = IdGenerator.generate(identifiers, key);
|
||||
|
||||
if (Objects.equals(rootID, key))
|
||||
throw new IllegalStateException("generated default ID: " + rootID);
|
||||
|
||||
return new ConnectedComponent(rootID,
|
||||
identifiers.stream().map(i -> i.getEntity().getId()).collect(Collectors.toSet()));
|
||||
}
|
||||
|
||||
private JavaPairRDD<Object, String> createVertexes(JavaSparkContext sc, String graphBasePath, String subEntity,
|
||||
DedupConfig dedupConf) {
|
||||
|
||||
return sc
|
||||
.textFile(graphBasePath + "/" + subEntity)
|
||||
.mapToPair(json -> {
|
||||
String id = MapDocumentUtil.getJPathString(dedupConf.getWf().getIdPath(), json);
|
||||
return new Tuple2<>(hash(id), json);
|
||||
});
|
||||
.textFile(DedupUtility.createEntityPath(graphBasePath, subEntity))
|
||||
.mapToPair(json -> {
|
||||
String id = MapDocumentUtil.getJPathString(dedupConf.getWf().getIdPath(), json);
|
||||
return new Tuple2<>(hash(id), id);
|
||||
});
|
||||
}
|
||||
|
||||
private Iterator<Tuple2<String, String>> ccToRels(ConnectedComponent cc) {
|
||||
return cc
|
||||
.getIds()
|
||||
.stream()
|
||||
.map(id -> new Tuple2<>(cc.getCcId(), id))
|
||||
.iterator();
|
||||
}
|
||||
|
||||
private Iterator<Relation> ccToMergeRel(ConnectedComponent cc, DedupConfig dedupConf) {
|
||||
return cc
|
||||
.getEntities()
|
||||
.getIds()
|
||||
.stream()
|
||||
.flatMap(
|
||||
e -> {
|
||||
id -> {
|
||||
List<Relation> tmp = new ArrayList<>();
|
||||
|
||||
tmp.add(rel(cc.getCcId(), MapDocumentUtil.getJPathString("$.id", e), ModelConstants.MERGES, dedupConf));
|
||||
tmp.add(rel( MapDocumentUtil.getJPathString("$.id", e), cc.getCcId(), ModelConstants.IS_MERGED_IN, dedupConf));
|
||||
tmp.add(rel(cc.getCcId(), id, ModelConstants.MERGES, dedupConf));
|
||||
tmp.add(rel(id, cc.getCcId(), ModelConstants.IS_MERGED_IN, dedupConf));
|
||||
|
||||
return tmp.stream();
|
||||
})
|
||||
|
@ -171,8 +228,8 @@ public class SparkCreateMergeRels extends AbstractSparkAction {
|
|||
info.setInvisible(false);
|
||||
info.setInferenceprovenance(dedupConf.getWf().getConfigurationId());
|
||||
Qualifier provenanceAction = new Qualifier();
|
||||
provenanceAction.setClassid(PROVENANCE_ACTION_CLASS);
|
||||
provenanceAction.setClassname(PROVENANCE_ACTION_CLASS);
|
||||
provenanceAction.setClassid(PROVENANCE_DEDUP);
|
||||
provenanceAction.setClassname(PROVENANCE_DEDUP);
|
||||
provenanceAction.setSchemeid(DNET_PROVENANCE_ACTIONS);
|
||||
provenanceAction.setSchemename(DNET_PROVENANCE_ACTIONS);
|
||||
info.setProvenanceaction(provenanceAction);
|
||||
|
|
|
@ -9,69 +9,60 @@ import java.util.Objects;
|
|||
import java.util.Set;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import org.apache.commons.lang.StringUtils;
|
||||
import org.apache.spark.api.java.function.MapFunction;
|
||||
import org.codehaus.jackson.annotate.JsonIgnore;
|
||||
|
||||
import com.fasterxml.jackson.databind.DeserializationFeature;
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.google.common.collect.Lists;
|
||||
|
||||
import eu.dnetlib.dhp.oa.dedup.DedupUtility;
|
||||
import eu.dnetlib.dhp.oa.dedup.IdGenerator;
|
||||
import eu.dnetlib.dhp.oa.dedup.model.Identifier;
|
||||
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.DHPUtils;
|
||||
import eu.dnetlib.pace.config.DedupConfig;
|
||||
import eu.dnetlib.pace.util.MapDocumentUtil;
|
||||
import org.apache.commons.lang.StringUtils;
|
||||
import org.apache.spark.api.java.function.MapFunction;
|
||||
import org.codehaus.jackson.annotate.JsonIgnore;
|
||||
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
|
||||
import eu.dnetlib.dhp.oa.dedup.DedupUtility;
|
||||
import eu.dnetlib.dhp.utils.DHPUtils;
|
||||
import eu.dnetlib.pace.util.PaceException;
|
||||
|
||||
public class ConnectedComponent implements Serializable {
|
||||
|
||||
private String ccId;
|
||||
private Set<String> entities;
|
||||
private Set<String> ids;
|
||||
|
||||
protected static final ObjectMapper OBJECT_MAPPER = new ObjectMapper()
|
||||
.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
|
||||
private static final String CONNECTED_COMPONENT_ID_PREFIX = "connect_comp";
|
||||
|
||||
public <T extends OafEntity> ConnectedComponent(Set<String> entities, String subEntity, final int cut) {
|
||||
this.entities = entities;
|
||||
final Class<T> clazz = ModelSupport.entityTypes.get(EntityType.valueOf(subEntity));
|
||||
public ConnectedComponent(Set<String> ids, final int cut) {
|
||||
this.ids = ids;
|
||||
|
||||
List<Identifier<T>> identifiers = Lists.newArrayList();
|
||||
this.ccId = createDefaultID();
|
||||
|
||||
entities.forEach(e -> {
|
||||
try {
|
||||
T entity = OBJECT_MAPPER.readValue(e, clazz);
|
||||
identifiers.add(Identifier.newInstance(entity));
|
||||
} catch (IOException e1) {
|
||||
}
|
||||
});
|
||||
|
||||
this.ccId = IdGenerator.generate(
|
||||
identifiers,
|
||||
createDefaultID()
|
||||
);
|
||||
|
||||
if (cut > 0 && entities.size() > cut) {
|
||||
this.entities = entities
|
||||
.stream()
|
||||
.filter(e -> !ccId.equalsIgnoreCase(MapDocumentUtil.getJPathString("$.id", e)))
|
||||
.limit(cut - 1)
|
||||
.collect(Collectors.toSet());
|
||||
if (cut > 0 && ids.size() > cut) {
|
||||
this.ids = ids
|
||||
.stream()
|
||||
.filter(id -> !ccId.equalsIgnoreCase(id))
|
||||
.limit(cut - 1)
|
||||
.collect(Collectors.toSet());
|
||||
// this.ids.add(ccId); ??
|
||||
}
|
||||
}
|
||||
|
||||
public ConnectedComponent(String ccId, Set<String> ids) {
|
||||
this.ccId = ccId;
|
||||
this.ids = ids;
|
||||
}
|
||||
|
||||
public String createDefaultID() {
|
||||
if (entities.size() > 1) {
|
||||
if (ids.size() > 1) {
|
||||
final String s = getMin();
|
||||
String prefix = s.split("\\|")[0];
|
||||
ccId = prefix + "|dedup_wf_001::" + DHPUtils.md5(s);
|
||||
ccId = prefix + "|" + CONNECTED_COMPONENT_ID_PREFIX + "::" + DHPUtils.md5(s);
|
||||
return ccId;
|
||||
} else {
|
||||
return MapDocumentUtil.getJPathString("$.id", entities.iterator().next());
|
||||
return ids.iterator().next();
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -80,15 +71,15 @@ public class ConnectedComponent implements Serializable {
|
|||
|
||||
final StringBuilder min = new StringBuilder();
|
||||
|
||||
entities
|
||||
ids
|
||||
.forEach(
|
||||
e -> {
|
||||
id -> {
|
||||
if (StringUtils.isBlank(min.toString())) {
|
||||
min.append(MapDocumentUtil.getJPathString("$.id", e));
|
||||
min.append(id);
|
||||
} else {
|
||||
if (min.toString().compareTo(MapDocumentUtil.getJPathString("$.id", e)) > 0) {
|
||||
if (min.toString().compareTo(id) > 0) {
|
||||
min.setLength(0);
|
||||
min.append(MapDocumentUtil.getJPathString("$.id", e));
|
||||
min.append(id);
|
||||
}
|
||||
}
|
||||
});
|
||||
|
@ -105,12 +96,12 @@ public class ConnectedComponent implements Serializable {
|
|||
}
|
||||
}
|
||||
|
||||
public Set<String> getEntities() {
|
||||
return entities;
|
||||
public Set<String> getIds() {
|
||||
return ids;
|
||||
}
|
||||
|
||||
public void setEntities(Set<String> docIds) {
|
||||
this.entities = entities;
|
||||
public void setIds(Set<String> ids) {
|
||||
this.ids = ids;
|
||||
}
|
||||
|
||||
public String getCcId() {
|
||||
|
|
|
@ -7,7 +7,7 @@ import scala.collection.JavaConversions;
|
|||
|
||||
object GraphProcessor {
|
||||
|
||||
def findCCs(vertexes: RDD[(VertexId, String)], edges: RDD[Edge[String]], subEntity: String, maxIterations: Int, cut:Int): RDD[ConnectedComponent] = {
|
||||
def findCCs(vertexes: RDD[(VertexId, String)], edges: RDD[Edge[String]], maxIterations: Int, cut:Int): RDD[ConnectedComponent] = {
|
||||
val graph: Graph[String, String] = Graph(vertexes, edges).partitionBy(PartitionStrategy.RandomVertexCut) //TODO remember to remove partitionby
|
||||
val cc = graph.connectedComponents(maxIterations).vertices
|
||||
|
||||
|
@ -22,15 +22,15 @@ object GraphProcessor {
|
|||
}
|
||||
}
|
||||
val connectedComponents = joinResult.groupByKey()
|
||||
.map[ConnectedComponent](cc => asConnectedComponent(cc, subEntity, cut))
|
||||
.map[ConnectedComponent](cc => asConnectedComponent(cc, cut))
|
||||
connectedComponents
|
||||
}
|
||||
|
||||
|
||||
|
||||
def asConnectedComponent(group: (VertexId, Iterable[String]), subEntity: String,cut:Int): ConnectedComponent = {
|
||||
def asConnectedComponent(group: (VertexId, Iterable[String]), cut:Int): ConnectedComponent = {
|
||||
val docs = group._2.toSet[String]
|
||||
val connectedComponent = new ConnectedComponent(JavaConversions.setAsJavaSet[String](docs), subEntity, cut);
|
||||
val connectedComponent = new ConnectedComponent(JavaConversions.setAsJavaSet[String](docs), cut);
|
||||
connectedComponent
|
||||
}
|
||||
|
||||
|
|
|
@ -1,7 +1,15 @@
|
|||
|
||||
package eu.dnetlib.dhp.oa.dedup.model;
|
||||
|
||||
import java.io.Serializable;
|
||||
import java.text.SimpleDateFormat;
|
||||
import java.util.*;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import org.apache.commons.lang3.StringUtils;
|
||||
|
||||
import com.google.common.collect.Sets;
|
||||
|
||||
import eu.dnetlib.dhp.oa.dedup.DatePicker;
|
||||
import eu.dnetlib.dhp.schema.common.EntityType;
|
||||
import eu.dnetlib.dhp.schema.common.ModelConstants;
|
||||
|
@ -9,12 +17,6 @@ import eu.dnetlib.dhp.schema.common.ModelSupport;
|
|||
import eu.dnetlib.dhp.schema.oaf.*;
|
||||
import eu.dnetlib.dhp.schema.oaf.utils.PidComparator;
|
||||
import eu.dnetlib.dhp.schema.oaf.utils.PidType;
|
||||
import org.apache.commons.lang3.StringUtils;
|
||||
|
||||
import java.io.Serializable;
|
||||
import java.text.SimpleDateFormat;
|
||||
import java.util.*;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
public class Identifier<T extends OafEntity> implements Serializable, Comparable<Identifier> {
|
||||
|
||||
|
|
|
@ -55,7 +55,6 @@ public class SparkDedupTest implements Serializable {
|
|||
private static String testOutputBasePath;
|
||||
private static String testDedupGraphBasePath;
|
||||
private static final String testActionSetId = "test-orchestrator";
|
||||
private static String testDedupAssertionsBasePath;
|
||||
|
||||
@BeforeAll
|
||||
public static void cleanUp() throws IOException, URISyntaxException {
|
||||
|
@ -70,10 +69,6 @@ public class SparkDedupTest implements Serializable {
|
|||
testDedupGraphBasePath = createTempDirectory(SparkDedupTest.class.getSimpleName() + "-")
|
||||
.toAbsolutePath()
|
||||
.toString();
|
||||
testDedupAssertionsBasePath = Paths
|
||||
.get(SparkDedupTest.class.getResource("/eu/dnetlib/dhp/dedup/assertions").toURI())
|
||||
.toFile()
|
||||
.getAbsolutePath();
|
||||
|
||||
FileUtils.deleteDirectory(new File(testOutputBasePath));
|
||||
FileUtils.deleteDirectory(new File(testDedupGraphBasePath));
|
||||
|
|
Loading…
Reference in New Issue