Do no longer use dedupId information from pivotHistory Database
This commit is contained in:
parent
02636e802c
commit
1287315ffb
|
@ -1,24 +1,23 @@
|
||||||
|
|
||||||
package eu.dnetlib.dhp.oa.dedup;
|
package eu.dnetlib.dhp.oa.dedup;
|
||||||
|
|
||||||
import com.google.common.hash.Hashing;
|
import static eu.dnetlib.dhp.schema.common.ModelConstants.DNET_PROVENANCE_ACTIONS;
|
||||||
import com.kwartile.lib.cc.ConnectedComponent;
|
import static eu.dnetlib.dhp.schema.common.ModelConstants.PROVENANCE_DEDUP;
|
||||||
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
|
import static org.apache.spark.sql.functions.*;
|
||||||
import eu.dnetlib.dhp.schema.common.EntityType;
|
|
||||||
import eu.dnetlib.dhp.schema.common.ModelConstants;
|
import java.io.IOException;
|
||||||
import eu.dnetlib.dhp.schema.common.ModelSupport;
|
import java.time.LocalDate;
|
||||||
import eu.dnetlib.dhp.schema.oaf.*;
|
import java.util.ArrayList;
|
||||||
import eu.dnetlib.dhp.schema.oaf.utils.PidType;
|
import java.util.Arrays;
|
||||||
import eu.dnetlib.dhp.utils.ISLookupClientFactory;
|
import java.util.Collections;
|
||||||
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException;
|
import java.util.Optional;
|
||||||
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService;
|
|
||||||
import eu.dnetlib.pace.config.DedupConfig;
|
|
||||||
import org.apache.commons.io.IOUtils;
|
import org.apache.commons.io.IOUtils;
|
||||||
import org.apache.commons.lang3.StringUtils;
|
import org.apache.commons.lang3.StringUtils;
|
||||||
import org.apache.spark.SparkConf;
|
import org.apache.spark.SparkConf;
|
||||||
import org.apache.spark.api.java.function.FlatMapFunction;
|
import org.apache.spark.api.java.function.FlatMapFunction;
|
||||||
import org.apache.spark.sql.Dataset;
|
|
||||||
import org.apache.spark.sql.*;
|
import org.apache.spark.sql.*;
|
||||||
|
import org.apache.spark.sql.Dataset;
|
||||||
import org.apache.spark.sql.catalyst.encoders.RowEncoder;
|
import org.apache.spark.sql.catalyst.encoders.RowEncoder;
|
||||||
import org.apache.spark.sql.expressions.UserDefinedFunction;
|
import org.apache.spark.sql.expressions.UserDefinedFunction;
|
||||||
import org.apache.spark.sql.expressions.Window;
|
import org.apache.spark.sql.expressions.Window;
|
||||||
|
@ -29,20 +28,23 @@ import org.dom4j.DocumentException;
|
||||||
import org.slf4j.Logger;
|
import org.slf4j.Logger;
|
||||||
import org.slf4j.LoggerFactory;
|
import org.slf4j.LoggerFactory;
|
||||||
import org.xml.sax.SAXException;
|
import org.xml.sax.SAXException;
|
||||||
|
|
||||||
|
import com.google.common.hash.Hashing;
|
||||||
|
import com.kwartile.lib.cc.ConnectedComponent;
|
||||||
|
|
||||||
|
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
|
||||||
|
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.*;
|
||||||
|
import eu.dnetlib.dhp.schema.oaf.utils.PidType;
|
||||||
|
import eu.dnetlib.dhp.utils.ISLookupClientFactory;
|
||||||
|
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException;
|
||||||
|
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService;
|
||||||
|
import eu.dnetlib.pace.config.DedupConfig;
|
||||||
import scala.Tuple3;
|
import scala.Tuple3;
|
||||||
import scala.collection.JavaConversions;
|
import scala.collection.JavaConversions;
|
||||||
|
|
||||||
import java.io.IOException;
|
|
||||||
import java.time.LocalDate;
|
|
||||||
import java.util.ArrayList;
|
|
||||||
import java.util.Arrays;
|
|
||||||
import java.util.Collections;
|
|
||||||
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 static org.apache.spark.sql.functions.*;
|
|
||||||
|
|
||||||
public class SparkCreateMergeRels extends AbstractSparkAction {
|
public class SparkCreateMergeRels extends AbstractSparkAction {
|
||||||
|
|
||||||
private static final Logger log = LoggerFactory.getLogger(SparkCreateMergeRels.class);
|
private static final Logger log = LoggerFactory.getLogger(SparkCreateMergeRels.class);
|
||||||
|
@ -121,6 +123,7 @@ public class SparkCreateMergeRels extends AbstractSparkAction {
|
||||||
.distinct()
|
.distinct()
|
||||||
.withColumn("vertexId", hashUDF.apply(functions.col("id")));
|
.withColumn("vertexId", hashUDF.apply(functions.col("id")));
|
||||||
|
|
||||||
|
// transform simrels into pairs of numeric ids
|
||||||
final Dataset<Row> edges = spark
|
final Dataset<Row> edges = spark
|
||||||
.read()
|
.read()
|
||||||
.load(DedupUtility.createSimRelPath(workingPath, actionSetId, subEntity))
|
.load(DedupUtility.createSimRelPath(workingPath, actionSetId, subEntity))
|
||||||
|
@ -128,27 +131,34 @@ public class SparkCreateMergeRels extends AbstractSparkAction {
|
||||||
.withColumn("source", hashUDF.apply(functions.col("source")))
|
.withColumn("source", hashUDF.apply(functions.col("source")))
|
||||||
.withColumn("target", hashUDF.apply(functions.col("target")));
|
.withColumn("target", hashUDF.apply(functions.col("target")));
|
||||||
|
|
||||||
|
// resolve connected components
|
||||||
|
// ("vertexId", "groupId")
|
||||||
Dataset<Row> cliques = ConnectedComponent
|
Dataset<Row> cliques = ConnectedComponent
|
||||||
.runOnPairs(edges, 50, spark);
|
.runOnPairs(edges, 50, spark);
|
||||||
|
|
||||||
|
// transform "vertexId" back to its original string value
|
||||||
|
// groupId is kept numeric as its string value is not used
|
||||||
|
// ("id", "groupId")
|
||||||
Dataset<Row> rawMergeRels = cliques
|
Dataset<Row> rawMergeRels = cliques
|
||||||
.join(vertexIdMap, JavaConversions.asScalaBuffer(Collections.singletonList("vertexId")), "inner")
|
.join(vertexIdMap, JavaConversions.asScalaBuffer(Collections.singletonList("vertexId")), "inner")
|
||||||
.drop("vertexId")
|
.drop("vertexId")
|
||||||
.distinct();
|
.distinct();
|
||||||
|
|
||||||
|
// empty dataframe if historydatabase is not used
|
||||||
Dataset<Row> pivotHistory = spark
|
Dataset<Row> pivotHistory = spark
|
||||||
.createDataset(
|
.createDataset(
|
||||||
Collections.emptyList(),
|
Collections.emptyList(),
|
||||||
RowEncoder
|
RowEncoder
|
||||||
.apply(StructType.fromDDL("id STRING, firstUsage STRING, lastUsage STRING, dedupId STRING")));
|
.apply(StructType.fromDDL("id STRING, lastUsage STRING")));
|
||||||
|
|
||||||
if (StringUtils.isNotBlank(pivotHistoryDatabase)) {
|
if (StringUtils.isNotBlank(pivotHistoryDatabase)) {
|
||||||
pivotHistory = spark
|
pivotHistory = spark
|
||||||
.read()
|
.read()
|
||||||
.table(pivotHistoryDatabase + "." + subEntity)
|
.table(pivotHistoryDatabase + "." + subEntity)
|
||||||
.selectExpr("id", "lastUsage", "dedupId");
|
.selectExpr("id", "lastUsage");
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// depending on resulttype collectefrom and dateofacceptance are evaluated differently
|
||||||
String collectedfromExpr = "false AS collectedfrom";
|
String collectedfromExpr = "false AS collectedfrom";
|
||||||
String dateExpr = "'' AS date";
|
String dateExpr = "'' AS date";
|
||||||
|
|
||||||
|
@ -164,8 +174,10 @@ public class SparkCreateMergeRels extends AbstractSparkAction {
|
||||||
dateExpr = "dateofacceptance.value AS date";
|
dateExpr = "dateofacceptance.value AS date";
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// cap pidType at w3id as from there on they are considered equal
|
||||||
UserDefinedFunction mapPid = udf(
|
UserDefinedFunction mapPid = udf(
|
||||||
(String s) -> Math.min(PidType.tryValueOf(s).ordinal(), PidType.w3id.ordinal()), DataTypes.IntegerType);
|
(String s) -> Math.min(PidType.tryValueOf(s).ordinal(), PidType.w3id.ordinal()), DataTypes.IntegerType);
|
||||||
|
|
||||||
UserDefinedFunction validDate = udf((String date) -> {
|
UserDefinedFunction validDate = udf((String date) -> {
|
||||||
if (StringUtils.isNotBlank(date)
|
if (StringUtils.isNotBlank(date)
|
||||||
&& date.matches(DatePicker.DATE_PATTERN) && DatePicker.inRange(date)) {
|
&& date.matches(DatePicker.DATE_PATTERN) && DatePicker.inRange(date)) {
|
||||||
|
@ -186,8 +198,6 @@ public class SparkCreateMergeRels extends AbstractSparkAction {
|
||||||
.withColumn("pidType", mapPid.apply(col("pidType"))) // ordinal of pid type
|
.withColumn("pidType", mapPid.apply(col("pidType"))) // ordinal of pid type
|
||||||
.withColumn("date", validDate.apply(col("date")));
|
.withColumn("date", validDate.apply(col("date")));
|
||||||
|
|
||||||
UserDefinedFunction generateDedupId = udf((String s) -> IdGenerator.generate(s), DataTypes.StringType);
|
|
||||||
|
|
||||||
// ordering to selected pivot id
|
// ordering to selected pivot id
|
||||||
WindowSpec w = Window
|
WindowSpec w = Window
|
||||||
.partitionBy("groupId")
|
.partitionBy("groupId")
|
||||||
|
@ -202,17 +212,15 @@ public class SparkCreateMergeRels extends AbstractSparkAction {
|
||||||
.join(pivotHistory, JavaConversions.asScalaBuffer(Collections.singletonList("id")), "full")
|
.join(pivotHistory, JavaConversions.asScalaBuffer(Collections.singletonList("id")), "full")
|
||||||
.join(pivotingData, JavaConversions.asScalaBuffer(Collections.singletonList("id")), "left")
|
.join(pivotingData, JavaConversions.asScalaBuffer(Collections.singletonList("id")), "left")
|
||||||
.withColumn("pivot", functions.first("id").over(w))
|
.withColumn("pivot", functions.first("id").over(w))
|
||||||
.withColumn("pivotDedupId", functions.first("dedupId").over(w))
|
|
||||||
.withColumn("position", functions.row_number().over(w))
|
.withColumn("position", functions.row_number().over(w))
|
||||||
.filter(cut > 0 ? col("position").lt(lit(cut)) : lit(true))
|
.filter(cut > 0 ? col("position").lt(lit(cut)) : lit(true)) // apply cut after choosing pivot
|
||||||
// .select("id", "groupId", "collectedfrom", "pivot", "dedupId", "pivotDedupId")
|
|
||||||
// .distinct()
|
|
||||||
.flatMap(
|
.flatMap(
|
||||||
(FlatMapFunction<Row, Tuple3<String, String, String>>) (Row r) -> {
|
(FlatMapFunction<Row, Tuple3<String, String, String>>) (Row r) -> {
|
||||||
String id = r.getAs("id");
|
String id = r.getAs("id");
|
||||||
|
String dedupId = IdGenerator.generate(id);
|
||||||
|
|
||||||
String pivot = r.getAs("pivot");
|
String pivot = r.getAs("pivot");
|
||||||
String pivotDedupId = r.getAs("pivotDedupId"); // dedupId associated with the pivot
|
String pivotDedupId = IdGenerator.generate(pivot);
|
||||||
String dedupId = r.getAs("dedupId"); // dedupId associated with this id if it was a pivot
|
|
||||||
|
|
||||||
// filter out id == pivotDedupId
|
// filter out id == pivotDedupId
|
||||||
// those are caused by claim expressed on pivotDedupId
|
// those are caused by claim expressed on pivotDedupId
|
||||||
|
@ -233,14 +241,9 @@ public class SparkCreateMergeRels extends AbstractSparkAction {
|
||||||
return res.iterator();
|
return res.iterator();
|
||||||
}
|
}
|
||||||
|
|
||||||
// new pivot, assign pivotDedupId with current IdGenerator
|
// this was a pivot in a previous graph but it has been merged into a new group with different
|
||||||
if (StringUtils.isBlank(pivotDedupId)) {
|
|
||||||
pivotDedupId = IdGenerator.generate(pivot);
|
|
||||||
}
|
|
||||||
|
|
||||||
// this was a pivot in a preceding graph but it has been merged into a new group with different
|
|
||||||
// pivot
|
// pivot
|
||||||
if (StringUtils.isNotBlank(dedupId) && !pivot.equals(id) && !dedupId.equals(pivotDedupId)) {
|
if (!r.isNullAt(r.fieldIndex("lastUsage")) && !pivot.equals(id) && !dedupId.equals(pivotDedupId)) {
|
||||||
// materialize the previous dedup record as a merge relation with the new one
|
// materialize the previous dedup record as a merge relation with the new one
|
||||||
res.add(new Tuple3<>(dedupId, pivotDedupId, null));
|
res.add(new Tuple3<>(dedupId, pivotDedupId, null));
|
||||||
}
|
}
|
||||||
|
|
Loading…
Reference in New Issue