[webcrawl] added code and test (code/resource) to verify the deletion of the relations related to results put in blacklist
This commit is contained in:
parent
6f1801d7d1
commit
fc60661ac5
|
@ -8,7 +8,7 @@ import java.util.*;
|
||||||
|
|
||||||
import org.apache.commons.io.IOUtils;
|
import org.apache.commons.io.IOUtils;
|
||||||
import org.apache.hadoop.io.Text;
|
import org.apache.hadoop.io.Text;
|
||||||
import org.apache.hadoop.io.compress.GzipCodec;
|
import org.apache.hadoop.io.compress.BZip2Codec;
|
||||||
import org.apache.hadoop.mapred.SequenceFileOutputFormat;
|
import org.apache.hadoop.mapred.SequenceFileOutputFormat;
|
||||||
import org.apache.spark.SparkConf;
|
import org.apache.spark.SparkConf;
|
||||||
import org.apache.spark.api.java.function.FilterFunction;
|
import org.apache.spark.api.java.function.FilterFunction;
|
||||||
|
@ -112,7 +112,7 @@ public class CreateActionSetFromWebEntries implements Serializable {
|
||||||
.mapToPair(
|
.mapToPair(
|
||||||
aa -> new Tuple2<>(new Text(aa.getClazz().getCanonicalName()),
|
aa -> new Tuple2<>(new Text(aa.getClazz().getCanonicalName()),
|
||||||
new Text(OBJECT_MAPPER.writeValueAsString(aa))))
|
new Text(OBJECT_MAPPER.writeValueAsString(aa))))
|
||||||
.saveAsHadoopFile(outputPath, Text.class, Text.class, SequenceFileOutputFormat.class, GzipCodec.class);
|
.saveAsHadoopFile(outputPath, Text.class, Text.class, SequenceFileOutputFormat.class, BZip2Codec.class);
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -1,244 +1,159 @@
|
||||||
|
|
||||||
package eu.dnetlib.dhp.actionmanager.webcrawl;
|
package eu.dnetlib.dhp.actionmanager.webcrawl;
|
||||||
|
|
||||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession;
|
||||||
import eu.dnetlib.dhp.actionmanager.Constants;
|
import static org.apache.spark.sql.functions.*;
|
||||||
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
|
|
||||||
import eu.dnetlib.dhp.schema.action.AtomicAction;
|
import java.io.File;
|
||||||
import eu.dnetlib.dhp.schema.common.ModelConstants;
|
import java.io.Serializable;
|
||||||
import eu.dnetlib.dhp.schema.oaf.Relation;
|
import java.util.Arrays;
|
||||||
import eu.dnetlib.dhp.schema.oaf.utils.IdentifierFactory;
|
import java.util.Optional;
|
||||||
import eu.dnetlib.dhp.schema.oaf.utils.OafMapperUtils;
|
|
||||||
import eu.dnetlib.dhp.schema.oaf.utils.PidCleaner;
|
import org.apache.commons.io.FileUtils;
|
||||||
import eu.dnetlib.dhp.schema.oaf.utils.PidType;
|
|
||||||
import org.apache.commons.io.IOUtils;
|
import org.apache.commons.io.IOUtils;
|
||||||
|
import org.apache.commons.io.filefilter.DirectoryFileFilter;
|
||||||
|
import org.apache.commons.io.filefilter.FileFileFilter;
|
||||||
import org.apache.hadoop.io.Text;
|
import org.apache.hadoop.io.Text;
|
||||||
|
import org.apache.hadoop.io.compress.BZip2Codec;
|
||||||
import org.apache.hadoop.io.compress.GzipCodec;
|
import org.apache.hadoop.io.compress.GzipCodec;
|
||||||
import org.apache.hadoop.mapred.SequenceFileOutputFormat;
|
import org.apache.hadoop.mapred.SequenceFileOutputFormat;
|
||||||
import org.apache.spark.SparkConf;
|
import org.apache.spark.SparkConf;
|
||||||
|
import org.apache.spark.api.java.JavaPairRDD;
|
||||||
|
import org.apache.spark.api.java.JavaRDD;
|
||||||
|
import org.apache.spark.api.java.JavaSparkContext;
|
||||||
import org.apache.spark.api.java.function.FilterFunction;
|
import org.apache.spark.api.java.function.FilterFunction;
|
||||||
import org.apache.spark.api.java.function.FlatMapFunction;
|
import org.apache.spark.api.java.function.MapFunction;
|
||||||
import org.apache.spark.sql.*;
|
import org.apache.spark.sql.*;
|
||||||
import org.apache.spark.sql.types.StructType;
|
import org.apache.spark.sql.types.*;
|
||||||
import org.slf4j.Logger;
|
import org.slf4j.Logger;
|
||||||
import org.slf4j.LoggerFactory;
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
|
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||||
|
|
||||||
|
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
|
||||||
|
import eu.dnetlib.dhp.common.HdfsSupport;
|
||||||
|
import eu.dnetlib.dhp.schema.action.AtomicAction;
|
||||||
|
import eu.dnetlib.dhp.schema.oaf.Relation;
|
||||||
|
import eu.dnetlib.dhp.schema.oaf.utils.IdentifierFactory;
|
||||||
import scala.Tuple2;
|
import scala.Tuple2;
|
||||||
|
|
||||||
import java.io.Serializable;
|
|
||||||
import java.util.ArrayList;
|
|
||||||
import java.util.Arrays;
|
|
||||||
import java.util.List;
|
|
||||||
import java.util.Optional;
|
|
||||||
|
|
||||||
import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession;
|
|
||||||
|
|
||||||
public class RemoveRelationFromActionSet
|
public class RemoveRelationFromActionSet
|
||||||
implements Serializable {
|
implements Serializable {
|
||||||
private static final Logger log = LoggerFactory.getLogger(CreateActionSetFromWebEntries.class);
|
private static final Logger log = LoggerFactory.getLogger(CreateActionSetFromWebEntries.class);
|
||||||
private static final String DOI_PREFIX = "50|doi_________::";
|
|
||||||
|
|
||||||
|
private static final ObjectMapper MAPPER = new ObjectMapper();
|
||||||
|
private static final StructType KV_SCHEMA = StructType$.MODULE$
|
||||||
|
.apply(
|
||||||
|
Arrays
|
||||||
|
.asList(
|
||||||
|
StructField$.MODULE$.apply("key", DataTypes.StringType, false, Metadata.empty()),
|
||||||
|
StructField$.MODULE$.apply("value", DataTypes.StringType, false, Metadata.empty())));
|
||||||
|
|
||||||
public static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
|
private static final StructType ATOMIC_ACTION_SCHEMA = StructType$.MODULE$
|
||||||
|
.apply(
|
||||||
|
Arrays
|
||||||
|
.asList(
|
||||||
|
StructField$.MODULE$.apply("clazz", DataTypes.StringType, false, Metadata.empty()),
|
||||||
|
StructField$.MODULE$
|
||||||
|
.apply(
|
||||||
|
"payload", DataTypes.StringType, false, Metadata.empty())));
|
||||||
|
|
||||||
public static void main(String[] args) throws Exception {
|
public static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
|
||||||
String jsonConfiguration = IOUtils
|
|
||||||
.toString(
|
|
||||||
CreateActionSetFromWebEntries.class
|
|
||||||
.getResourceAsStream(
|
|
||||||
"/eu/dnetlib/dhp/actionmanager/webcrawl/as_parameters.json"));
|
|
||||||
|
|
||||||
final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration);
|
public static void main(String[] args) throws Exception {
|
||||||
parser.parseArgument(args);
|
String jsonConfiguration = IOUtils
|
||||||
|
.toString(
|
||||||
|
CreateActionSetFromWebEntries.class
|
||||||
|
.getResourceAsStream(
|
||||||
|
"/eu/dnetlib/dhp/actionmanager/webcrawl/as_parameters.json"));
|
||||||
|
|
||||||
Boolean isSparkSessionManaged = Optional
|
final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration);
|
||||||
.ofNullable(parser.get("isSparkSessionManaged"))
|
parser.parseArgument(args);
|
||||||
.map(Boolean::valueOf)
|
|
||||||
.orElse(Boolean.TRUE);
|
|
||||||
|
|
||||||
log.info("isSparkSessionManaged: {}", isSparkSessionManaged);
|
Boolean isSparkSessionManaged = Optional
|
||||||
|
.ofNullable(parser.get("isSparkSessionManaged"))
|
||||||
|
.map(Boolean::valueOf)
|
||||||
|
.orElse(Boolean.TRUE);
|
||||||
|
|
||||||
final String inputPath = parser.get("actionSetPath");
|
log.info("isSparkSessionManaged: {}", isSparkSessionManaged);
|
||||||
log.info("inputPath: {}", inputPath);
|
|
||||||
|
|
||||||
final String outputPath = parser.get("outputPath");
|
// the actionSet path
|
||||||
log.info("outputPath: {}", outputPath);
|
final String inputPath = parser.get("sourcePath");
|
||||||
|
log.info("inputPath: {}", inputPath);
|
||||||
|
|
||||||
final String blackListInputPath = parser.get("blackListPath");
|
final String outputPath = parser.get("outputPath");
|
||||||
log.info("blackListInputPath: {}", blackListInputPath);
|
log.info("outputPath: {}", outputPath);
|
||||||
|
|
||||||
SparkConf conf = new SparkConf();
|
final String blackListInputPath = parser.get("blackListPath");
|
||||||
|
log.info("blackListInputPath: {}", blackListInputPath);
|
||||||
|
|
||||||
runWithSparkSession(
|
SparkConf conf = new SparkConf();
|
||||||
conf,
|
|
||||||
isSparkSessionManaged,
|
|
||||||
spark -> {
|
|
||||||
|
|
||||||
removeFromActionSet(spark, inputPath, outputPath, blackListInputPath);
|
runWithSparkSession(
|
||||||
|
conf,
|
||||||
|
isSparkSessionManaged,
|
||||||
|
spark -> {
|
||||||
|
|
||||||
});
|
removeFromActionSet(spark, inputPath, outputPath, blackListInputPath);
|
||||||
}
|
|
||||||
|
|
||||||
private static void removeFromActionSet(SparkSession spark, String inputPath, String outputPath, String blackListInputPath) {
|
});
|
||||||
|
}
|
||||||
|
|
||||||
}
|
private static void removeFromActionSet(SparkSession spark, String inputPath, String outputPath,
|
||||||
|
String blackListInputPath) {
|
||||||
|
// read the blacklist
|
||||||
|
Dataset<String> blackList = readBlackList(spark, blackListInputPath)
|
||||||
|
.map(
|
||||||
|
(MapFunction<Row, String>) r -> IdentifierFactory
|
||||||
|
.idFromPid("50", "doi", ((String) r.getAs("DOI / PMID")).substring(16), true),
|
||||||
|
Encoders.STRING());
|
||||||
|
|
||||||
public static void createActionSet(SparkSession spark, String inputPath,
|
// read the old actionset and get the relations in the payload
|
||||||
String outputPath, String blackListInputPath) {
|
JavaPairRDD<Text, Text> seq = JavaSparkContext
|
||||||
|
.fromSparkContext(spark.sparkContext())
|
||||||
|
.sequenceFile(inputPath, Text.class, Text.class);
|
||||||
|
|
||||||
final Dataset<Row> dataset = readWebCrawl(spark, inputPath)
|
JavaRDD<Row> rdd = seq
|
||||||
.filter("country_code=='IE'")
|
.map(x -> RowFactory.create(x._1().toString(), x._2().toString()));
|
||||||
.drop("publication_year");
|
|
||||||
|
|
||||||
final Dataset<Row> blackList = readBlackList(spark, blackListInputPath);
|
Dataset<Row> actionSet = spark
|
||||||
|
.createDataFrame(rdd, KV_SCHEMA)
|
||||||
|
.withColumn("atomic_action", from_json(col("value"), ATOMIC_ACTION_SCHEMA))
|
||||||
|
.select(expr("atomic_action.*"));
|
||||||
|
|
||||||
dataset
|
Dataset<Relation> relation = actionSet
|
||||||
.join(blackList, dataset.col("id").equalTo(blackList.col("OpenAlexId")), "left")
|
.map(
|
||||||
.filter((FilterFunction<Row>) r -> r.getAs("OpenAlexId") == null)
|
(MapFunction<Row, Relation>) r -> MAPPER.readValue((String) r.getAs("payload"), Relation.class),
|
||||||
.drop("OpenAlexId")
|
Encoders.bean(Relation.class));
|
||||||
.flatMap((FlatMapFunction<Row, Relation>) row -> {
|
|
||||||
List<Relation> ret = new ArrayList<>();
|
|
||||||
final String ror = ROR_PREFIX
|
|
||||||
+ IdentifierFactory.md5(PidCleaner.normalizePidValue("ROR", row.getAs("ror")));
|
|
||||||
ret.addAll(createAffiliationRelationPairDOI(row.getAs("doi"), ror));
|
|
||||||
|
|
||||||
return ret
|
// select only the relation not matching any pid in the blacklist as source for the relation
|
||||||
.iterator();
|
Dataset<Relation> relNoSource = relation
|
||||||
}, Encoders.bean(Relation.class))
|
.joinWith(blackList, relation.col("source").equalTo(blackList.col("value")), "left")
|
||||||
.toJavaRDD()
|
.filter((FilterFunction<Tuple2<Relation, String>>) t2 -> t2._2() == null)
|
||||||
.map(p -> new AtomicAction(p.getClass(), p))
|
.map((MapFunction<Tuple2<Relation, String>, Relation>) t2 -> t2._1(), Encoders.bean(Relation.class));
|
||||||
.mapToPair(
|
|
||||||
aa -> new Tuple2<>(new Text(aa.getClazz().getCanonicalName()),
|
|
||||||
new Text(OBJECT_MAPPER.writeValueAsString(aa))))
|
|
||||||
.saveAsHadoopFile(outputPath, Text.class, Text.class, SequenceFileOutputFormat.class, GzipCodec.class);
|
|
||||||
|
|
||||||
}
|
// select only the relation not matching any pid in the blacklist as target of the relation
|
||||||
|
relNoSource
|
||||||
|
.joinWith(blackList, relNoSource.col("target").equalTo(blackList.col("value")), "left")
|
||||||
|
.filter((FilterFunction<Tuple2<Relation, String>>) t2 -> t2._2() == null)
|
||||||
|
.map((MapFunction<Tuple2<Relation, String>, Relation>) t2 -> t2._1(), Encoders.bean(Relation.class))
|
||||||
|
.toJavaRDD()
|
||||||
|
.map(p -> new AtomicAction(p.getClass(), p))
|
||||||
|
.mapToPair(
|
||||||
|
aa -> new Tuple2<>(new Text(aa.getClazz().getCanonicalName()),
|
||||||
|
new Text(OBJECT_MAPPER.writeValueAsString(aa))))
|
||||||
|
.saveAsHadoopFile(outputPath, Text.class, Text.class, SequenceFileOutputFormat.class, BZip2Codec.class);
|
||||||
|
;
|
||||||
|
|
||||||
private static Dataset<Row> readWebCrawl(SparkSession spark, String inputPath) {
|
}
|
||||||
StructType webInfo = StructType
|
|
||||||
.fromDDL(
|
|
||||||
"`id` STRING , `doi` STRING, `ids` STRUCT<`pmid` :STRING, `pmcid`: STRING >, `publication_year` STRING, "
|
|
||||||
+
|
|
||||||
"`authorships` ARRAY<STRUCT <`institutions`: ARRAY <STRUCT <`ror`: STRING, `country_code` :STRING>>>>");
|
|
||||||
|
|
||||||
return spark
|
private static Dataset<Row> readBlackList(SparkSession spark, String inputPath) {
|
||||||
.read()
|
|
||||||
.schema(webInfo)
|
|
||||||
.json(inputPath)
|
|
||||||
.withColumn(
|
|
||||||
"authors", functions
|
|
||||||
.explode(
|
|
||||||
functions.col("authorships")))
|
|
||||||
.selectExpr("id", "doi", "ids", "publication_year", "authors.institutions as institutions")
|
|
||||||
.withColumn(
|
|
||||||
"institution", functions
|
|
||||||
.explode(
|
|
||||||
functions.col("institutions")))
|
|
||||||
|
|
||||||
.selectExpr(
|
return spark
|
||||||
"id", "doi", "institution.ror as ror",
|
.read()
|
||||||
"institution.country_code as country_code", "publication_year")
|
.option("header", true)
|
||||||
.distinct();
|
.csv(inputPath)
|
||||||
|
.select("DOI / PMID");
|
||||||
|
}
|
||||||
|
|
||||||
}
|
|
||||||
|
|
||||||
private static Dataset<Row> readBlackList(SparkSession spark, String inputPath) {
|
|
||||||
|
|
||||||
return spark
|
|
||||||
.read()
|
|
||||||
.option("header", true)
|
|
||||||
.csv(inputPath)
|
|
||||||
.select("OpenAlexId");
|
|
||||||
}
|
|
||||||
|
|
||||||
private static List<Relation> createAffiliationRelationPairPMCID(String pmcid, String ror) {
|
|
||||||
if (pmcid == null)
|
|
||||||
return new ArrayList<>();
|
|
||||||
|
|
||||||
return createAffiliatioRelationPair(
|
|
||||||
PMCID_PREFIX
|
|
||||||
+ IdentifierFactory
|
|
||||||
.md5(PidCleaner.normalizePidValue(PidType.pmc.toString(), removeResolver("PMC", pmcid))),
|
|
||||||
ror);
|
|
||||||
}
|
|
||||||
|
|
||||||
private static List<Relation> createAffiliationRelationPairPMID(String pmid, String ror) {
|
|
||||||
if (pmid == null)
|
|
||||||
return new ArrayList<>();
|
|
||||||
|
|
||||||
return createAffiliatioRelationPair(
|
|
||||||
PMID_PREFIX
|
|
||||||
+ IdentifierFactory
|
|
||||||
.md5(PidCleaner.normalizePidValue(PidType.pmid.toString(), removeResolver("PMID", pmid))),
|
|
||||||
ror);
|
|
||||||
}
|
|
||||||
|
|
||||||
private static String removeResolver(String pidType, String pid) {
|
|
||||||
switch (pidType) {
|
|
||||||
case "PMID":
|
|
||||||
return pid.substring(33);
|
|
||||||
case "PMC":
|
|
||||||
return "PMC" + pid.substring(43);
|
|
||||||
case "DOI":
|
|
||||||
return pid.substring(16);
|
|
||||||
}
|
|
||||||
|
|
||||||
throw new RuntimeException();
|
|
||||||
|
|
||||||
}
|
|
||||||
|
|
||||||
private static List<Relation> createAffiliationRelationPairDOI(String doi, String ror) {
|
|
||||||
if (doi == null)
|
|
||||||
return new ArrayList<>();
|
|
||||||
|
|
||||||
return createAffiliatioRelationPair(
|
|
||||||
DOI_PREFIX
|
|
||||||
+ IdentifierFactory
|
|
||||||
.md5(PidCleaner.normalizePidValue(PidType.doi.toString(), removeResolver("DOI", doi))),
|
|
||||||
ror);
|
|
||||||
|
|
||||||
}
|
|
||||||
|
|
||||||
private static List<Relation> createAffiliatioRelationPair(String resultId, String orgId) {
|
|
||||||
ArrayList<Relation> newRelations = new ArrayList();
|
|
||||||
|
|
||||||
newRelations
|
|
||||||
.add(
|
|
||||||
OafMapperUtils
|
|
||||||
.getRelation(
|
|
||||||
orgId, resultId, ModelConstants.RESULT_ORGANIZATION, ModelConstants.AFFILIATION,
|
|
||||||
ModelConstants.IS_AUTHOR_INSTITUTION_OF,
|
|
||||||
Arrays
|
|
||||||
.asList(
|
|
||||||
OafMapperUtils.keyValue(Constants.WEB_CRAWL_ID, Constants.WEB_CRAWL_NAME)),
|
|
||||||
OafMapperUtils
|
|
||||||
.dataInfo(
|
|
||||||
false, null, false, false,
|
|
||||||
OafMapperUtils
|
|
||||||
.qualifier(
|
|
||||||
"sysimport:crasswalk:webcrawl", "Imported from Webcrawl",
|
|
||||||
ModelConstants.DNET_PROVENANCE_ACTIONS, ModelConstants.DNET_PROVENANCE_ACTIONS),
|
|
||||||
"0.9"),
|
|
||||||
null));
|
|
||||||
|
|
||||||
newRelations
|
|
||||||
.add(
|
|
||||||
OafMapperUtils
|
|
||||||
.getRelation(
|
|
||||||
resultId, orgId, ModelConstants.RESULT_ORGANIZATION, ModelConstants.AFFILIATION,
|
|
||||||
ModelConstants.HAS_AUTHOR_INSTITUTION,
|
|
||||||
Arrays
|
|
||||||
.asList(
|
|
||||||
OafMapperUtils.keyValue(Constants.WEB_CRAWL_ID, Constants.WEB_CRAWL_NAME)),
|
|
||||||
OafMapperUtils
|
|
||||||
.dataInfo(
|
|
||||||
false, null, false, false,
|
|
||||||
OafMapperUtils
|
|
||||||
.qualifier(
|
|
||||||
"sysimport:crasswalk:webcrawl", "Imported from Webcrawl",
|
|
||||||
ModelConstants.DNET_PROVENANCE_ACTIONS, ModelConstants.DNET_PROVENANCE_ACTIONS),
|
|
||||||
"0.9"),
|
|
||||||
null));
|
|
||||||
|
|
||||||
return newRelations;
|
|
||||||
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -1,3 +1,11 @@
|
||||||
sourcePath=/user/miriam.baglioni/openalex-snapshot/data/works/
|
#PROPERTIES TO CREATE THE ACTION SET
|
||||||
outputPath=/tmp/miriam/webcrawlComplete/
|
#sourcePath=/user/miriam.baglioni/openalex-snapshot/data/works/
|
||||||
blackListPath=/user/miriam.baglioni/openalex-blackList
|
#outputPath=/tmp/miriam/webcrawlComplete/
|
||||||
|
#blackListPath=/user/miriam.baglioni/openalex-blackList
|
||||||
|
#resumeFrom=create
|
||||||
|
|
||||||
|
#PROPERTIES TO REMOVE FROM THE ACTION SET
|
||||||
|
sourcePath=/var/lib/dnet/actionManager_PROD/webcrawl/rawset_28247629-468b-478e-9a42-bc540877125d_1718121542061/
|
||||||
|
outputPath=/tmp/miriam/webcrawlRemoved/
|
||||||
|
blackListPath=/user/miriam.baglioni/oalexBlackListNormalized
|
||||||
|
resumeFrom=remove
|
|
@ -20,12 +20,19 @@
|
||||||
</configuration>
|
</configuration>
|
||||||
</global>
|
</global>
|
||||||
|
|
||||||
<start to="create_actionset"/>
|
<start to="resumeFrom"/>
|
||||||
|
|
||||||
<kill name="Kill">
|
<kill name="Kill">
|
||||||
<message>Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}]</message>
|
<message>Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}]</message>
|
||||||
</kill>
|
</kill>
|
||||||
|
|
||||||
|
<decision name="resumeFrom">
|
||||||
|
<switch>
|
||||||
|
<case to="create_actionset">${wf:conf('resumeFrom') eq 'create'}</case>
|
||||||
|
<default to="remove_from_actionset"/>
|
||||||
|
</switch>
|
||||||
|
</decision>
|
||||||
|
|
||||||
<action name="create_actionset">
|
<action name="create_actionset">
|
||||||
<spark xmlns="uri:oozie:spark-action:0.2">
|
<spark xmlns="uri:oozie:spark-action:0.2">
|
||||||
<master>yarn</master>
|
<master>yarn</master>
|
||||||
|
@ -50,5 +57,30 @@
|
||||||
<ok to="End"/>
|
<ok to="End"/>
|
||||||
<error to="Kill"/>
|
<error to="Kill"/>
|
||||||
</action>
|
</action>
|
||||||
|
|
||||||
|
<action name="remove_from_actionset">
|
||||||
|
<spark xmlns="uri:oozie:spark-action:0.2">
|
||||||
|
<master>yarn</master>
|
||||||
|
<mode>cluster</mode>
|
||||||
|
<name>Removes some relations found to be wrong from the AS</name>
|
||||||
|
<class>eu.dnetlib.dhp.actionmanager.webcrawl.RemoveRelationFromActionSet</class>
|
||||||
|
<jar>dhp-aggregation-${projectVersion}.jar</jar>
|
||||||
|
<spark-opts>
|
||||||
|
--executor-memory=${sparkExecutorMemory}
|
||||||
|
--executor-cores=${sparkExecutorCores}
|
||||||
|
--driver-memory=${sparkDriverMemory}
|
||||||
|
--conf spark.extraListeners=${spark2ExtraListeners}
|
||||||
|
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
|
||||||
|
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
|
||||||
|
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
|
||||||
|
--conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir}
|
||||||
|
</spark-opts>
|
||||||
|
<arg>--sourcePath</arg><arg>${sourcePath}</arg>
|
||||||
|
<arg>--outputPath</arg><arg>${outputPath}</arg>
|
||||||
|
<arg>--blackListPath</arg><arg>${blackListPath}</arg>
|
||||||
|
</spark>
|
||||||
|
<ok to="End"/>
|
||||||
|
<error to="Kill"/>
|
||||||
|
</action>
|
||||||
<end name="End"/>
|
<end name="End"/>
|
||||||
</workflow-app>
|
</workflow-app>
|
|
@ -2,6 +2,7 @@
|
||||||
package eu.dnetlib.dhp.actionmanager.webcrawl;
|
package eu.dnetlib.dhp.actionmanager.webcrawl;
|
||||||
|
|
||||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||||
|
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.nio.file.Files;
|
import java.nio.file.Files;
|
||||||
|
@ -101,7 +102,10 @@ public class CreateASTest {
|
||||||
.map(value -> OBJECT_MAPPER.readValue(value._2().toString(), AtomicAction.class))
|
.map(value -> OBJECT_MAPPER.readValue(value._2().toString(), AtomicAction.class))
|
||||||
.map(aa -> ((Relation) aa.getPayload()));
|
.map(aa -> ((Relation) aa.getPayload()));
|
||||||
|
|
||||||
Assertions.assertEquals(58, tmp.count());
|
tmp.foreach(r -> System.out.println(new ObjectMapper().writeValueAsString(r)));
|
||||||
|
tmp.foreach(r -> assertTrue(r.getSource().startsWith("20|ror") || r.getSource().startsWith("50|doi")));
|
||||||
|
tmp.foreach(r -> assertTrue(r.getTarget().startsWith("20|ror") || r.getTarget().startsWith("50|doi")));
|
||||||
|
Assertions.assertEquals(24, tmp.count());
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -112,7 +116,7 @@ public class CreateASTest {
|
||||||
|
|
||||||
String inputPath = getClass()
|
String inputPath = getClass()
|
||||||
.getResource(
|
.getResource(
|
||||||
"/eu/dnetlib/dhp/actionmanager/webcrawl/")
|
"/eu/dnetlib/dhp/actionmanager/webcrawl/input/")
|
||||||
.getPath();
|
.getPath();
|
||||||
String blackListPath = getClass()
|
String blackListPath = getClass()
|
||||||
.getResource(
|
.getResource(
|
||||||
|
@ -194,7 +198,7 @@ public class CreateASTest {
|
||||||
|
|
||||||
Assertions
|
Assertions
|
||||||
.assertEquals(
|
.assertEquals(
|
||||||
2, tmp
|
1, tmp
|
||||||
.filter(
|
.filter(
|
||||||
r -> r
|
r -> r
|
||||||
.getSource()
|
.getSource()
|
||||||
|
@ -207,7 +211,7 @@ public class CreateASTest {
|
||||||
|
|
||||||
Assertions
|
Assertions
|
||||||
.assertEquals(
|
.assertEquals(
|
||||||
2, tmp
|
1, tmp
|
||||||
.filter(
|
.filter(
|
||||||
r -> r
|
r -> r
|
||||||
.getTarget()
|
.getTarget()
|
||||||
|
@ -228,13 +232,13 @@ public class CreateASTest {
|
||||||
"20|ror_________::" + IdentifierFactory
|
"20|ror_________::" + IdentifierFactory
|
||||||
.md5(
|
.md5(
|
||||||
PidCleaner
|
PidCleaner
|
||||||
.normalizePidValue(PidType.doi.toString(), "https://ror.org/03265fv13")))
|
.normalizePidValue("ROR", "https://ror.org/03265fv13")))
|
||||||
&& r.getSource().startsWith("50|doi"))
|
&& r.getSource().startsWith("50|doi"))
|
||||||
.count());
|
.count());
|
||||||
|
|
||||||
Assertions
|
Assertions
|
||||||
.assertEquals(
|
.assertEquals(
|
||||||
1, tmp
|
0, tmp
|
||||||
.filter(
|
.filter(
|
||||||
r -> r
|
r -> r
|
||||||
.getTarget()
|
.getTarget()
|
||||||
|
@ -268,6 +272,10 @@ public class CreateASTest {
|
||||||
.getResource(
|
.getResource(
|
||||||
"/eu/dnetlib/dhp/actionmanager/webcrawl")
|
"/eu/dnetlib/dhp/actionmanager/webcrawl")
|
||||||
.getPath();
|
.getPath();
|
||||||
|
String blackListPath = getClass()
|
||||||
|
.getResource(
|
||||||
|
"/eu/dnetlib/dhp/actionmanager/webcrawl/blackList/")
|
||||||
|
.getPath();
|
||||||
|
|
||||||
CreateActionSetFromWebEntries
|
CreateActionSetFromWebEntries
|
||||||
.main(
|
.main(
|
||||||
|
@ -277,7 +285,8 @@ public class CreateASTest {
|
||||||
"-sourcePath",
|
"-sourcePath",
|
||||||
inputPath,
|
inputPath,
|
||||||
"-outputPath",
|
"-outputPath",
|
||||||
workingDir.toString() + "/actionSet1"
|
workingDir.toString() + "/actionSet1",
|
||||||
|
"-blackListPath", blackListPath
|
||||||
});
|
});
|
||||||
|
|
||||||
final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext());
|
final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext());
|
||||||
|
|
|
@ -0,0 +1,108 @@
|
||||||
|
|
||||||
|
package eu.dnetlib.dhp.actionmanager.webcrawl;
|
||||||
|
|
||||||
|
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.nio.file.Files;
|
||||||
|
import java.nio.file.Path;
|
||||||
|
|
||||||
|
import org.apache.commons.io.FileUtils;
|
||||||
|
import org.apache.hadoop.io.Text;
|
||||||
|
import org.apache.spark.SparkConf;
|
||||||
|
import org.apache.spark.api.java.JavaRDD;
|
||||||
|
import org.apache.spark.api.java.JavaSparkContext;
|
||||||
|
import org.apache.spark.sql.SparkSession;
|
||||||
|
import org.junit.jupiter.api.AfterAll;
|
||||||
|
import org.junit.jupiter.api.Assertions;
|
||||||
|
import org.junit.jupiter.api.BeforeAll;
|
||||||
|
import org.junit.jupiter.api.Test;
|
||||||
|
import org.slf4j.Logger;
|
||||||
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
|
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||||
|
|
||||||
|
import eu.dnetlib.dhp.schema.action.AtomicAction;
|
||||||
|
import eu.dnetlib.dhp.schema.oaf.Relation;
|
||||||
|
import eu.dnetlib.dhp.schema.oaf.utils.IdentifierFactory;
|
||||||
|
import eu.dnetlib.dhp.schema.oaf.utils.PidCleaner;
|
||||||
|
import eu.dnetlib.dhp.schema.oaf.utils.PidType;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @author miriam.baglioni
|
||||||
|
* @Date 22/04/24
|
||||||
|
*/
|
||||||
|
public class RemoveFromASTest {
|
||||||
|
private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
|
||||||
|
|
||||||
|
private static SparkSession spark;
|
||||||
|
|
||||||
|
private static Path workingDir;
|
||||||
|
private static final Logger log = LoggerFactory
|
||||||
|
.getLogger(RemoveFromASTest.class);
|
||||||
|
|
||||||
|
@BeforeAll
|
||||||
|
public static void beforeAll() throws IOException {
|
||||||
|
workingDir = Files
|
||||||
|
.createTempDirectory(RemoveFromASTest.class.getSimpleName());
|
||||||
|
log.info("using work dir {}", workingDir);
|
||||||
|
|
||||||
|
SparkConf conf = new SparkConf();
|
||||||
|
conf.setAppName(RemoveFromASTest.class.getSimpleName());
|
||||||
|
|
||||||
|
conf.setMaster("local[*]");
|
||||||
|
conf.set("spark.driver.host", "localhost");
|
||||||
|
conf.set("hive.metastore.local", "true");
|
||||||
|
conf.set("spark.ui.enabled", "false");
|
||||||
|
conf.set("spark.sql.warehouse.dir", workingDir.toString());
|
||||||
|
conf.set("hive.metastore.warehouse.dir", workingDir.resolve("warehouse").toString());
|
||||||
|
|
||||||
|
spark = SparkSession
|
||||||
|
.builder()
|
||||||
|
.appName(RemoveFromASTest.class.getSimpleName())
|
||||||
|
.config(conf)
|
||||||
|
.getOrCreate();
|
||||||
|
}
|
||||||
|
|
||||||
|
@AfterAll
|
||||||
|
public static void afterAll() throws IOException {
|
||||||
|
FileUtils.deleteDirectory(workingDir.toFile());
|
||||||
|
spark.stop();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
void testNumberofRelations() throws Exception {
|
||||||
|
|
||||||
|
String inputPath = getClass()
|
||||||
|
.getResource(
|
||||||
|
"/eu/dnetlib/dhp/actionmanager/webcrawl/actionSet/")
|
||||||
|
.getPath();
|
||||||
|
String blackListPath = getClass()
|
||||||
|
.getResource(
|
||||||
|
"/eu/dnetlib/dhp/actionmanager/webcrawl/blackListRemove/")
|
||||||
|
.getPath();
|
||||||
|
|
||||||
|
RemoveRelationFromActionSet
|
||||||
|
.main(
|
||||||
|
new String[] {
|
||||||
|
"-isSparkSessionManaged",
|
||||||
|
Boolean.FALSE.toString(),
|
||||||
|
"-sourcePath",
|
||||||
|
inputPath,
|
||||||
|
"-outputPath",
|
||||||
|
workingDir.toString() + "/actionSet1",
|
||||||
|
"-blackListPath", blackListPath
|
||||||
|
});
|
||||||
|
|
||||||
|
final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext());
|
||||||
|
|
||||||
|
JavaRDD<Relation> tmp = sc
|
||||||
|
.sequenceFile(workingDir.toString() + "/actionSet1", Text.class, Text.class)
|
||||||
|
.map(value -> OBJECT_MAPPER.readValue(value._2().toString(), AtomicAction.class))
|
||||||
|
.map(aa -> ((Relation) aa.getPayload()));
|
||||||
|
|
||||||
|
Assertions.assertEquals(22, tmp.count());
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,2 @@
|
||||||
|
DOI / PMID,OpenAlexId,Comments,
|
||||||
|
https://doi.org/10.1098/rstl.1684.0023,https://openalex.org/W2124362779,,
|
|
Loading…
Reference in New Issue