various refactorings on the dnet-dedup-openaire workflow
This commit is contained in:
parent
038ac7afd7
commit
71813795f6
|
@ -4,11 +4,14 @@ import com.fasterxml.jackson.databind.DeserializationFeature;
|
||||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||||
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
|
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
|
||||||
import eu.dnetlib.dhp.common.HdfsSupport;
|
import eu.dnetlib.dhp.common.HdfsSupport;
|
||||||
|
import eu.dnetlib.dhp.schema.common.ModelSupport;
|
||||||
import eu.dnetlib.dhp.schema.oaf.*;
|
import eu.dnetlib.dhp.schema.oaf.*;
|
||||||
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException;
|
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException;
|
||||||
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService;
|
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService;
|
||||||
import eu.dnetlib.pace.config.DedupConfig;
|
import eu.dnetlib.pace.config.DedupConfig;
|
||||||
import org.apache.spark.SparkConf;
|
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.apache.spark.sql.SparkSession;
|
||||||
import org.dom4j.Document;
|
import org.dom4j.Document;
|
||||||
import org.dom4j.DocumentException;
|
import org.dom4j.DocumentException;
|
||||||
|
@ -73,47 +76,21 @@ abstract class AbstractSparkAction implements Serializable {
|
||||||
|
|
||||||
abstract void run(ISLookUpService isLookUpService) throws DocumentException, IOException, ISLookUpException;
|
abstract void run(ISLookUpService isLookUpService) throws DocumentException, IOException, ISLookUpException;
|
||||||
|
|
||||||
protected static SparkSession getSparkSession(ArgumentApplicationParser parser) {
|
protected static SparkSession getSparkSession(SparkConf conf) {
|
||||||
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
|
|
||||||
});
|
|
||||||
|
|
||||||
return SparkSession
|
return SparkSession
|
||||||
.builder()
|
.builder()
|
||||||
.appName(SparkCreateSimRels.class.getSimpleName())
|
|
||||||
.master(parser.get("master"))
|
|
||||||
.config(conf)
|
.config(conf)
|
||||||
.getOrCreate();
|
.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) {
|
protected static void removeOutputDir(SparkSession spark, String path) {
|
||||||
HdfsSupport.remove(path, spark.sparkContext().hadoopConfiguration());
|
HdfsSupport.remove(path, spark.sparkContext().hadoopConfiguration());
|
||||||
}
|
}
|
||||||
|
|
|
@ -1,274 +1,95 @@
|
||||||
package eu.dnetlib.dhp.oa.dedup;
|
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 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.dhp.schema.oaf.*;
|
||||||
import eu.dnetlib.pace.config.DedupConfig;
|
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.JavaPairRDD;
|
||||||
import org.apache.spark.api.java.JavaRDD;
|
import org.apache.spark.api.java.function.FilterFunction;
|
||||||
import org.apache.spark.api.java.JavaSparkContext;
|
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.api.java.function.PairFunction;
|
||||||
|
import org.apache.spark.sql.Dataset;
|
||||||
import org.apache.spark.sql.Encoders;
|
import org.apache.spark.sql.Encoders;
|
||||||
import org.apache.spark.sql.SparkSession;
|
import org.apache.spark.sql.SparkSession;
|
||||||
import org.codehaus.jackson.map.ObjectMapper;
|
|
||||||
import scala.Tuple2;
|
import scala.Tuple2;
|
||||||
|
|
||||||
import java.util.Collection;
|
import java.util.Collection;
|
||||||
|
import java.util.Iterator;
|
||||||
|
|
||||||
public class DedupRecordFactory {
|
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();
|
long ts = System.currentTimeMillis();
|
||||||
|
|
||||||
//<id, json_entity>
|
//<id, json_entity>
|
||||||
final JavaPairRDD<String, String> inputJsonEntities = sc.textFile(entitiesInputPath)
|
Dataset<Tuple2<String, T>> entities = spark.read()
|
||||||
.mapToPair((PairFunction<String, String, String>) it ->
|
.textFile(entitiesInputPath)
|
||||||
new Tuple2<String, String>(MapDocumentUtil.getJPathString(dedupConf.getWf().getIdPath(), it), it)
|
.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
|
//<source, target>: source is the dedup_id, target is the id of the mergedIn
|
||||||
JavaPairRDD<String, String> mergeRels = spark
|
Dataset<Tuple2<String, String>> mergeRels = spark
|
||||||
.read().load(mergeRelsInputPath).as(Encoders.bean(Relation.class))
|
.read()
|
||||||
.where("relClass=='merges'")
|
.load(mergeRelsInputPath)
|
||||||
.javaRDD()
|
.as(Encoders.bean(Relation.class))
|
||||||
.mapToPair(
|
.where("relClass == 'merges'")
|
||||||
(PairFunction<Relation, String, String>) r ->
|
.map((MapFunction<Relation, Tuple2<String, String>>)
|
||||||
new Tuple2<String, String>(r.getTarget(), r.getSource())
|
r -> new Tuple2<>(r.getSource(), r.getTarget()), Encoders.tuple(Encoders.STRING(), Encoders.STRING()));
|
||||||
);
|
|
||||||
|
|
||||||
//<dedup_id, json_entity_merged>
|
//<dedup_id, json_entity_merged>
|
||||||
final JavaPairRDD<String, String> joinResult = mergeRels.join(inputJsonEntities).mapToPair((PairFunction<Tuple2<String, Tuple2<String, String>>, String, String>) Tuple2::_2);
|
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)
|
||||||
JavaPairRDD<String, Iterable<String>> sortedJoinResult = joinResult.groupByKey();
|
.map((MapFunction<Tuple2<Tuple2<String, String>, Tuple2<String, T>>, T>)
|
||||||
|
value -> value._2()._2(), Encoders.kryo(clazz))
|
||||||
switch (entityType) {
|
.groupByKey((MapFunction<T, String>) value -> value.getId(), Encoders.STRING())
|
||||||
case publication:
|
.mapGroups((MapGroupsFunction<String, T, T>)
|
||||||
return sortedJoinResult.map(p -> DedupRecordFactory.publicationMerger(p, ts));
|
(key, values) -> entityMerger(key, values, ts, clazz), Encoders.bean(clazz));
|
||||||
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;
|
|
||||||
}
|
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
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());
|
if (er.getDateofacceptance() != null) {
|
||||||
|
dates.add(r1.getDateofacceptance().getValue());
|
||||||
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);
|
|
||||||
}
|
}
|
||||||
o.mergeFrom(organization);
|
|
||||||
|
|
||||||
} catch (Exception exc) {
|
|
||||||
throw new RuntimeException(exc);
|
|
||||||
}
|
}
|
||||||
});
|
});
|
||||||
|
|
||||||
if (o.getDataInfo() == null)
|
if (ModelSupport.isSubClass(entity, Result.class)) {
|
||||||
{
|
((Result) entity).setDateofacceptance(DatePicker.pick(dates));
|
||||||
o.setDataInfo(new DataInfo());
|
}
|
||||||
|
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;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -1,74 +1,50 @@
|
||||||
package eu.dnetlib.dhp.oa.dedup;
|
package eu.dnetlib.dhp.oa.dedup;
|
||||||
|
|
||||||
|
import eu.dnetlib.dhp.oa.dedup.model.Block;
|
||||||
import eu.dnetlib.pace.config.DedupConfig;
|
import eu.dnetlib.pace.config.DedupConfig;
|
||||||
import eu.dnetlib.pace.model.MapDocument;
|
import eu.dnetlib.pace.model.MapDocument;
|
||||||
import eu.dnetlib.pace.util.BlockProcessor;
|
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.JavaPairRDD;
|
||||||
import org.apache.spark.api.java.JavaRDD;
|
|
||||||
import org.apache.spark.api.java.JavaSparkContext;
|
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.apache.spark.util.LongAccumulator;
|
||||||
import org.slf4j.Logger;
|
|
||||||
import org.slf4j.LoggerFactory;
|
|
||||||
import scala.Serializable;
|
import scala.Serializable;
|
||||||
import scala.Tuple2;
|
import scala.Tuple2;
|
||||||
|
|
||||||
import java.util.ArrayList;
|
|
||||||
import java.util.Comparator;
|
|
||||||
import java.util.List;
|
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.stream.Collectors;
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
public class Deduper implements Serializable {
|
public class Deduper implements Serializable {
|
||||||
|
|
||||||
private static final Logger log = LoggerFactory.getLogger(Deduper.class);
|
public static JavaPairRDD<String, String> computeRelations(JavaSparkContext context, JavaPairRDD<String, Block> blocks, DedupConfig config) {
|
||||||
|
|
||||||
public static JavaPairRDD<String, String> computeRelations(JavaSparkContext context, JavaPairRDD<String, List<MapDocument>> blocks, DedupConfig config) {
|
|
||||||
Map<String, LongAccumulator> accumulators = DedupUtility.constructAccumulator(config, context.sc());
|
Map<String, LongAccumulator> accumulators = DedupUtility.constructAccumulator(config, context.sc());
|
||||||
|
|
||||||
return blocks.flatMapToPair((PairFlatMapFunction<Tuple2<String, List<MapDocument>>, String, String>) it -> {
|
return blocks
|
||||||
try {
|
.flatMapToPair(it -> {
|
||||||
final SparkReporter reporter = new SparkReporter(accumulators);
|
final SparkReporter reporter = new SparkReporter(accumulators);
|
||||||
new BlockProcessor(config).processSortedBlock(it._1(), it._2(), reporter);
|
new BlockProcessor(config).processSortedBlock(it._1(), it._2().getDocuments(), reporter);
|
||||||
return reporter.getRelations().iterator();
|
return reporter.getRelations().iterator();
|
||||||
} catch (Exception e) {
|
})
|
||||||
throw new RuntimeException(it._2().get(0).getIdentifier(), e);
|
.mapToPair(it -> new Tuple2<>(it._1() + it._2(), it))
|
||||||
}
|
|
||||||
}).mapToPair(
|
|
||||||
(PairFunction<Tuple2<String, String>, String, Tuple2<String, String>>) item ->
|
|
||||||
new Tuple2<String, Tuple2<String, String>>(item._1() + item._2(), item))
|
|
||||||
.reduceByKey((a, b) -> a)
|
.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 String of = config.getWf().getOrderField();
|
||||||
final int maxQueueSize = config.getWf().getGroupMaxSize();
|
final int maxQueueSize = config.getWf().getGroupMaxSize();
|
||||||
|
|
||||||
return mapDocs
|
return mapDocs
|
||||||
//the reduce is just to be sure that we haven't document with same id
|
//the reduce is just to be sure that we haven't document with same id
|
||||||
.reduceByKey((a, b) -> a)
|
.reduceByKey((a, b) -> a)
|
||||||
.map(Tuple2::_2)
|
.map(Tuple2::_2)
|
||||||
//Clustering: from <id, doc> to List<groupkey,doc>
|
//Clustering: from <id, doc> to List<groupkey,doc>
|
||||||
.flatMapToPair((PairFlatMapFunction<MapDocument, String, List<MapDocument>>) a ->
|
.flatMap(a -> DedupUtility.getGroupingKeys(config, a)
|
||||||
DedupUtility.getGroupingKeys(config, a)
|
|
||||||
.stream()
|
.stream()
|
||||||
.map(it -> {
|
.map(it -> Block.from(it, a))
|
||||||
List<MapDocument> tmp = new ArrayList<>();
|
|
||||||
tmp.add(a);
|
|
||||||
return new Tuple2<>(it, tmp);
|
|
||||||
}
|
|
||||||
)
|
|
||||||
.collect(Collectors.toList())
|
.collect(Collectors.toList())
|
||||||
.iterator())
|
.iterator())
|
||||||
.reduceByKey((Function2<List<MapDocument>, List<MapDocument>, List<MapDocument>>) (v1, v2) -> {
|
.mapToPair(block -> new Tuple2<>(block.getKey(), block))
|
||||||
v1.addAll(v2);
|
.reduceByKey((b1, b2) -> Block.from(b1, b2, of, maxQueueSize));
|
||||||
v1.sort(Comparator.comparing(a -> a.getFieldMap().get(of).stringValue()));
|
|
||||||
if (v1.size() > maxQueueSize)
|
|
||||||
return new ArrayList<>(v1.subList(0, maxQueueSize));
|
|
||||||
return v1;
|
|
||||||
});
|
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
|
@ -1,15 +0,0 @@
|
||||||
package eu.dnetlib.dhp.oa.dedup;
|
|
||||||
|
|
||||||
public enum OafEntityType {
|
|
||||||
|
|
||||||
datasource,
|
|
||||||
organization,
|
|
||||||
project,
|
|
||||||
dataset,
|
|
||||||
otherresearchproduct,
|
|
||||||
software,
|
|
||||||
publication
|
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
}
|
|
|
@ -1,14 +1,21 @@
|
||||||
package eu.dnetlib.dhp.oa.dedup;
|
package eu.dnetlib.dhp.oa.dedup;
|
||||||
|
|
||||||
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
|
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.schema.oaf.OafEntity;
|
||||||
import eu.dnetlib.dhp.utils.ISLookupClientFactory;
|
import eu.dnetlib.dhp.utils.ISLookupClientFactory;
|
||||||
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException;
|
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException;
|
||||||
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService;
|
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService;
|
||||||
import eu.dnetlib.pace.config.DedupConfig;
|
import eu.dnetlib.pace.config.DedupConfig;
|
||||||
import org.apache.commons.io.IOUtils;
|
import org.apache.commons.io.IOUtils;
|
||||||
|
import org.apache.spark.SparkConf;
|
||||||
import org.apache.spark.api.java.JavaRDD;
|
import org.apache.spark.api.java.JavaRDD;
|
||||||
import org.apache.spark.api.java.JavaSparkContext;
|
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.apache.spark.sql.SparkSession;
|
||||||
import org.dom4j.DocumentException;
|
import org.dom4j.DocumentException;
|
||||||
import org.slf4j.Logger;
|
import org.slf4j.Logger;
|
||||||
|
@ -30,7 +37,12 @@ public class SparkCreateDedupRecord extends AbstractSparkAction {
|
||||||
SparkCreateSimRels.class.getResourceAsStream("/eu/dnetlib/dhp/oa/dedup/createDedupRecord_parameters.json")));
|
SparkCreateSimRels.class.getResourceAsStream("/eu/dnetlib/dhp/oa/dedup/createDedupRecord_parameters.json")));
|
||||||
parser.parseArgument(args);
|
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
|
@Override
|
||||||
|
@ -46,8 +58,6 @@ public class SparkCreateDedupRecord extends AbstractSparkAction {
|
||||||
log.info("actionSetId: '{}'", actionSetId);
|
log.info("actionSetId: '{}'", actionSetId);
|
||||||
log.info("workingPath: '{}'", workingPath);
|
log.info("workingPath: '{}'", workingPath);
|
||||||
|
|
||||||
final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext());
|
|
||||||
|
|
||||||
for (DedupConfig dedupConf: getConfigurations(isLookUpService, actionSetId)) {
|
for (DedupConfig dedupConf: getConfigurations(isLookUpService, actionSetId)) {
|
||||||
String subEntity = dedupConf.getWf().getSubEntityValue();
|
String subEntity = dedupConf.getWf().getSubEntityValue();
|
||||||
log.info("Creating deduprecords for: '{}'", subEntity);
|
log.info("Creating deduprecords for: '{}'", subEntity);
|
||||||
|
@ -57,11 +67,14 @@ public class SparkCreateDedupRecord extends AbstractSparkAction {
|
||||||
|
|
||||||
final String mergeRelPath = DedupUtility.createMergeRelPath(workingPath, actionSetId, subEntity);
|
final String mergeRelPath = DedupUtility.createMergeRelPath(workingPath, actionSetId, subEntity);
|
||||||
final String entityPath = DedupUtility.createEntityPath(graphBasePath, 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 -> OBJECT_MAPPER.writeValueAsString(r)).saveAsTextFile(outputPath);
|
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);
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -4,6 +4,7 @@ import com.google.common.hash.Hashing;
|
||||||
import eu.dnetlib.dhp.oa.dedup.graph.ConnectedComponent;
|
import eu.dnetlib.dhp.oa.dedup.graph.ConnectedComponent;
|
||||||
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
|
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
|
||||||
import eu.dnetlib.dhp.oa.dedup.graph.GraphProcessor;
|
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.DataInfo;
|
||||||
import eu.dnetlib.dhp.schema.oaf.Qualifier;
|
import eu.dnetlib.dhp.schema.oaf.Qualifier;
|
||||||
import eu.dnetlib.dhp.schema.oaf.Relation;
|
import eu.dnetlib.dhp.schema.oaf.Relation;
|
||||||
|
@ -13,8 +14,8 @@ import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService;
|
||||||
import eu.dnetlib.pace.config.DedupConfig;
|
import eu.dnetlib.pace.config.DedupConfig;
|
||||||
import eu.dnetlib.pace.util.MapDocumentUtil;
|
import eu.dnetlib.pace.util.MapDocumentUtil;
|
||||||
import org.apache.commons.io.IOUtils;
|
import org.apache.commons.io.IOUtils;
|
||||||
|
import org.apache.spark.SparkConf;
|
||||||
import org.apache.spark.api.java.JavaPairRDD;
|
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.JavaSparkContext;
|
||||||
import org.apache.spark.api.java.function.PairFunction;
|
import org.apache.spark.api.java.function.PairFunction;
|
||||||
import org.apache.spark.graphx.Edge;
|
import org.apache.spark.graphx.Edge;
|
||||||
|
@ -52,7 +53,11 @@ public class SparkCreateMergeRels extends AbstractSparkAction {
|
||||||
final String isLookUpUrl = parser.get("isLookUpUrl");
|
final String isLookUpUrl = parser.get("isLookUpUrl");
|
||||||
log.info("isLookupUrl {}", isLookUpUrl);
|
log.info("isLookupUrl {}", isLookUpUrl);
|
||||||
|
|
||||||
new SparkCreateMergeRels(parser, getSparkSession(parser)).run(ISLookupClientFactory.getLookUpService(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
|
@Override
|
||||||
|
@ -79,32 +84,30 @@ public class SparkCreateMergeRels extends AbstractSparkAction {
|
||||||
log.info("Max iterations {}", maxIterations);
|
log.info("Max iterations {}", maxIterations);
|
||||||
|
|
||||||
final String mergeRelPath = DedupUtility.createMergeRelPath(workingPath, actionSetId, subEntity);
|
final String mergeRelPath = DedupUtility.createMergeRelPath(workingPath, actionSetId, subEntity);
|
||||||
|
|
||||||
final JavaPairRDD<Object, String> vertexes = sc.textFile(graphBasePath + "/" + subEntity)
|
final JavaPairRDD<Object, String> vertexes = sc.textFile(graphBasePath + "/" + subEntity)
|
||||||
.map(s -> MapDocumentUtil.getJPathString(dedupConf.getWf().getIdPath(), s))
|
.map(s -> MapDocumentUtil.getJPathString(dedupConf.getWf().getIdPath(), s))
|
||||||
.mapToPair((PairFunction<String, Object, String>)
|
.mapToPair((PairFunction<String, Object, String>) s -> new Tuple2<>(hash(s), s));
|
||||||
s -> new Tuple2<>(getHashcode(s), s));
|
|
||||||
|
|
||||||
final Dataset<Relation> similarityRelations = spark
|
final RDD<Edge<String>> edgeRdd = spark
|
||||||
.read()
|
.read()
|
||||||
.load(DedupUtility.createSimRelPath(workingPath, actionSetId, subEntity))
|
.load(DedupUtility.createSimRelPath(workingPath, actionSetId, subEntity))
|
||||||
.as(Encoders.bean(Relation.class));
|
.as(Encoders.bean(Relation.class))
|
||||||
|
|
||||||
final RDD<Edge<String>> edgeRdd = similarityRelations
|
|
||||||
.javaRDD()
|
.javaRDD()
|
||||||
.map(it -> new Edge<>(getHashcode(it.getSource()), getHashcode(it.getTarget()), it.getRelClass()))
|
.map(it -> new Edge<>(hash(it.getSource()), hash(it.getTarget()), it.getRelClass()))
|
||||||
.rdd();
|
.rdd();
|
||||||
|
|
||||||
final RDD<Relation> connectedComponents = GraphProcessor.findCCs(vertexes.rdd(), edgeRdd, maxIterations)
|
final Dataset<Relation> mergeRels = spark
|
||||||
|
.createDataset(GraphProcessor.findCCs(vertexes.rdd(), edgeRdd, maxIterations)
|
||||||
.toJavaRDD()
|
.toJavaRDD()
|
||||||
.filter(k -> k.getDocIds().size() > 1)
|
.filter(k -> k.getDocIds().size() > 1)
|
||||||
.flatMap(cc -> ccToMergeRel(cc, dedupConf))
|
.flatMap(cc -> ccToMergeRel(cc, dedupConf))
|
||||||
.rdd();
|
.rdd(), Encoders.bean(Relation.class));
|
||||||
|
|
||||||
spark
|
mergeRels
|
||||||
.createDataset(connectedComponents, Encoders.bean(Relation.class))
|
|
||||||
.write()
|
.write()
|
||||||
.mode(SaveMode.Append)
|
.mode(SaveMode.Append)
|
||||||
.save(mergeRelPath);
|
.parquet(mergeRelPath);
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -148,7 +151,7 @@ public class SparkCreateMergeRels extends AbstractSparkAction {
|
||||||
return r;
|
return r;
|
||||||
}
|
}
|
||||||
|
|
||||||
public static long getHashcode(final String id) {
|
public static long hash(final String id) {
|
||||||
return Hashing.murmur3_128().hashString(id).asLong();
|
return Hashing.murmur3_128().hashString(id).asLong();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -1,23 +1,26 @@
|
||||||
package eu.dnetlib.dhp.oa.dedup;
|
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.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.DataInfo;
|
||||||
import eu.dnetlib.dhp.schema.oaf.Relation;
|
import eu.dnetlib.dhp.schema.oaf.Relation;
|
||||||
import eu.dnetlib.dhp.utils.ISLookupClientFactory;
|
import eu.dnetlib.dhp.utils.ISLookupClientFactory;
|
||||||
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException;
|
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException;
|
||||||
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService;
|
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService;
|
||||||
import eu.dnetlib.pace.config.DedupConfig;
|
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.model.MapDocument;
|
||||||
import eu.dnetlib.pace.util.MapDocumentUtil;
|
import eu.dnetlib.pace.util.MapDocumentUtil;
|
||||||
import org.apache.commons.io.IOUtils;
|
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.JavaPairRDD;
|
||||||
import org.apache.spark.api.java.JavaRDD;
|
import org.apache.spark.api.java.JavaRDD;
|
||||||
import org.apache.spark.api.java.JavaSparkContext;
|
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.api.java.function.PairFunction;
|
||||||
|
import org.apache.spark.sql.Dataset;
|
||||||
import org.apache.spark.sql.Encoders;
|
import org.apache.spark.sql.Encoders;
|
||||||
import org.apache.spark.sql.SaveMode;
|
import org.apache.spark.sql.SaveMode;
|
||||||
import org.apache.spark.sql.SparkSession;
|
import org.apache.spark.sql.SparkSession;
|
||||||
|
@ -27,7 +30,6 @@ import org.slf4j.LoggerFactory;
|
||||||
import scala.Tuple2;
|
import scala.Tuple2;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.List;
|
|
||||||
|
|
||||||
public class SparkCreateSimRels extends AbstractSparkAction {
|
public class SparkCreateSimRels extends AbstractSparkAction {
|
||||||
|
|
||||||
|
@ -43,7 +45,17 @@ public class SparkCreateSimRels extends AbstractSparkAction {
|
||||||
SparkCreateSimRels.class.getResourceAsStream("/eu/dnetlib/dhp/oa/dedup/createSimRels_parameters.json")));
|
SparkCreateSimRels.class.getResourceAsStream("/eu/dnetlib/dhp/oa/dedup/createSimRels_parameters.json")));
|
||||||
parser.parseArgument(args);
|
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
|
@Override
|
||||||
|
@ -60,8 +72,6 @@ public class SparkCreateSimRels extends AbstractSparkAction {
|
||||||
log.info("actionSetId: '{}'", actionSetId);
|
log.info("actionSetId: '{}'", actionSetId);
|
||||||
log.info("workingPath: '{}'", workingPath);
|
log.info("workingPath: '{}'", workingPath);
|
||||||
|
|
||||||
final JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext());
|
|
||||||
|
|
||||||
//for each dedup configuration
|
//for each dedup configuration
|
||||||
for (DedupConfig dedupConf: getConfigurations(isLookUpService, actionSetId)) {
|
for (DedupConfig dedupConf: getConfigurations(isLookUpService, actionSetId)) {
|
||||||
|
|
||||||
|
@ -72,29 +82,30 @@ public class SparkCreateSimRels extends AbstractSparkAction {
|
||||||
final String outputPath = DedupUtility.createSimRelPath(workingPath, actionSetId, subEntity);
|
final String outputPath = DedupUtility.createSimRelPath(workingPath, actionSetId, subEntity);
|
||||||
removeOutputDir(spark, outputPath);
|
removeOutputDir(spark, outputPath);
|
||||||
|
|
||||||
JavaPairRDD<String, MapDocument> mapDocument = sc.textFile(DedupUtility.createEntityPath(graphBasePath, subEntity))
|
JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext());
|
||||||
.mapToPair((PairFunction<String, String, MapDocument>) s -> {
|
|
||||||
|
JavaPairRDD<String, MapDocument> mapDocuments = sc.textFile(DedupUtility.createEntityPath(graphBasePath, subEntity))
|
||||||
|
.mapToPair((PairFunction<String, String, MapDocument>) s -> {
|
||||||
MapDocument d = MapDocumentUtil.asMapDocumentWithJPath(dedupConf, s);
|
MapDocument d = MapDocumentUtil.asMapDocumentWithJPath(dedupConf, s);
|
||||||
return new Tuple2<>(d.getIdentifier(), d);
|
return new Tuple2<>(d.getIdentifier(), d);
|
||||||
});
|
});
|
||||||
|
|
||||||
//create blocks for deduplication
|
//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
|
//create relations by comparing only elements in the same group
|
||||||
final JavaPairRDD<String, String> dedupRels = Deduper.computeRelations(sc, blocks, dedupConf);
|
JavaRDD<Relation> relations = Deduper.computeRelations(sc, blocks, dedupConf)
|
||||||
|
.map(t -> createSimRel(t._1(), t._2(), entity));
|
||||||
JavaRDD<Relation> relationsRDD = dedupRels.map(r -> createSimRel(r._1(), r._2(), entity));
|
|
||||||
|
|
||||||
//save the simrel in the workingdir
|
//save the simrel in the workingdir
|
||||||
spark.createDataset(relationsRDD.rdd(), Encoders.bean(Relation.class))
|
spark.createDataset(relations.rdd(), Encoders.bean(Relation.class))
|
||||||
.write()
|
.write()
|
||||||
.mode(SaveMode.Append)
|
.mode(SaveMode.Append)
|
||||||
.save(outputPath);
|
.save(outputPath);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
public Relation createSimRel(String source, String target, String entity) {
|
private Relation createSimRel(String source, String target, String entity) {
|
||||||
final Relation r = new Relation();
|
final Relation r = new Relation();
|
||||||
r.setSource(source);
|
r.setSource(source);
|
||||||
r.setTarget(target);
|
r.setTarget(target);
|
||||||
|
@ -102,7 +113,7 @@ public class SparkCreateSimRels extends AbstractSparkAction {
|
||||||
r.setRelClass("isSimilarTo");
|
r.setRelClass("isSimilarTo");
|
||||||
r.setDataInfo(new DataInfo());
|
r.setDataInfo(new DataInfo());
|
||||||
|
|
||||||
switch(entity){
|
switch(entity) {
|
||||||
case "result":
|
case "result":
|
||||||
r.setRelType("resultResult");
|
r.setRelType("resultResult");
|
||||||
break;
|
break;
|
||||||
|
|
|
@ -3,6 +3,7 @@ package eu.dnetlib.dhp.oa.dedup;
|
||||||
import com.fasterxml.jackson.databind.DeserializationFeature;
|
import com.fasterxml.jackson.databind.DeserializationFeature;
|
||||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||||
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
|
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.DataInfo;
|
||||||
import eu.dnetlib.dhp.schema.oaf.Relation;
|
import eu.dnetlib.dhp.schema.oaf.Relation;
|
||||||
import eu.dnetlib.dhp.utils.ISLookupClientFactory;
|
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.commons.io.IOUtils;
|
||||||
import org.apache.hadoop.fs.FileSystem;
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
import org.apache.hadoop.fs.Path;
|
import org.apache.hadoop.fs.Path;
|
||||||
|
import org.apache.spark.SparkConf;
|
||||||
import org.apache.spark.api.java.function.MapFunction;
|
import org.apache.spark.api.java.function.MapFunction;
|
||||||
import org.apache.spark.sql.Dataset;
|
import org.apache.spark.sql.*;
|
||||||
import org.apache.spark.sql.Encoders;
|
|
||||||
import org.apache.spark.sql.Row;
|
|
||||||
import org.apache.spark.sql.SparkSession;
|
|
||||||
import org.slf4j.Logger;
|
import org.slf4j.Logger;
|
||||||
import org.slf4j.LoggerFactory;
|
import org.slf4j.LoggerFactory;
|
||||||
import scala.Tuple2;
|
import scala.Tuple2;
|
||||||
|
@ -43,7 +42,11 @@ public class SparkPropagateRelation extends AbstractSparkAction {
|
||||||
|
|
||||||
parser.parseArgument(args);
|
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")));
|
.run(ISLookupClientFactory.getLookUpService(parser.get("isLookUpUrl")));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -90,7 +93,7 @@ public class SparkPropagateRelation extends AbstractSparkAction {
|
||||||
processDataset(rels, mergedIds, FieldType.SOURCE, getDeletedFn()),
|
processDataset(rels, mergedIds, FieldType.SOURCE, getDeletedFn()),
|
||||||
mergedIds, FieldType.TARGET, getDeletedFn());
|
mergedIds, FieldType.TARGET, getDeletedFn());
|
||||||
|
|
||||||
save(newRels.union(updated), outputRelationPath);
|
save(newRels.union(updated), outputRelationPath, SaveMode.Overwrite);
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -164,26 +167,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) {
|
private static boolean containsDedup(final Relation r) {
|
||||||
return r.getSource().toLowerCase().contains("dedup") || r.getTarget().toLowerCase().contains("dedup");
|
return r.getSource().toLowerCase().contains("dedup") || r.getTarget().toLowerCase().contains("dedup");
|
||||||
}
|
}
|
||||||
|
|
|
@ -13,9 +13,9 @@ import java.util.Map;
|
||||||
|
|
||||||
public class SparkReporter implements Serializable, Reporter {
|
public class SparkReporter implements Serializable, Reporter {
|
||||||
|
|
||||||
final List<Tuple2<String, String>> relations = new ArrayList<>();
|
private final List<Tuple2<String, String>> relations = new ArrayList<>();
|
||||||
private static final Log log = LogFactory.getLog(SparkReporter.class);
|
|
||||||
Map<String, LongAccumulator> accumulators;
|
private Map<String, LongAccumulator> accumulators;
|
||||||
|
|
||||||
public SparkReporter(Map<String, LongAccumulator> accumulators){
|
public SparkReporter(Map<String, LongAccumulator> accumulators){
|
||||||
this.accumulators = accumulators;
|
this.accumulators = accumulators;
|
||||||
|
|
|
@ -1,13 +1,17 @@
|
||||||
package eu.dnetlib.dhp.oa.dedup;
|
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.application.ArgumentApplicationParser;
|
||||||
import eu.dnetlib.dhp.schema.common.EntityType;
|
import eu.dnetlib.dhp.oa.dedup.model.Block;
|
||||||
import eu.dnetlib.dhp.schema.common.ModelSupport;
|
import eu.dnetlib.dhp.schema.common.ModelSupport;
|
||||||
import eu.dnetlib.dhp.schema.oaf.*;
|
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.dhp.utils.ISLookupClientFactory;
|
||||||
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService;
|
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 eu.dnetlib.pace.util.MapDocumentUtil;
|
||||||
import org.apache.commons.io.IOUtils;
|
import org.apache.commons.io.IOUtils;
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
@ -15,6 +19,7 @@ import org.apache.hadoop.fs.FileStatus;
|
||||||
import org.apache.hadoop.fs.FileSystem;
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
import org.apache.hadoop.fs.Path;
|
import org.apache.hadoop.fs.Path;
|
||||||
import org.apache.hadoop.io.compress.GzipCodec;
|
import org.apache.hadoop.io.compress.GzipCodec;
|
||||||
|
import org.apache.spark.SparkConf;
|
||||||
import org.apache.spark.api.java.JavaPairRDD;
|
import org.apache.spark.api.java.JavaPairRDD;
|
||||||
import org.apache.spark.api.java.JavaRDD;
|
import org.apache.spark.api.java.JavaRDD;
|
||||||
import org.apache.spark.api.java.JavaSparkContext;
|
import org.apache.spark.api.java.JavaSparkContext;
|
||||||
|
@ -28,7 +33,6 @@ import org.slf4j.LoggerFactory;
|
||||||
import scala.Tuple2;
|
import scala.Tuple2;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.Map;
|
|
||||||
|
|
||||||
public class SparkUpdateEntity extends AbstractSparkAction {
|
public class SparkUpdateEntity extends AbstractSparkAction {
|
||||||
|
|
||||||
|
@ -43,10 +47,16 @@ public class SparkUpdateEntity extends AbstractSparkAction {
|
||||||
public static void main(String[] args) throws Exception {
|
public static void main(String[] args) throws Exception {
|
||||||
ArgumentApplicationParser parser = new ArgumentApplicationParser(
|
ArgumentApplicationParser parser = new ArgumentApplicationParser(
|
||||||
IOUtils.toString(
|
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);
|
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());
|
||||||
|
|
||||||
|
new SparkUpdateEntity(parser, getSparkSession(conf))
|
||||||
|
.run(ISLookupClientFactory.getLookUpService(parser.get("isLookUpUrl")));
|
||||||
}
|
}
|
||||||
|
|
||||||
public void run(ISLookUpService isLookUpService) throws IOException {
|
public void run(ISLookUpService isLookUpService) throws IOException {
|
||||||
|
|
|
@ -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;
|
||||||
|
}
|
||||||
|
}
|
|
@ -91,7 +91,6 @@
|
||||||
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
|
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
|
||||||
--conf spark.sql.shuffle.partitions=7680
|
--conf spark.sql.shuffle.partitions=7680
|
||||||
</spark-opts>
|
</spark-opts>
|
||||||
<arg>-mt</arg><arg>yarn</arg>
|
|
||||||
<arg>--i</arg><arg>${graphBasePath}</arg>
|
<arg>--i</arg><arg>${graphBasePath}</arg>
|
||||||
<arg>--o</arg><arg>${dedupGraphPath}</arg>
|
<arg>--o</arg><arg>${dedupGraphPath}</arg>
|
||||||
<arg>--w</arg><arg>${workingPath}</arg>
|
<arg>--w</arg><arg>${workingPath}</arg>
|
||||||
|
|
|
@ -1,10 +1,4 @@
|
||||||
[
|
[
|
||||||
{
|
|
||||||
"paramName": "mt",
|
|
||||||
"paramLongName": "master",
|
|
||||||
"paramDescription": "should be local or yarn",
|
|
||||||
"paramRequired": true
|
|
||||||
},
|
|
||||||
{
|
{
|
||||||
"paramName": "asi",
|
"paramName": "asi",
|
||||||
"paramLongName": "actionSetId",
|
"paramLongName": "actionSetId",
|
||||||
|
|
|
@ -1,10 +1,4 @@
|
||||||
[
|
[
|
||||||
{
|
|
||||||
"paramName": "mt",
|
|
||||||
"paramLongName": "master",
|
|
||||||
"paramDescription": "should be local or yarn",
|
|
||||||
"paramRequired": true
|
|
||||||
},
|
|
||||||
{
|
{
|
||||||
"paramName": "i",
|
"paramName": "i",
|
||||||
"paramLongName": "graphBasePath",
|
"paramLongName": "graphBasePath",
|
||||||
|
|
|
@ -1,10 +1,4 @@
|
||||||
[
|
[
|
||||||
{
|
|
||||||
"paramName": "mt",
|
|
||||||
"paramLongName": "master",
|
|
||||||
"paramDescription": "should be local or yarn",
|
|
||||||
"paramRequired": true
|
|
||||||
},
|
|
||||||
{
|
{
|
||||||
"paramName": "la",
|
"paramName": "la",
|
||||||
"paramLongName": "isLookUpUrl",
|
"paramLongName": "isLookUpUrl",
|
||||||
|
|
|
@ -1,10 +1,4 @@
|
||||||
[
|
[
|
||||||
{
|
|
||||||
"paramName": "mt",
|
|
||||||
"paramLongName": "master",
|
|
||||||
"paramDescription": "should be local or yarn",
|
|
||||||
"paramRequired": true
|
|
||||||
},
|
|
||||||
{
|
{
|
||||||
"paramName": "i",
|
"paramName": "i",
|
||||||
"paramLongName": "graphBasePath",
|
"paramLongName": "graphBasePath",
|
||||||
|
|
|
@ -98,7 +98,6 @@
|
||||||
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
|
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
|
||||||
--conf spark.sql.shuffle.partitions=3840
|
--conf spark.sql.shuffle.partitions=3840
|
||||||
</spark-opts>
|
</spark-opts>
|
||||||
<arg>-mt</arg><arg>yarn</arg>
|
|
||||||
<arg>--i</arg><arg>${graphBasePath}</arg>
|
<arg>--i</arg><arg>${graphBasePath}</arg>
|
||||||
<arg>--la</arg><arg>${isLookUpUrl}</arg>
|
<arg>--la</arg><arg>${isLookUpUrl}</arg>
|
||||||
<arg>--asi</arg><arg>${actionSetId}</arg>
|
<arg>--asi</arg><arg>${actionSetId}</arg>
|
||||||
|
@ -125,7 +124,6 @@
|
||||||
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
|
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
|
||||||
--conf spark.sql.shuffle.partitions=3840
|
--conf spark.sql.shuffle.partitions=3840
|
||||||
</spark-opts>
|
</spark-opts>
|
||||||
<arg>-mt</arg><arg>yarn</arg>
|
|
||||||
<arg>--i</arg><arg>${graphBasePath}</arg>
|
<arg>--i</arg><arg>${graphBasePath}</arg>
|
||||||
<arg>--w</arg><arg>${workingPath}</arg>
|
<arg>--w</arg><arg>${workingPath}</arg>
|
||||||
<arg>--la</arg><arg>${isLookUpUrl}</arg>
|
<arg>--la</arg><arg>${isLookUpUrl}</arg>
|
||||||
|
@ -152,7 +150,6 @@
|
||||||
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
|
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
|
||||||
--conf spark.sql.shuffle.partitions=3840
|
--conf spark.sql.shuffle.partitions=3840
|
||||||
</spark-opts>
|
</spark-opts>
|
||||||
<arg>-mt</arg><arg>yarn</arg>
|
|
||||||
<arg>--i</arg><arg>${graphBasePath}</arg>
|
<arg>--i</arg><arg>${graphBasePath}</arg>
|
||||||
<arg>--w</arg><arg>${workingPath}</arg>
|
<arg>--w</arg><arg>${workingPath}</arg>
|
||||||
<arg>--la</arg><arg>${isLookUpUrl}</arg>
|
<arg>--la</arg><arg>${isLookUpUrl}</arg>
|
||||||
|
@ -179,7 +176,6 @@
|
||||||
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
|
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
|
||||||
--conf spark.sql.shuffle.partitions=3840
|
--conf spark.sql.shuffle.partitions=3840
|
||||||
</spark-opts>
|
</spark-opts>
|
||||||
<arg>-mt</arg><arg>yarn</arg>
|
|
||||||
<arg>--i</arg><arg>${graphBasePath}</arg>
|
<arg>--i</arg><arg>${graphBasePath}</arg>
|
||||||
<arg>--w</arg><arg>${workingPath}</arg>
|
<arg>--w</arg><arg>${workingPath}</arg>
|
||||||
<arg>--o</arg><arg>${dedupGraphPath}</arg>
|
<arg>--o</arg><arg>${dedupGraphPath}</arg>
|
||||||
|
|
|
@ -1,10 +1,4 @@
|
||||||
[
|
[
|
||||||
{
|
|
||||||
"paramName": "mt",
|
|
||||||
"paramLongName": "master",
|
|
||||||
"paramDescription": "should be local or yarn",
|
|
||||||
"paramRequired": true
|
|
||||||
},
|
|
||||||
{
|
{
|
||||||
"paramName": "i",
|
"paramName": "i",
|
||||||
"paramLongName": "graphBasePath",
|
"paramLongName": "graphBasePath",
|
||||||
|
|
Loading…
Reference in New Issue