fixed a conflict

pull/88/head
Michele Artini 3 years ago
commit 69ba3203c0

@ -4,6 +4,7 @@ package eu.dnetlib.dhp.oa.merge;
import java.text.Normalizer;
import java.util.*;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import org.apache.commons.lang3.StringUtils;
@ -32,27 +33,33 @@ public class AuthorMerger {
}
public static List<Author> mergeAuthor(final List<Author> a, final List<Author> b) {
public static List<Author> mergeAuthor(final List<Author> a, final List<Author> b, Double threshold) {
int pa = countAuthorsPids(a);
int pb = countAuthorsPids(b);
List<Author> base, enrich;
int sa = authorsSize(a);
int sb = authorsSize(b);
if (pa == pb) {
base = sa > sb ? a : b;
enrich = sa > sb ? b : a;
} else {
if (sa == sb) {
base = pa > pb ? a : b;
enrich = pa > pb ? b : a;
} else {
base = sa > sb ? a : b;
enrich = sa > sb ? b : a;
}
enrichPidFromList(base, enrich);
enrichPidFromList(base, enrich, threshold);
return base;
}
private static void enrichPidFromList(List<Author> base, List<Author> enrich) {
public static List<Author> mergeAuthor(final List<Author> a, final List<Author> b) {
return mergeAuthor(a, b, THRESHOLD);
}
private static void enrichPidFromList(List<Author> base, List<Author> enrich, Double threshold) {
if (base == null || enrich == null)
return;
// <pidComparableString, Author> (if an Author has more than 1 pid, it appears 2 times in the list)
final Map<String, Author> basePidAuthorMap = base
.stream()
.filter(a -> a.getPid() != null && a.getPid().size() > 0)
@ -63,6 +70,7 @@ public class AuthorMerger {
.map(p -> new Tuple2<>(pidToComparableString(p), a)))
.collect(Collectors.toMap(Tuple2::_1, Tuple2::_2, (x1, x2) -> x1));
// <pid, Author> (list of pid that are missing in the other list)
final List<Tuple2<StructuredProperty, Author>> pidToEnrich = enrich
.stream()
.filter(a -> a.getPid() != null && a.getPid().size() > 0)
@ -83,10 +91,10 @@ public class AuthorMerger {
.max(Comparator.comparing(Tuple2::_1));
if (simAuthor.isPresent()) {
double th = THRESHOLD;
double th = threshold;
// increase the threshold if the surname is too short
if (simAuthor.get()._2().getSurname() != null
&& simAuthor.get()._2().getSurname().length() <= 3)
&& simAuthor.get()._2().getSurname().length() <= 3 && threshold > 0.0)
th = 0.99;
if (simAuthor.get()._1() > th) {
@ -156,7 +164,7 @@ public class AuthorMerger {
}
private static String normalize(final String s) {
return nfd(s)
String[] normalized = nfd(s)
.toLowerCase()
// do not compact the regexes in a single expression, would cause StackOverflowError
// in case
@ -166,7 +174,12 @@ public class AuthorMerger {
.replaceAll("(\\p{Punct})+", " ")
.replaceAll("(\\d)+", " ")
.replaceAll("(\\n)+", " ")
.trim();
.trim()
.split(" ");
Arrays.sort(normalized);
return String.join(" ", normalized);
}
private static String nfd(final String s) {

@ -0,0 +1,100 @@
package eu.dnetlib.dhp.oa.merge;
import java.io.BufferedReader;
import java.io.FileReader;
import java.io.IOException;
import java.nio.file.Paths;
import java.util.ArrayList;
import java.util.List;
import java.util.stream.Collectors;
import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import com.fasterxml.jackson.databind.ObjectMapper;
import eu.dnetlib.dhp.schema.oaf.Author;
import eu.dnetlib.dhp.schema.oaf.Publication;
import eu.dnetlib.dhp.schema.oaf.StructuredProperty;
import eu.dnetlib.pace.util.MapDocumentUtil;
import scala.Tuple2;
public class AuthorMergerTest {
private String publicationsBasePath;
private List<List<Author>> authors;
@BeforeEach
public void setUp() throws Exception {
publicationsBasePath = Paths
.get(AuthorMergerTest.class.getResource("/eu/dnetlib/dhp/oa/merge").toURI())
.toFile()
.getAbsolutePath();
authors = readSample(publicationsBasePath + "/publications_with_authors.json", Publication.class)
.stream()
.map(p -> p._2().getAuthor())
.collect(Collectors.toList());
}
@Test
public void mergeTest() { // used in the dedup: threshold set to 0.95
for (List<Author> authors1 : authors) {
System.out.println("List " + (authors.indexOf(authors1) + 1));
for (Author author : authors1) {
System.out.println(authorToString(author));
}
}
List<Author> merge = AuthorMerger.merge(authors);
System.out.println("Merge ");
for (Author author : merge) {
System.out.println(authorToString(author));
}
Assertions.assertEquals(7, merge.size());
}
public <T> List<Tuple2<String, T>> readSample(String path, Class<T> clazz) {
List<Tuple2<String, T>> res = new ArrayList<>();
BufferedReader reader;
try {
reader = new BufferedReader(new FileReader(path));
String line = reader.readLine();
while (line != null) {
res
.add(
new Tuple2<>(
MapDocumentUtil.getJPathString("$.id", line),
new ObjectMapper().readValue(line, clazz)));
// read next line
line = reader.readLine();
}
reader.close();
} catch (IOException e) {
e.printStackTrace();
}
return res;
}
public String authorToString(Author a) {
String print = "Fullname = ";
print += a.getFullname() + " pid = [";
if (a.getPid() != null)
for (StructuredProperty sp : a.getPid()) {
print += sp.toComparableString() + " ";
}
print += "]";
return print;
}
}

File diff suppressed because one or more lines are too long

@ -105,6 +105,8 @@ public class ModelConstants {
public static final KeyValue UNKNOWN_REPOSITORY = keyValue(
"10|openaire____::55045bd2a65019fd8e6741a755395c8c", "Unknown Repository");
public static final Qualifier UNKNOWN_COUNTRY = qualifier(UNKNOWN, "Unknown", DNET_COUNTRY_TYPE, DNET_COUNTRY_TYPE);
private static Qualifier qualifier(
final String classid,
final String classname,

@ -243,7 +243,7 @@ public class Result extends OafEntity implements Serializable {
Result r = (Result) e;
// TODO consider merging also Measures
measures = mergeLists(measures, r.getMeasures());
instance = mergeLists(instance, r.getInstance());

@ -0,0 +1,28 @@
package eu.dnetlib.dhp.actionmanager.bipfinder;
import java.io.Serializable;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
/**
* Class that maps the model of the bipFinder! input data.
* Only needed for deserialization purposes
*/
public class BipDeserialize extends HashMap<String, List<Score>> implements Serializable {
public BipDeserialize() {
super();
}
public List<Score> get(String key) {
if (super.get(key) == null) {
return new ArrayList<>();
}
return super.get(key);
}
}

@ -0,0 +1,30 @@
package eu.dnetlib.dhp.actionmanager.bipfinder;
import java.io.Serializable;
import java.util.List;
/**
* Rewriting of the bipFinder input data by extracting the identifier of the result (doi)
*/
public class BipScore implements Serializable {
private String id; // doi
private List<Score> scoreList; // unit as given in the inputfile
public String getId() {
return id;
}
public void setId(String id) {
this.id = id;
}
public List<Score> getScoreList() {
return scoreList;
}
public void setScoreList(List<Score> scoreList) {
this.scoreList = scoreList;
}
}

@ -0,0 +1,85 @@
package eu.dnetlib.dhp.actionmanager.bipfinder;
import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession;
import java.io.Serializable;
import java.util.Optional;
import org.apache.commons.io.IOUtils;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.mapred.SequenceFileOutputFormat;
import org.apache.spark.SparkConf;
import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.sql.SparkSession;
import org.slf4j.Logger;
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.oaf.Result;
/**
* Just collects all the atomic actions produced for the different results and saves them in
* outputpath for the ActionSet
*/
public class CollectAndSave implements Serializable {
private static final Logger log = LoggerFactory.getLogger(CollectAndSave.class);
private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
public static <I extends Result> void main(String[] args) throws Exception {
String jsonConfiguration = IOUtils
.toString(
CollectAndSave.class
.getResourceAsStream(
"/eu/dnetlib/dhp/actionmanager/bipfinder/input_actionset_parameter.json"));
final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration);
parser.parseArgument(args);
Boolean isSparkSessionManaged = Optional
.ofNullable(parser.get("isSparkSessionManaged"))
.map(Boolean::valueOf)
.orElse(Boolean.TRUE);
log.info("isSparkSessionManaged: {}", isSparkSessionManaged);
final String inputPath = parser.get("inputPath");
log.info("inputPath {}: ", inputPath);
final String outputPath = parser.get("outputPath");
log.info("outputPath {}: ", outputPath);
SparkConf conf = new SparkConf();
runWithSparkSession(
conf,
isSparkSessionManaged,
spark -> {
removeOutputDir(spark, outputPath);
collectAndSave(spark, inputPath, outputPath);
});
}
private static void collectAndSave(SparkSession spark, String inputPath, String outputPath) {
JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext());
sc
.sequenceFile(inputPath + "/publication", Text.class, Text.class)
.union(sc.sequenceFile(inputPath + "/dataset", Text.class, Text.class))
.union(sc.sequenceFile(inputPath + "/otherresearchproduct", Text.class, Text.class))
.union(sc.sequenceFile(inputPath + "/software", Text.class, Text.class))
.saveAsHadoopFile(outputPath, Text.class, Text.class, SequenceFileOutputFormat.class);
;
}
private static void removeOutputDir(SparkSession spark, String path) {
HdfsSupport.remove(path, spark.sparkContext().hadoopConfiguration());
}
}

@ -0,0 +1,26 @@
package eu.dnetlib.dhp.actionmanager.bipfinder;
import java.io.Serializable;
public class KeyValue implements Serializable {
private String key;
private String value;
public String getKey() {
return key;
}
public void setKey(String key) {
this.key = key;
}
public String getValue() {
return value;
}
public void setValue(String value) {
this.value = value;
}
}

@ -0,0 +1,28 @@
package eu.dnetlib.dhp.actionmanager.bipfinder;
import java.io.Serializable;
/**
* Subset of the information of the generic results that are needed to create the atomic action
*/
public class PreparedResult implements Serializable {
private String id; // openaire id
private String value; // doi
public String getId() {
return id;
}
public void setId(String id) {
this.id = id;
}
public String getValue() {
return value;
}
public void setValue(String value) {
this.value = value;
}
}

@ -0,0 +1,30 @@
package eu.dnetlib.dhp.actionmanager.bipfinder;
import java.io.Serializable;
import java.util.List;
/**
* represents the score in the input file
*/
public class Score implements Serializable {
private String id;
private List<KeyValue> unit;
public String getId() {
return id;
}
public void setId(String id) {
this.id = id;
}
public List<KeyValue> getUnit() {
return unit;
}
public void setUnit(List<KeyValue> unit) {
this.unit = unit;
}
}

@ -0,0 +1,200 @@
package eu.dnetlib.dhp.actionmanager.bipfinder;
import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession;
import java.io.Serializable;
import java.util.List;
import java.util.Optional;
import java.util.stream.Collectors;
import org.apache.commons.io.IOUtils;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.mapred.SequenceFileOutputFormat;
import org.apache.spark.SparkConf;
import org.apache.spark.api.java.JavaRDD;
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.sql.Dataset;
import org.apache.spark.sql.Encoders;
import org.apache.spark.sql.SparkSession;
import org.slf4j.Logger;
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.*;
import eu.dnetlib.dhp.schema.oaf.KeyValue;
import scala.Tuple2;
/**
* created the Atomic Action for each tipe of results
*/
public class SparkAtomicActionScoreJob implements Serializable {
private static String DOI = "doi";
private static final Logger log = LoggerFactory.getLogger(SparkAtomicActionScoreJob.class);
private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
public static <I extends Result> void main(String[] args) throws Exception {
String jsonConfiguration = IOUtils
.toString(
SparkAtomicActionScoreJob.class
.getResourceAsStream(
"/eu/dnetlib/dhp/actionmanager/bipfinder/input_parameters.json"));
final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration);
parser.parseArgument(args);
Boolean isSparkSessionManaged = Optional
.ofNullable(parser.get("isSparkSessionManaged"))
.map(Boolean::valueOf)
.orElse(Boolean.TRUE);
log.info("isSparkSessionManaged: {}", isSparkSessionManaged);
final String inputPath = parser.get("inputPath");
log.info("inputPath {}: ", inputPath);
final String outputPath = parser.get("outputPath");
log.info("outputPath {}: ", outputPath);
final String bipScorePath = parser.get("bipScorePath");
log.info("bipScorePath: {}", bipScorePath);
final String resultClassName = parser.get("resultTableName");
log.info("resultTableName: {}", resultClassName);
Class<I> inputClazz = (Class<I>) Class.forName(resultClassName);
SparkConf conf = new SparkConf();
runWithSparkSession(
conf,
isSparkSessionManaged,
spark -> {
removeOutputDir(spark, outputPath);
prepareResults(spark, inputPath, outputPath, bipScorePath, inputClazz);
});
}
private static <I extends Result> void prepareResults(SparkSession spark, String inputPath, String outputPath,
String bipScorePath, Class<I> inputClazz) {
final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext());
JavaRDD<BipDeserialize> bipDeserializeJavaRDD = sc
.textFile(bipScorePath)
.map(item -> OBJECT_MAPPER.readValue(item, BipDeserialize.class));
Dataset<BipScore> bipScores = spark
.createDataset(bipDeserializeJavaRDD.flatMap(entry -> entry.keySet().stream().map(key -> {
BipScore bs = new BipScore();
bs.setId(key);
bs.setScoreList(entry.get(key));
return bs;
}).collect(Collectors.toList()).iterator()).rdd(), Encoders.bean(BipScore.class));
System.out.println(bipScores.count());
Dataset<I> results = readPath(spark, inputPath, inputClazz);
results.createOrReplaceTempView("result");
Dataset<PreparedResult> preparedResult = spark
.sql(
"select pIde.value value, id " +
"from result " +
"lateral view explode (pid) p as pIde " +
"where dataInfo.deletedbyinference = false and pIde.qualifier.classid = '" + DOI + "'")
.as(Encoders.bean(PreparedResult.class));
bipScores
.joinWith(
preparedResult, bipScores.col("id").equalTo(preparedResult.col("value")),
"inner")
.map((MapFunction<Tuple2<BipScore, PreparedResult>, BipScore>) value -> {
BipScore ret = value._1();
ret.setId(value._2().getId());
return ret;
}, Encoders.bean(BipScore.class))
.groupByKey((MapFunction<BipScore, String>) value -> value.getId(), Encoders.STRING())
.mapGroups((MapGroupsFunction<String, BipScore, Result>) (k, it) -> {
Result ret = new Result();
ret.setDataInfo(getDataInfo());
BipScore first = it.next();
ret.setId(first.getId());
ret.setMeasures(getMeasure(first));
it.forEachRemaining(value -> ret.getMeasures().addAll(getMeasure(value)));
return ret;
}, Encoders.bean(Result.class))
.toJavaRDD()
.map(p -> new AtomicAction(inputClazz, p))
.mapToPair(
aa -> new Tuple2<>(new Text(aa.getClazz().getCanonicalName()),
new Text(OBJECT_MAPPER.writeValueAsString(aa))))
.saveAsHadoopFile(outputPath, Text.class, Text.class, SequenceFileOutputFormat.class);
}
private static List<Measure> getMeasure(BipScore value) {
return value
.getScoreList()
.stream()
.map(score -> {
Measure m = new Measure();
m.setId(score.getId());
m
.setUnit(
score
.getUnit()
.stream()
.map(unit -> {
KeyValue kv = new KeyValue();
kv.setValue(unit.getValue());
kv.setKey(unit.getKey());
kv.setDataInfo(getDataInfo());
return kv;
})
.collect(Collectors.toList()));
return m;
})
.collect(Collectors.toList());
}
private static DataInfo getDataInfo() {
DataInfo di = new DataInfo();
di.setInferred(false);
di.setInvisible(false);
di.setDeletedbyinference(false);
di.setTrust("");
Qualifier qualifier = new Qualifier();
qualifier.setClassid("sysimport:actionset");
qualifier.setClassname("Harvested");
qualifier.setSchemename("dnet:provenanceActions");
qualifier.setSchemeid("dnet:provenanceActions");
di.setProvenanceaction(qualifier);
return di;
}
private static void removeOutputDir(SparkSession spark, String path) {
HdfsSupport.remove(path, spark.sparkContext().hadoopConfiguration());
}
public static <R> Dataset<R> readPath(
SparkSession spark, String inputPath, Class<R> clazz) {
return spark
.read()
.textFile(inputPath)
.map((MapFunction<String, R>) value -> OBJECT_MAPPER.readValue(value, clazz), Encoders.bean(clazz));
}
}

@ -0,0 +1,20 @@
[
{
"paramName": "issm",
"paramLongName": "isSparkSessionManaged",
"paramDescription": "when true will stop SparkSession after job execution",
"paramRequired": false
},
{
"paramName": "ip",
"paramLongName": "inputPath",
"paramDescription": "the URL from where to get the programme file",
"paramRequired": true
},
{
"paramName": "o",
"paramLongName": "outputPath",
"paramDescription": "the path of the new ActionSet",
"paramRequired": true
}
]

@ -0,0 +1,32 @@
[
{
"paramName": "issm",
"paramLongName": "isSparkSessionManaged",
"paramDescription": "when true will stop SparkSession after job execution",
"paramRequired": false
},
{
"paramName": "ip",
"paramLongName": "inputPath",
"paramDescription": "the URL from where to get the programme file",
"paramRequired": true
},
{
"paramName": "o",
"paramLongName": "outputPath",
"paramDescription": "the path of the new ActionSet",
"paramRequired": true
},
{
"paramName": "rtn",
"paramLongName": "resultTableName",
"paramDescription": "the path of the new ActionSet",
"paramRequired": true
},
{
"paramName": "bsp",
"paramLongName": "bipScorePath",
"paramDescription": "the path of the new ActionSet",
"paramRequired": true
}
]

@ -0,0 +1,171 @@
<workflow-app name="BipFinderScore" xmlns="uri:oozie:workflow:0.5">
<parameters>
<property>
<name>inputPath</name>
<description>the input path of the resources to be extended</description>
</property>
<property>
<name>bipScorePath</name>
<description>the path where to find the bipFinder scores</description>
</property>
<property>
<name>outputPath</name>
<description>the path where to store the actionset</description>
</property>
</parameters>
<start to="deleteoutputpath"/>
<kill name="Kill">
<message>Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}]</message>
</kill>
<action name="deleteoutputpath">
<fs>
<delete path="${outputPath}"/>
<mkdir path="${outputPath}"/>
<delete path="${workingDir}"/>
<mkdir path="${workingDir}"/>
</fs>
<ok to="atomicactions"/>
<error to="Kill"/>
</action>
<fork name="atomicactions">
<path start="atomicactions_publication"/>
<path start="atomicactions_dataset"/>
<path start="atomicactions_orp"/>
<path start="atomicactions_software"/>
</fork>
<action name="atomicactions_publication">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn</master>
<mode>cluster</mode>
<name>Produces the atomic action with the bip finder scores for publications</name>
<class>eu.dnetlib.dhp.actionmanager.bipfinder.SparkAtomicActionScoreJob</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>--inputPath</arg><arg>${inputPath}/publication</arg>
<arg>--resultTableName</arg><arg>eu.dnetlib.dhp.schema.oaf.Publication</arg>
<arg>--outputPath</arg><arg>${workingDir}/publication</arg>
<arg>--bipScorePath</arg><arg>${bipScorePath}</arg>
</spark>
<ok to="join_aa"/>
<error to="Kill"/>
</action>
<action name="atomicactions_dataset">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn</master>
<mode>cluster</mode>
<name>Produces the atomic action with the bip finder scores for datasets</name>
<class>eu.dnetlib.dhp.actionmanager.bipfinder.SparkAtomicActionScoreJob</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>--inputPath</arg><arg>${inputPath}/dataset</arg>
<arg>--resultTableName</arg><arg>eu.dnetlib.dhp.schema.oaf.Dataset</arg>
<arg>--outputPath</arg><arg>${workingDir}/dataset</arg>
<arg>--bipScorePath</arg><arg>${bipScorePath}</arg>
</spark>
<ok to="join_aa"/>
<error to="Kill"/>
</action>
<action name="atomicactions_orp">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn</master>
<mode>cluster</mode>
<name>Produces the atomic action with the bip finder scores for orp</name>
<class>eu.dnetlib.dhp.actionmanager.bipfinder.SparkAtomicActionScoreJob</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>--inputPath</arg><arg>${inputPath}/otherresearchproduct</arg>
<arg>--resultTableName</arg><arg>eu.dnetlib.dhp.schema.oaf.OtherResearchProduct</arg>
<arg>--outputPath</arg><arg>${workingDir}/otherresearchproduct</arg>
<arg>--bipScorePath</arg><arg>${bipScorePath}</arg>
</spark>
<ok to="join_aa"/>
<error to="Kill"/>
</action>
<action name="atomicactions_software">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn</master>
<mode>cluster</mode>
<name>Produces the atomic action with the bip finder scores for software</name>
<class>eu.dnetlib.dhp.actionmanager.bipfinder.SparkAtomicActionScoreJob</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>--inputPath</arg><arg>${inputPath}/software</arg>
<arg>--resultTableName</arg><arg>eu.dnetlib.dhp.schema.oaf.Software</arg>
<arg>--outputPath</arg><arg>${workingDir}/software</arg>
<arg>--bipScorePath</arg><arg>${bipScorePath}</arg>
</spark>
<ok to="join_aa"/>
<error to="Kill"/>
</action>
<join name="join_aa" to="collectandsave"/>
<action name="collectandsave">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn</master>
<mode>cluster</mode>
<name>saves all the aa produced for the several types of results in the as output path</name>
<class>eu.dnetlib.dhp.actionmanager.bipfinder.CollectAndSave</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>--inputPath</arg><arg>${workingDir}</arg>
<arg>--outputPath</arg><arg>${outputPath}</arg>
</spark>
<ok to="End"/>
<error to="Kill"/>
</action>
<end name="End"/>
</workflow-app>

@ -0,0 +1,323 @@
package eu.dnetlib.dhp.actionmanager.bipfinder;
import java.io.IOException;
import java.nio.file.Files;
import java.nio.file.Path;
import java.util.List;
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.Dataset;
import org.apache.spark.sql.Encoders;
import org.apache.spark.sql.Row;
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.Publication;
public class SparkAtomicActionScoreJobTest {
private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
private static SparkSession spark;
private static Path workingDir;
private static final Logger log = LoggerFactory
.getLogger(SparkAtomicActionScoreJobTest.class);
@BeforeAll
public static void beforeAll() throws IOException {
workingDir = Files
.createTempDirectory(SparkAtomicActionScoreJobTest.class.getSimpleName());
log.info("using work dir {}", workingDir);
SparkConf conf = new SparkConf();
conf.setAppName(SparkAtomicActionScoreJobTest.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(SparkAtomicActionScoreJobTest.class.getSimpleName())
.config(conf)
.getOrCreate();
}
@AfterAll
public static void afterAll() throws IOException {
FileUtils.deleteDirectory(workingDir.toFile());
spark.stop();
}
@Test
public void matchOne() throws Exception {
String bipScoresPath = getClass()
.getResource("/eu/dnetlib/dhp/actionmanager/bipfinder/bip_scores.json")
.getPath();
String inputPath = getClass()
.getResource(
"/eu/dnetlib/dhp/actionmanager/bipfinder/publication.json")
.getPath();
SparkAtomicActionScoreJob
.main(
new String[] {
"-isSparkSessionManaged",
Boolean.FALSE.toString(),
"-inputPath",
inputPath,
"-bipScorePath",
bipScoresPath,
"-resultTableName",
"eu.dnetlib.dhp.schema.oaf.Publication",
"-outputPath",
workingDir.toString() + "/actionSet"
});
final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext());
JavaRDD<Publication> tmp = sc
.sequenceFile(workingDir.toString() + "/actionSet", Text.class, Text.class)
.map(value -> OBJECT_MAPPER.readValue(value._2().toString(), AtomicAction.class))
.map(aa -> ((Publication) aa.getPayload()));
Assertions.assertTrue(tmp.count() == 1);
Dataset<Publication> verificationDataset = spark.createDataset(tmp.rdd(), Encoders.bean(Publication.class));
verificationDataset.createOrReplaceTempView("publication");
Dataset<Row> execVerification = spark
.sql(
"Select p.id oaid, mes.id, mUnit.value from publication p " +
"lateral view explode(measures) m as mes " +
"lateral view explode(mes.unit) u as mUnit ");
Assertions.assertEquals(2, execVerification.count());
Assertions
.assertEquals(
"50|355e65625b88::ffa5bad14f4adc0c9a15c00efbbccddb",
execVerification.select("oaid").collectAsList().get(0).getString(0));
Assertions
.assertEquals(
"1.47565045883e-08",
execVerification.filter("id = 'influence'").select("value").collectAsList().get(0).getString(0));
Assertions
.assertEquals(
"0.227515392",
execVerification.filter("id = 'popularity'").select("value").collectAsList().get(0).getString(0));
}
@Test
public void matchOneWithTwo() throws Exception {
String bipScoresPath = getClass()
.getResource("/eu/dnetlib/dhp/actionmanager/bipfinder/bip_scores.json")
.getPath();
String inputPath = getClass()
.getResource(
"/eu/dnetlib/dhp/actionmanager/bipfinder/publication_2.json")
.getPath();
SparkAtomicActionScoreJob
.main(
new String[] {
"-isSparkSessionManaged",
Boolean.FALSE.toString(),
"-inputPath",
inputPath,
"-bipScorePath",
bipScoresPath,
"-resultTableName",
"eu.dnetlib.dhp.schema.oaf.Publication",
"-outputPath",
workingDir.toString() + "/actionSet"
});
final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext());
JavaRDD<Publication> tmp = sc
.sequenceFile(workingDir.toString() + "/actionSet", Text.class, Text.class)
.map(value -> OBJECT_MAPPER.readValue(value._2().toString(), AtomicAction.class))
.map(aa -> ((Publication) aa.getPayload()));
Assertions.assertTrue(tmp.count() == 1);
Dataset<Publication> verificationDataset = spark.createDataset(tmp.rdd(), Encoders.bean(Publication.class));
verificationDataset.createOrReplaceTempView("publication");
Dataset<Row> execVerification = spark
.sql(
"Select p.id oaid, mes.id, mUnit.value from publication p " +
"lateral view explode(measures) m as mes " +
"lateral view explode(mes.unit) u as mUnit ");
Assertions.assertEquals(4, execVerification.count());
Assertions
.assertEquals(
"50|355e65625b88::ffa5bad14f4adc0c9a15c00efbbccddb",
execVerification.select("oaid").collectAsList().get(0).getString(0));
Assertions
.assertEquals(
2,
execVerification.filter("id = 'influence'").count());
Assertions
.assertEquals(
2,
execVerification.filter("id = 'popularity'").count());
List<Row> tmp_ds = execVerification.filter("id = 'influence'").select("value").collectAsList();
String tmp_influence = tmp_ds.get(0).getString(0);
Assertions
.assertTrue(
"1.47565045883e-08".equals(tmp_influence) ||
"1.98956540239e-08".equals(tmp_influence));
tmp_influence = tmp_ds.get(1).getString(0);
Assertions
.assertTrue(
"1.47565045883e-08".equals(tmp_influence) ||
"1.98956540239e-08".equals(tmp_influence));
Assertions.assertTrue(!tmp_ds.get(0).getString(0).equals(tmp_ds.get(1).getString(0)));
}
@Test
public void matchTwo() throws Exception {
String bipScoresPath = getClass()
.getResource("/eu/dnetlib/dhp/actionmanager/bipfinder/bip_scores.json")
.getPath();
String inputPath = getClass()
.getResource(
"/eu/dnetlib/dhp/actionmanager/bipfinder/publication_3.json")
.getPath();
SparkAtomicActionScoreJob
.main(
new String[] {
"-isSparkSessionManaged",
Boolean.FALSE.toString(),
"-inputPath",
inputPath,
"-bipScorePath",
bipScoresPath,
"-resultTableName",
"eu.dnetlib.dhp.schema.oaf.Publication",
"-outputPath",
workingDir.toString() + "/actionSet"
});
final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext());
JavaRDD<Publication> tmp = sc
.sequenceFile(workingDir.toString() + "/actionSet", Text.class, Text.class)
.map(value -> OBJECT_MAPPER.readValue(value._2().toString(), AtomicAction.class))
.map(aa -> ((Publication) aa.getPayload()));
Assertions.assertTrue(tmp.count() == 2);
Dataset<Publication> verificationDataset = spark.createDataset(tmp.rdd(), Encoders.bean(Publication.class));
verificationDataset.createOrReplaceTempView("publication");
Dataset<Row> execVerification = spark
.sql(
"Select p.id oaid, mes.id, mUnit.value from publication p " +
"lateral view explode(measures) m as mes " +
"lateral view explode(mes.unit) u as mUnit ");
Assertions.assertEquals(4, execVerification.count());
Assertions
.assertEquals(
2,
execVerification.filter("oaid = '50|355e65625b88::ffa5bad14f4adc0c9a15c00efbbccddb'").count());
Assertions
.assertEquals(
2,
execVerification.filter("oaid = '50|acm_________::faed5b7a1bd8f51118d13ed29cfaee09'").count());
Assertions
.assertEquals(
2,
execVerification.filter("id = 'influence'").count());
Assertions
.assertEquals(
2,
execVerification.filter("id = 'popularity'").count());
Assertions
.assertEquals(
"1.47565045883e-08",
execVerification
.filter(
"oaid = '50|355e65625b88::ffa5bad14f4adc0c9a15c00efbbccddb' " +
"and id = 'influence'")
.select("value")
.collectAsList()
.get(0)
.getString(0));
Assertions
.assertEquals(
"1.98956540239e-08",
execVerification
.filter(
"oaid = '50|acm_________::faed5b7a1bd8f51118d13ed29cfaee09' " +
"and id = 'influence'")
.select("value")
.collectAsList()
.get(0)
.getString(0));
Assertions
.assertEquals(
"0.282046161584",
execVerification
.filter(
"oaid = '50|acm_________::faed5b7a1bd8f51118d13ed29cfaee09' " +
"and id = 'popularity'")
.select("value")
.collectAsList()
.get(0)
.getString(0));
Assertions
.assertEquals(
"0.227515392",
execVerification
.filter(
"oaid = '50|355e65625b88::ffa5bad14f4adc0c9a15c00efbbccddb' " +
"and id = 'popularity'")
.select("value")
.collectAsList()
.get(0)
.getString(0));
}
}

@ -87,7 +87,7 @@
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
--conf spark.sql.shuffle.partitions=3840
</spark-opts>
<arg>--workingPath</arg><arg>${workingPath}</arg>
<arg>--outputDir</arg><arg>${outputDir}</arg>
<arg>--opendoarIds</arg><arg>${opendoarIds}</arg>
</spark>
<ok to="End"/>

@ -10,10 +10,11 @@ import java.io.Serializable;
import java.nio.file.Paths;
import java.util.*;
import org.codehaus.jackson.map.ObjectMapper;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import com.fasterxml.jackson.databind.ObjectMapper;
import eu.dnetlib.dhp.oa.merge.AuthorMerger;
import eu.dnetlib.dhp.schema.oaf.*;
import eu.dnetlib.pace.util.MapDocumentUtil;
@ -100,8 +101,8 @@ public class EntityMergerTest implements Serializable {
assertEquals(pub_merged.getDateofacceptance().getValue(), "2018-09-30");
// verify authors
assertEquals(pub_merged.getAuthor().size(), 9);
assertEquals(AuthorMerger.countAuthorsPids(pub_merged.getAuthor()), 4);
assertEquals(13, pub_merged.getAuthor().size());
assertEquals(4, AuthorMerger.countAuthorsPids(pub_merged.getAuthor()));
// verify title
int count = 0;

@ -7,7 +7,6 @@ import java.util.List;
import org.apache.commons.io.IOUtils;
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.FlatMapFunction;
import org.apache.spark.api.java.function.MapFunction;
import org.apache.spark.api.java.function.PairFunction;
@ -16,8 +15,8 @@ import org.apache.spark.rdd.RDD;
import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Encoders;
import org.apache.spark.sql.SparkSession;
import org.codehaus.jackson.map.ObjectMapper;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.hash.Hashing;
import eu.dnetlib.dedup.graph.ConnectedComponent;

@ -10,7 +10,8 @@ import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.api.java.function.MapFunction;
import org.apache.spark.sql.Encoders;
import org.apache.spark.sql.SparkSession;
import org.codehaus.jackson.map.ObjectMapper;
import com.fasterxml.jackson.databind.ObjectMapper;
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
import eu.dnetlib.dhp.schema.oaf.Oaf;

@ -4,14 +4,13 @@ import eu.dnetlib.dhp.schema.action.AtomicAction
import eu.dnetlib.dhp.schema.oaf.{DataInfo, Dataset, Field, Instance, KeyValue, Oaf, Organization, Publication, Qualifier, Relation, Result, StructuredProperty}
import eu.dnetlib.dhp.utils.DHPUtils
import org.apache.commons.lang3.StringUtils
import org.codehaus.jackson.map.ObjectMapper
import com.fasterxml.jackson.databind.ObjectMapper
import org.json4s
import org.json4s.DefaultFormats
import org.json4s.jackson.JsonMethods.parse
import org.slf4j.{Logger, LoggerFactory}
import scala.collection.JavaConverters._
import scala.io.Source
case class HostedByItemType(id: String, officialname: String, issn: String, eissn: String, lissn: String, openAccess: Boolean) {}
@ -19,23 +18,18 @@ case class HostedByItemType(id: String, officialname: String, issn: String, eiss
case class DoiBoostAffiliation(PaperId:Long, AffiliationId:Long, GridId:Option[String], OfficialPage:Option[String], DisplayName:Option[String]){}
object DoiBoostMappingUtil {
def getUnknownCountry(): Qualifier = {
createQualifier("UNKNOWN","UNKNOWN","dnet:countries","dnet:countries")
}
def generateMAGAffiliationId(affId: String): String = {
s"20|microsoft___$SEPARATOR${DHPUtils.md5(affId)}"
}
val logger: Logger = LoggerFactory.getLogger(getClass)
//STATIC STRING
val MAG = "microsoft"
val MAG_NAME = "Microsoft Academic Graph"
val ORCID = "ORCID"
val ORCID = "orcid"
val ORCID_PENDING = "orcid_pending"
val CROSSREF = "Crossref"
val UNPAYWALL = "UnpayWall"
val GRID_AC = "grid.ac"

@ -39,33 +39,38 @@ object SparkGenerateDOIBoostActionSet {
val dbaffiliationRelationPath = parser.get("dbaffiliationRelationPath")
val dbOrganizationPath = parser.get("dbOrganizationPath")
val workingDirPath = parser.get("targetPath")
val sequenceFilePath = parser.get("sFilePath")
spark.read.load(dbDatasetPath).as[OafDataset]
val asDataset = spark.read.load(dbDatasetPath).as[OafDataset]
.map(d =>DoiBoostMappingUtil.fixResult(d))
.map(d=>DoiBoostMappingUtil.toActionSet(d))(Encoders.tuple(Encoders.STRING, Encoders.STRING))
.write.mode(SaveMode.Overwrite).save(s"$workingDirPath/actionSet")
// .write.mode(SaveMode.Overwrite).save(s"$workingDirPath/actionSet")
spark.read.load(dbPublicationPath).as[Publication]
val asPublication =spark.read.load(dbPublicationPath).as[Publication]
.map(d=>DoiBoostMappingUtil.toActionSet(d))(Encoders.tuple(Encoders.STRING, Encoders.STRING))
.write.mode(SaveMode.Append).save(s"$workingDirPath/actionSet")
// .write.mode(SaveMode.Append).save(s"$workingDirPath/actionSet")
spark.read.load(dbOrganizationPath).as[Organization]
val asOrganization = spark.read.load(dbOrganizationPath).as[Organization]
.map(d=>DoiBoostMappingUtil.toActionSet(d))(Encoders.tuple(Encoders.STRING, Encoders.STRING))
.write.mode(SaveMode.Append).save(s"$workingDirPath/actionSet")
// .write.mode(SaveMode.Append).save(s"$workingDirPath/actionSet")
spark.read.load(crossRefRelation).as[Relation]
val asCRelation = spark.read.load(crossRefRelation).as[Relation]
.map(d=>DoiBoostMappingUtil.toActionSet(d))(Encoders.tuple(Encoders.STRING, Encoders.STRING))
.write.mode(SaveMode.Append).save(s"$workingDirPath/actionSet")
// .write.mode(SaveMode.Append).save(s"$workingDirPath/actionSet")
spark.read.load(dbaffiliationRelationPath).as[Relation]
val asRelAffiliation = spark.read.load(dbaffiliationRelationPath).as[Relation]
.map(d=>DoiBoostMappingUtil.toActionSet(d))(Encoders.tuple(Encoders.STRING, Encoders.STRING))
.write.mode(SaveMode.Append).save(s"$workingDirPath/actionSet")
// .write.mode(SaveMode.Append).save(s"$workingDirPath/actionSet")
val d: Dataset[(String, String)] =spark.read.load(s"$workingDirPath/actionSet").as[(String,String)]
d.rdd.map(s => (new Text(s._1), new Text(s._2))).saveAsHadoopFile(s"$workingDirPath/rawset", classOf[Text], classOf[Text], classOf[SequenceFileOutputFormat[Text,Text]], classOf[GzipCodec])
val d: Dataset[(String, String)] = asDataset.union(asPublication).union(asOrganization).union(asCRelation).union(asRelAffiliation)
// spark.read.load(s"$workingDirPath/actionSet").as[(String,String)]
d.rdd.repartition(6000).map(s => (new Text(s._1), new Text(s._2))).saveAsHadoopFile(s"$sequenceFilePath", classOf[Text], classOf[Text], classOf[SequenceFileOutputFormat[Text,Text]], classOf[GzipCodec])

@ -2,6 +2,7 @@ package eu.dnetlib.doiboost
import eu.dnetlib.dhp.application.ArgumentApplicationParser
import eu.dnetlib.dhp.oa.merge.AuthorMerger
import eu.dnetlib.dhp.schema.common.ModelConstants
import eu.dnetlib.dhp.schema.oaf.{Organization, Publication, Relation, Dataset => OafDataset}
import eu.dnetlib.doiboost.mag.ConversionUtil
import org.apache.commons.io.IOUtils
@ -30,7 +31,7 @@ object SparkGenerateDoiBoost {
import spark.implicits._
val hostedByMapPath = parser.get("hostedByMapPath")
val workingDirPath = parser.get("workingDirPath")
val workingDirPath = parser.get("workingPath")
implicit val mapEncoderPub: Encoder[Publication] = Encoders.kryo[Publication]
@ -132,7 +133,7 @@ object SparkGenerateDoiBoost {
o.setLegalname(DoiBoostMappingUtil.asField(affiliation.DisplayName.get))
if (affiliation.OfficialPage.isDefined)
o.setWebsiteurl(DoiBoostMappingUtil.asField(affiliation.OfficialPage.get))
o.setCountry(DoiBoostMappingUtil.getUnknownCountry())
o.setCountry(ModelConstants.UNKNOWN_COUNTRY)
o
}
else

@ -200,7 +200,7 @@ case object Crossref2Oaf {
a.setSurname(family)
a.setFullname(s"$given $family")
if (StringUtils.isNotBlank(orcid))
a.setPid(List(createSP(orcid, ORCID, PID_TYPES, generateDataInfo())).asJava)
a.setPid(List(createSP(orcid, ORCID_PENDING, PID_TYPES, generateDataInfo())).asJava)
a
}

@ -2,18 +2,16 @@
package eu.dnetlib.doiboost.crossref;
import java.io.ByteArrayOutputStream;
import java.util.Optional;
import java.util.zip.Inflater;
import org.apache.commons.codec.binary.Base64;
import org.apache.commons.io.IOUtils;
import org.apache.commons.lang3.StringUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.IntWritable;
import org.apache.hadoop.io.SequenceFile;
import org.apache.hadoop.io.Text;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
@ -30,34 +28,45 @@ public class CrossrefImporter {
parser.parseArgument(args);
final String hdfsuri = parser.get("namenode");
System.out.println("HDFS URI" + hdfsuri);
Path hdfswritepath = new Path(parser.get("targetPath"));
System.out.println("TargetPath: " + hdfsuri);
final String namenode = parser.get("namenode");
System.out.println("namenode: " + namenode);
final Long timestamp = StringUtils.isNotBlank(parser.get("timestamp"))
? Long.parseLong(parser.get("timestamp"))
: -1;
Path targetPath = new Path(parser.get("targetPath"));
System.out.println("targetPath: " + targetPath);
if (timestamp > 0)
System.out.println("Timestamp added " + timestamp);
final Long timestamp = Optional
.ofNullable(parser.get("timestamp"))
.map(s -> {
try {
return Long.parseLong(s);
} catch (NumberFormatException e) {
return -1L;
}
})
.orElse(-1L);
System.out.println("timestamp: " + timestamp);
final String esServer = parser.get("esServer");
System.out.println("esServer: " + esServer);
final String esIndex = parser.get("esIndex");
System.out.println("esIndex: " + esIndex);
// ====== Init HDFS File System Object
Configuration conf = new Configuration();
// Set FileSystem URI
conf.set("fs.defaultFS", hdfsuri);
conf.set("fs.defaultFS", namenode);
// Because of Maven
conf.set("fs.hdfs.impl", org.apache.hadoop.hdfs.DistributedFileSystem.class.getName());
conf.set("fs.file.impl", org.apache.hadoop.fs.LocalFileSystem.class.getName());
ESClient client = timestamp > 0
? new ESClient("ip-90-147-167-25.ct1.garrservices.it", "crossref", timestamp)
: new ESClient("ip-90-147-167-25.ct1.garrservices.it", "crossref");
// "ip-90-147-167-25.ct1.garrservices.it", "crossref"
final ESClient client = new ESClient(esServer, esIndex, timestamp);
try (SequenceFile.Writer writer = SequenceFile
.createWriter(
conf,
SequenceFile.Writer.file(hdfswritepath),
SequenceFile.Writer.file(targetPath),
SequenceFile.Writer.keyClass(IntWritable.class),
SequenceFile.Writer.valueClass(Text.class))) {
@ -74,8 +83,7 @@ public class CrossrefImporter {
end = System.currentTimeMillis();
final float time = (end - start) / 1000.0F;
System.out
.println(
String.format("Imported %d records last 100000 imported in %f seconds", i, time));
.println(String.format("Imported %s records last 100000 imported in %s seconds", i, time));
start = System.currentTimeMillis();
}
}

@ -1,11 +1,11 @@
package eu.dnetlib.doiboost.crossref;
import java.io.IOException;
import java.util.Iterator;
import java.util.List;
import org.apache.commons.io.IOUtils;
import org.apache.http.HttpHeaders;
import org.apache.http.client.methods.CloseableHttpResponse;
import org.apache.http.client.methods.HttpPost;
import org.apache.http.entity.StringEntity;
@ -17,13 +17,17 @@ import org.slf4j.LoggerFactory;
import com.jayway.jsonpath.JsonPath;
public class ESClient implements Iterator<String> {
private static final Logger logger = LoggerFactory.getLogger(ESClient.class);
static final String blobPath = "$.hits[*].hits[*]._source.blob";
static final String scrollIdPath = "$._scroll_id";
static final String JSON_NO_TS = "{\"size\":1000}";
static final String JSON_WITH_TS = "{\"size\":1000, \"query\":{\"range\":{\"timestamp\":{\"gte\":%d}}}}";
static final String JSON_SCROLL = "{\"scroll_id\":\"%s\",\"scroll\" : \"1m\"}";
private static final String BLOB_PATH = "$.hits.hits[*]._source.blob";
private static final String SCROLL_ID_PATH = "$._scroll_id";
private static final String JSON_NO_TS = "{\"size\":1000}";
private static final String JSON_WITH_TS = "{\"size\":1000, \"query\":{\"range\":{\"timestamp\":{\"gte\":%d}}}}";
private static final String JSON_SCROLL = "{\"scroll_id\":\"%s\",\"scroll\" : \"1m\"}";
public static final String APPLICATION_JSON = "application/json";
public static final String ES_SEARCH_URL = "http://%s:9200/%s/_search?scroll=1m";
public static final String ES_SCROLL_URL = "http://%s:9200/_search/scroll";
private final String scrollId;
@ -31,47 +35,30 @@ public class ESClient implements Iterator<String> {
private final String esHost;
public ESClient(final String esHost, final String esIndex) throws IOException {
public ESClient(final String esHost, final String esIndex, final long timestamp) {
this.esHost = esHost;
final String body = getResponse(
String.format("http://%s:9200/%s/_search?scroll=1m", esHost, esIndex), JSON_NO_TS);
scrollId = getJPathString(scrollIdPath, body);
buffer = getBlobs(body);
}
public ESClient(final String esHost, final String esIndex, final long timestamp)
throws IOException {
this.esHost = esHost;
final String body = getResponse(
String.format("http://%s:9200/%s/_search?scroll=1m", esHost, esIndex),
String.format(JSON_WITH_TS, timestamp));
scrollId = getJPathString(scrollIdPath, body);
final String body = timestamp > 0
? getResponse(String.format(ES_SEARCH_URL, esHost, esIndex), String.format(JSON_WITH_TS, timestamp))
: getResponse(String.format(ES_SEARCH_URL, esHost, esIndex), JSON_NO_TS);
scrollId = getJPathString(SCROLL_ID_PATH, body);
buffer = getBlobs(body);
}
private String getResponse(final String url, final String json) {
CloseableHttpClient client = HttpClients.createDefault();
try {
try (CloseableHttpClient client = HttpClients.createDefault()) {
HttpPost httpPost = new HttpPost(url);
if (json != null) {
StringEntity entity = new StringEntity(json);
httpPost.setEntity(entity);
httpPost.setHeader("Accept", "application/json");
httpPost.setHeader("Content-type", "application/json");
httpPost.setHeader(HttpHeaders.ACCEPT, APPLICATION_JSON);
httpPost.setHeader(HttpHeaders.CONTENT_TYPE, APPLICATION_JSON);
}
try (CloseableHttpResponse response = client.execute(httpPost)) {
return IOUtils.toString(response.getEntity().getContent());
}
CloseableHttpResponse response = client.execute(httpPost);
return IOUtils.toString(response.getEntity().getContent());
} catch (Throwable e) {
throw new RuntimeException("Error on executing request ", e);
} finally {
try {
client.close();
} catch (IOException e) {
throw new RuntimeException("Unable to close client ", e);
}
}
}
@ -87,7 +74,7 @@ public class ESClient implements Iterator<String> {
}
private List<String> getBlobs(final String body) {
final List<String> res = JsonPath.read(body, "$.hits.hits[*]._source.blob");
final List<String> res = JsonPath.read(body, BLOB_PATH);
return res;
}
@ -102,11 +89,11 @@ public class ESClient implements Iterator<String> {
if (buffer.isEmpty()) {
final String json_param = String.format(JSON_SCROLL, scrollId);
final String body = getResponse(String.format("http://%s:9200/_search/scroll", esHost), json_param);
final String body = getResponse(String.format(ES_SCROLL_URL, esHost), json_param);
try {
buffer = getBlobs(body);
} catch (Throwable e) {
logger.error("Error on get next page: body:" + body);
System.out.println("Error on get next page: body:" + body);
}
}
return nextItem;

@ -11,7 +11,7 @@ import org.slf4j.{Logger, LoggerFactory}
import scala.collection.JavaConverters._
object SparkPreProcessMAG {
object SparkProcessMAG {
def main(args: Array[String]): Unit = {
val logger: Logger = LoggerFactory.getLogger(getClass)

@ -1,11 +1,11 @@
package eu.dnetlib.doiboost.orcid
import com.fasterxml.jackson.databind.ObjectMapper
import eu.dnetlib.dhp.schema.oaf.{Author, DataInfo, Publication}
import eu.dnetlib.dhp.schema.orcid.OrcidDOI
import eu.dnetlib.doiboost.DoiBoostMappingUtil
import eu.dnetlib.doiboost.DoiBoostMappingUtil.{ORCID, PID_TYPES, createSP, generateDataInfo, generateIdentifier}
import org.apache.commons.lang.StringUtils
import org.codehaus.jackson.map.ObjectMapper
import org.slf4j.{Logger, LoggerFactory}
import scala.collection.JavaConverters._
@ -18,7 +18,7 @@ case class ORCIDItem(oid:String,name:String,surname:String,creditName:String,err
case class ORCIDElement(doi:String, authors:List[ORCIDItem]) {}
object ORCIDToOAF {
val logger: Logger = LoggerFactory.getLogger(ORCIDToOAF.getClass)
val mapper = new ObjectMapper
val mapper = new ObjectMapper()
def isJsonValid(inputStr: String): Boolean = {
import java.io.IOException

@ -3,10 +3,8 @@ package eu.dnetlib.doiboost.orcid;
import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession;
import java.io.IOException;
import java.text.SimpleDateFormat;
import java.util.Date;
import java.util.List;
import java.util.Optional;
import org.apache.commons.io.IOUtils;
@ -18,11 +16,9 @@ import org.apache.http.impl.client.CloseableHttpClient;
import org.apache.http.impl.client.HttpClients;
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.Function;
import org.apache.spark.util.LongAccumulator;
import org.mortbay.log.Log;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -36,7 +32,7 @@ public class SparkDownloadOrcidAuthors {
static final String DATE_FORMAT = "yyyy-MM-dd HH:mm:ss";
static final String lastUpdate = "2020-09-29 00:00:00";
public static void main(String[] args) throws IOException, Exception {
public static void main(String[] args) throws Exception {
final ArgumentApplicationParser parser = new ArgumentApplicationParser(
IOUtils
@ -51,12 +47,12 @@ public class SparkDownloadOrcidAuthors {
.orElse(Boolean.TRUE);
logger.info("isSparkSessionManaged: {}", isSparkSessionManaged);
final String workingPath = parser.get("workingPath");
logger.info("workingPath: ", workingPath);
logger.info("workingPath: {}", workingPath);
final String outputPath = parser.get("outputPath");
logger.info("outputPath: ", outputPath);
logger.info("outputPath: {}", outputPath);
final String token = parser.get("token");
final String lambdaFileName = parser.get("lambdaFileName");
logger.info("lambdaFileName: ", lambdaFileName);
logger.info("lambdaFileName: {}", lambdaFileName);
SparkConf conf = new SparkConf();
runWithSparkSession(
@ -171,8 +167,8 @@ public class SparkDownloadOrcidAuthors {
}
private static boolean isModified(String orcidId, String modifiedDate) {
Date modifiedDateDt = null;
Date lastUpdateDt = null;
Date modifiedDateDt;
Date lastUpdateDt;
try {
if (modifiedDate.length() != 19) {
modifiedDate = modifiedDate.substring(0, 19);

@ -5,5 +5,6 @@
{"paramName": "cr", "paramLongName":"crossRefRelation", "paramDescription": "the UnpayWall Publication Path", "paramRequired": true},
{"paramName": "da", "paramLongName":"dbaffiliationRelationPath", "paramDescription": "the MAG Publication Path", "paramRequired": true},
{"paramName": "do", "paramLongName":"dbOrganizationPath", "paramDescription": "the MAG Publication Path", "paramRequired": true},
{"paramName": "w", "paramLongName":"targetPath", "paramDescription": "the Working Path", "paramRequired": true}
{"paramName": "w", "paramLongName":"targetPath", "paramDescription": "the Working Path", "paramRequired": true},
{"paramName": "sp", "paramLongName":"sFilePath", "paramDescription": "the Sequence file Path", "paramRequired": true}
]

@ -3,5 +3,5 @@
{"paramName": "hb", "paramLongName":"hostedByMapPath", "paramDescription": "the hosted By Map Path", "paramRequired": true},
{"paramName": "ap", "paramLongName":"affiliationPath", "paramDescription": "the Affliation Path", "paramRequired": true},
{"paramName": "pa", "paramLongName":"paperAffiliationPath", "paramDescription": "the paperAffiliation Path", "paramRequired": true},
{"paramName": "w", "paramLongName":"workingDirPath", "paramDescription": "the Working Path", "paramRequired": true}
{"paramName": "w", "paramLongName":"workingPath", "paramDescription": "the Working Path", "paramRequired": true}
]

@ -1,5 +1,7 @@
[
{"paramName":"t", "paramLongName":"targetPath", "paramDescription": "the path of the sequencial file to write", "paramRequired": true},
{"paramName":"n", "paramLongName":"namenode", "paramDescription": "the hive metastore uris", "paramRequired": true},
{"paramName":"ts", "paramLongName":"timestamp", "paramDescription": "timestamp", "paramRequired": false}
{"paramName":"t", "paramLongName":"targetPath", "paramDescription": "the path of the sequencial file to write", "paramRequired": true},
{"paramName":"n", "paramLongName":"namenode", "paramDescription": "the hive metastore uris", "paramRequired": true},
{"paramName":"ts", "paramLongName":"timestamp", "paramDescription": "timestamp", "paramRequired": false},
{"paramName":"ess", "paramLongName":"esServer", "paramDescription": "elasticsearch server url", "paramRequired": true},
{"paramName":"esi", "paramLongName":"esIndex", "paramDescription": "elasticsearch index name", "paramRequired": true}
]

@ -0,0 +1,42 @@
<configuration>
<property>
<name>jobTracker</name>
<value>yarnRM</value>
</property>
<property>
<name>nameNode</name>
<value>hdfs://nameservice1</value>
</property>
<property>
<name>oozie.use.system.libpath</name>
<value>true</value>
</property>
<property>
<name>oozie.action.sharelib.for.spark</name>
<value>spark2</value>
</property>
<property>
<name>oozie.launcher.mapreduce.user.classpath.first</name>
<value>true</value>
</property>
<property>
<name>hive_metastore_uris</name>
<value>thrift://iis-cdh5-test-m3.ocean.icm.edu.pl:9083</value>
</property>
<property>
<name>spark2YarnHistoryServerAddress</name>
<value>http://iis-cdh5-test-gw.ocean.icm.edu.pl:18089</value>
</property>
<property>
<name>spark2EventLogDir</name>
<value>/user/spark/spark2ApplicationHistory</value>
</property>
<property>
<name>spark2ExtraListeners</name>
<value>"com.cloudera.spark.lineage.NavigatorAppListener"</value>
</property>
<property>
<name>spark2SqlQueryExecutionListeners</name>
<value>"com.cloudera.spark.lineage.NavigatorQueryListener"</value>
</property>
</configuration>

@ -0,0 +1,335 @@
<workflow-app name="Generate DOIBoost ActionSet" xmlns="uri:oozie:workflow:0.5">
<parameters>
<property>
<name>sparkDriverMemory</name>
<description>memory for driver process</description>
</property>
<property>
<name>sparkExecutorMemory</name>
<description>memory for individual executor</description>
</property>
<property>
<name>sparkExecutorIntersectionMemory</name>
<description>memory for individual executor</description>
</property>
<property>
<name>sparkExecutorCores</name>
<description>number of cores used by single executor</description>
</property>
<!-- Itersection Parameters -->
<property>
<name>workingPath</name>
<description>the working Path</description>
</property>
<property>
<name>hostedByMapPath</name>
<description>the hostedByMap Path</description>
</property>
<property>
<name>outputPath</name>
<description>the Path of the sequence file action set</description>
</property>
<!-- Crossref Parameters -->
<property>
<name>inputPathCrossref</name>
<description>the Crossref input path</description>
</property>
<property>
<name>crossrefTimestamp</name>
<description>Timestamp for the Crossref incremental Harvesting</description>
</property>
<property>
<name>esServer</name>
<description>elasticsearch server url for the Crossref Harvesting</description>
</property>
<property>
<name>esIndex</name>
<description>elasticsearch index name for the Crossref Harvesting</description>
</property>
<!-- MAG Parameters -->
<property>
<name>inputPathMAG</name>
<description>the MAG working path</description>
</property>
<!-- UnpayWall Parameters -->
<property>
<name>inputPathUnpayWall</name>
<description>the UnpayWall working path</description>
</property>
<!-- ORCID Parameters -->
<property>
<name>inputPathOrcid</name>
<description>the ORCID working path</description>
</property>
</parameters>
<global>
<job-tracker>${jobTracker}</job-tracker>
<name-node>${nameNode}</name-node>
<configuration>
<property>
<name>oozie.action.sharelib.for.spark</name>
<value>${oozieActionShareLibForSpark2}</value>
</property>
</configuration>
</global>
<start to="resume_from"/>
<decision name="resume_from">
<switch>
<case to="ConvertCrossrefToOAF">${wf:conf('resumeFrom') eq 'ConvertCrossrefToOAF'}</case>
<case to="ResetMagWorkingPath">${wf:conf('resumeFrom') eq 'ResetMagWorkingPath'}</case>
<case to="ProcessMAG">${wf:conf('resumeFrom') eq 'PreprocessMag'}</case>
<case to="ProcessUW">${wf:conf('resumeFrom') eq 'PreprocessUW'}</case>
<case to="ProcessORCID">${wf:conf('resumeFrom') eq 'PreprocessORCID'}</case>
<case to="CreateDOIBoost">${wf:conf('resumeFrom') eq 'CreateDOIBoost'}</case>
<case to="GenerateActionSet">${wf:conf('resumeFrom') eq 'GenerateActionSet'}</case>
<default to="ImportCrossRef"/>
</switch>
</decision>
<kill name="Kill">
<message>Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}]</message>
</kill>
<action name="ImportCrossRef">
<java>
<main-class>eu.dnetlib.doiboost.crossref.CrossrefImporter</main-class>
<arg>--targetPath</arg><arg>${inputPathCrossref}/index_update</arg>
<arg>--namenode</arg><arg>${nameNode}</arg>
<arg>--esServer</arg><arg>${esServer}</arg>
<arg>--esIndex</arg><arg>${esIndex}</arg>
<arg>--timestamp</arg><arg>${crossrefTimestamp}</arg>
</java>
<ok to="GenerateCrossrefDataset"/>
<error to="Kill"/>
</action>
<!-- CROSSREF SECTION -->
<action name="GenerateCrossrefDataset">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn-cluster</master>
<mode>cluster</mode>
<name>GenerateCrossrefDataset</name>
<class>eu.dnetlib.doiboost.crossref.CrossrefDataset</class>
<jar>dhp-doiboost-${projectVersion}.jar</jar>
<spark-opts>
--executor-memory=${sparkExecutorMemory}
--executor-cores=${sparkExecutorCores}
--driver-memory=${sparkDriverMemory}
--conf spark.sql.shuffle.partitions=3840
${sparkExtraOPT}
</spark-opts>
<arg>--workingPath</arg><arg>${inputPathCrossref}</arg>
<arg>--master</arg><arg>yarn-cluster</arg>
</spark>
<ok to="RenameDataset"/>
<error to="Kill"/>
</action>
<action name="RenameDataset">
<fs>
<delete path="${inputPathCrossref}/crossref_ds"/>
<move source="${inputPathCrossref}/crossref_ds_updated"
target="${inputPathCrossref}/crossref_ds"/>
</fs>
<ok to="ConvertCrossrefToOAF"/>
<error to="Kill"/>
</action>
<action name="ConvertCrossrefToOAF">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn-cluster</master>
<mode>cluster</mode>
<name>ConvertCrossrefToOAF</name>
<class>eu.dnetlib.doiboost.crossref.SparkMapDumpIntoOAF</class>
<jar>dhp-doiboost-${projectVersion}.jar</jar>
<spark-opts>
--executor-memory=${sparkExecutorMemory}
--executor-cores=${sparkExecutorCores}
--driver-memory=${sparkDriverMemory}
--conf spark.sql.shuffle.partitions=3840
${sparkExtraOPT}
</spark-opts>
<arg>--sourcePath</arg><arg>${inputPathCrossref}/crossref_ds</arg>
<arg>--targetPath</arg><arg>${workingPath}</arg>
<arg>--master</arg><arg>yarn-cluster</arg>
</spark>
<ok to="ResetMagWorkingPath"/>
<error to="Kill"/>
</action>
<!-- MAG SECTION -->
<action name="ResetMagWorkingPath">
<fs>
<delete path="${inputPathMAG}/dataset"/>
<delete path="${inputPathMAG}/process"/>
<delete path="${inputPathMAG}/dataset"/>
</fs>
<ok to="ConvertMagToDataset"/>
<error to="Kill"/>
</action>
<action name="ConvertMagToDataset">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn-cluster</master>
<mode>cluster</mode>
<name>Convert Mag to Dataset</name>
<class>eu.dnetlib.doiboost.mag.SparkImportMagIntoDataset</class>
<jar>dhp-doiboost-${projectVersion}.jar</jar>
<spark-opts>
--executor-memory=${sparkExecutorMemory}
--executor-cores=${sparkExecutorCores}
--driver-memory=${sparkDriverMemory}
${sparkExtraOPT}
</spark-opts>
<arg>--sourcePath</arg><arg>${inputPathMAG}/input</arg>
<arg>--targetPath</arg><arg>${inputPathMAG}/dataset</arg>
<arg>--master</arg><arg>yarn-cluster</arg>
</spark>
<ok to="ProcessMAG"/>
<error to="Kill"/>
</action>
<action name="ProcessMAG">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn-cluster</master>
<mode>cluster</mode>
<name>Convert Mag to OAF Dataset</name>
<class>eu.dnetlib.doiboost.mag.SparkProcessMAG</class>
<jar>dhp-doiboost-${projectVersion}.jar</jar>
<spark-opts>
--executor-memory=${sparkExecutorMemory}
--executor-cores=${sparkExecutorCores}
--driver-memory=${sparkDriverMemory}
--conf spark.sql.shuffle.partitions=3840
${sparkExtraOPT}
</spark-opts>
<arg>--sourcePath</arg><arg>${inputPathMAG}/dataset</arg>
<arg>--workingPath</arg><arg>${inputPathMAG}/process</arg>
<arg>--targetPath</arg><arg>${workingPath}</arg>
<arg>--master</arg><arg>yarn-cluster</arg>
</spark>
<ok to="ProcessUW"/>
<error to="Kill"/>
</action>
<!-- UnpayWall SECTION -->
<action name="ProcessUW">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn-cluster</master>
<mode>cluster</mode>
<name>Convert UnpayWall to Dataset</name>
<class>eu.dnetlib.doiboost.uw.SparkMapUnpayWallToOAF</class>
<jar>dhp-doiboost-${projectVersion}.jar</jar>
<spark-opts>
--executor-memory=${sparkExecutorMemory}
--executor-cores=${sparkExecutorCores}
--driver-memory=${sparkDriverMemory}
--conf spark.sql.shuffle.partitions=3840
${sparkExtraOPT}
</spark-opts>
<arg>--sourcePath</arg><arg>${inputPathUnpayWall}/uw_extracted</arg>
<arg>--targetPath</arg><arg>${workingPath}</arg>
<arg>--master</arg><arg>yarn-cluster</arg>
</spark>
<ok to="ProcessORCID"/>
<error to="Kill"/>
</action>
<!-- ORCID SECTION -->
<action name="ProcessORCID">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn-cluster</master>
<mode>cluster</mode>
<name>Convert ORCID to Dataset</name>
<class>eu.dnetlib.doiboost.orcid.SparkConvertORCIDToOAF</class>
<jar>dhp-doiboost-${projectVersion}.jar</jar>
<spark-opts>
--executor-memory=${sparkExecutorMemory}
--executor-cores=${sparkExecutorCores}
--driver-memory=${sparkDriverMemory}
--conf spark.sql.shuffle.partitions=3840
${sparkExtraOPT}
</spark-opts>
<arg>--sourcePath</arg><arg>${inputPathOrcid}</arg>
<arg>--targetPath</arg><arg>${workingPath}</arg>
<arg>--master</arg><arg>yarn-cluster</arg>
</spark>
<ok to="CreateDOIBoost"/>
<error to="Kill"/>
</action>
<!-- INTERSECTION SECTION-->
<action name="CreateDOIBoost">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn-cluster</master>
<mode>cluster</mode>
<name>Create DOIBoost Infospace</name>
<class>eu.dnetlib.doiboost.SparkGenerateDoiBoost</class>
<jar>dhp-doiboost-${projectVersion}.jar</jar>
<spark-opts>
--executor-memory=${sparkExecutorIntersectionMemory}
--executor-cores=${sparkExecutorCores}
--driver-memory=${sparkDriverMemory}
--conf spark.sql.shuffle.partitions=3840
${sparkExtraOPT}
</spark-opts>
<arg>--hostedByMapPath</arg><arg>${hostedByMapPath}</arg>
<arg>--affiliationPath</arg><arg>${inputPathMAG}/process/Affiliations</arg>
<arg>--paperAffiliationPath</arg><arg>${inputPathMAG}/process/PaperAuthorAffiliations</arg>
<arg>--workingPath</arg><arg>${workingPath}</arg>
<arg>--master</arg><arg>yarn-cluster</arg>
</spark>
<ok to="GenerateActionSet"/>
<error to="Kill"/>
</action>
<action name="GenerateActionSet">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn-cluster</master>
<mode>cluster</mode>
<name>Generate DOIBoost ActionSet</name>
<class>eu.dnetlib.doiboost.SparkGenerateDOIBoostActionSet</class>
<jar>dhp-doiboost-${projectVersion}.jar</jar>
<spark-opts>
--executor-memory=${sparkExecutorMemory}
--executor-cores=${sparkExecutorCores}
--driver-memory=${sparkDriverMemory}
--conf spark.sql.shuffle.partitions=3840
${sparkExtraOPT}
</spark-opts>
<arg>--dbPublicationPath</arg><arg>${workingPath}/doiBoostPublicationFiltered</arg>
<arg>--dbDatasetPath</arg><arg>${workingPath}/crossrefDataset</arg>
<arg>--crossRefRelation</arg><arg>${workingPath}/crossrefRelation</arg>
<arg>--dbaffiliationRelationPath</arg><arg>${workingPath}/doiBoostPublicationAffiliation</arg>
<arg>--dbOrganizationPath</arg><arg>${workingPath}/doiBoostOrganization</arg>
<arg>--targetPath</arg><arg>${workingPath}/actionDataSet</arg>
<arg>--sFilePath</arg><arg>${outputPath}</arg>
<arg>--master</arg><arg>yarn-cluster</arg>
</spark>
<ok to="End"/>
<error to="Kill"/>
</action>
<end name="End"/>
</workflow-app>

@ -1,9 +1,9 @@
package eu.dnetlib.doiboost.orcid
import com.fasterxml.jackson.databind.ObjectMapper
import eu.dnetlib.dhp.schema.oaf.Publication
import eu.dnetlib.doiboost.orcid.SparkConvertORCIDToOAF.getClass
import org.apache.spark.sql.{Encoder, Encoders, SparkSession}
import org.codehaus.jackson.map.ObjectMapper
import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.api.Test
import org.slf4j.{Logger, LoggerFactory}

@ -104,7 +104,7 @@ public class PrepareResultOrcidAssociationStep1 {
+ " LATERAL VIEW EXPLODE (author) a AS MyT "
+ " LATERAL VIEW EXPLODE (MyT.pid) p AS MyP "
+ " WHERE lower(MyP.qualifier.classid) = '" + ModelConstants.ORCID + "' or "
+ " lower(MyP.qalifier.classid) = '" + ModelConstants.ORCID_PENDING + "') tmp "
+ " lower(MyP.qualifier.classid) = '" + ModelConstants.ORCID_PENDING + "') tmp "
+ " GROUP BY id) r_t "
+ " JOIN ("
+ " SELECT source, target "

@ -16,7 +16,10 @@ import eu.dnetlib.dhp.schema.oaf.*;
public class CleaningFunctions {
public static final String DOI_URL_PREFIX_REGEX = "(^http(s?):\\/\\/)(((dx\\.)?doi\\.org)|(handle\\.test\\.datacite\\.org))\\/";
public static final String ORCID_PREFIX_REGEX = "^http(s?):\\/\\/orcid\\.org\\/";
public static final String ORCID_CLEANING_REGEX = ".*([0-9]{4}).*[-–—−=].*([0-9]{4}).*[-–—−=].*([0-9]{4}).*[-–—−=].*([0-9x]{4})";
public static final int ORCID_LEN = 19;
public static final String CLEANING_REGEX = "(?:\\n|\\r|\\t)";
public static final Set<String> PID_BLACKLIST = new HashSet<>();
@ -86,7 +89,7 @@ public class CleaningFunctions {
} else if (value instanceof Organization) {
Organization o = (Organization) value;
if (Objects.isNull(o.getCountry()) || StringUtils.isBlank(o.getCountry().getClassid())) {
o.setCountry(qualifier("UNKNOWN", "Unknown", ModelConstants.DNET_COUNTRY_TYPE));
o.setCountry(ModelConstants.UNKNOWN_COUNTRY);
}
} else if (value instanceof Relation) {
// nothing to clean here
@ -153,12 +156,14 @@ public class CleaningFunctions {
if (Objects.isNull(r.getResourcetype()) || StringUtils.isBlank(r.getResourcetype().getClassid())) {
r
.setResourcetype(
qualifier("UNKNOWN", "Unknown", ModelConstants.DNET_DATA_CITE_RESOURCE));
qualifier(ModelConstants.UNKNOWN, "Unknown", ModelConstants.DNET_DATA_CITE_RESOURCE));
}
if (Objects.nonNull(r.getInstance())) {
for (Instance i : r.getInstance()) {
if (Objects.isNull(i.getAccessright()) || StringUtils.isBlank(i.getAccessright().getClassid())) {
i.setAccessright(qualifier("UNKNOWN", "not available", ModelConstants.DNET_ACCESS_MODES));
i
.setAccessright(
qualifier(ModelConstants.UNKNOWN, "not available", ModelConstants.DNET_ACCESS_MODES));
}
if (Objects.isNull(i.getHostedby()) || StringUtils.isBlank(i.getHostedby().getKey())) {
i.setHostedby(ModelConstants.UNKNOWN_REPOSITORY);
@ -173,7 +178,7 @@ public class CleaningFunctions {
if (Objects.isNull(bestaccessrights)) {
r
.setBestaccessright(
qualifier("UNKNOWN", "not available", ModelConstants.DNET_ACCESS_MODES));
qualifier(ModelConstants.UNKNOWN, "not available", ModelConstants.DNET_ACCESS_MODES));
} else {
r.setBestaccessright(bestaccessrights);
}
@ -211,14 +216,31 @@ public class CleaningFunctions {
.map(Qualifier::getClassid)
.orElse(""))
.orElse("");
if (pidProvenance.equals(ModelConstants.SYSIMPORT_CROSSWALK_ENTITYREGISTRY)) {
p.getQualifier().setClassid(ModelConstants.ORCID);
} else {
p.getQualifier().setClassid(ModelConstants.ORCID_PENDING);
if (p
.getQualifier()
.getClassid()
.toLowerCase()
.contains(ModelConstants.ORCID)) {
if (pidProvenance
.equals(ModelConstants.SYSIMPORT_CROSSWALK_ENTITYREGISTRY)) {
p.getQualifier().setClassid(ModelConstants.ORCID);
} else {
p.getQualifier().setClassid(ModelConstants.ORCID_PENDING);
}
final String orcid = p
.getValue()
.trim()
.toLowerCase()
.replaceAll(ORCID_CLEANING_REGEX, "$1-$2-$3-$4");
if (orcid.length() == ORCID_LEN) {
p.setValue(orcid);
} else {
p.setValue("");
}
}
p.setValue(p.getValue().trim().replaceAll(ORCID_PREFIX_REGEX, ""));
return p;
})
.filter(p -> StringUtils.isNotBlank(p.getValue()))
.collect(
Collectors
.toMap(

@ -21,6 +21,7 @@ import org.apache.spark.sql.expressions.Aggregator;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import com.fasterxml.jackson.databind.DeserializationFeature;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.jayway.jsonpath.Configuration;
import com.jayway.jsonpath.DocumentContext;
@ -44,7 +45,8 @@ public class GroupEntitiesAndRelationsSparkJob {
private final static String SOURCE_JPATH = "$.source";
private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper()
.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
public static void main(String[] args) throws Exception {

@ -23,7 +23,15 @@ import static eu.dnetlib.dhp.schema.common.ModelConstants.RESULT_PROJECT;
import static eu.dnetlib.dhp.schema.common.ModelConstants.RESULT_RESULT;
import static eu.dnetlib.dhp.schema.common.ModelConstants.SOFTWARE_DEFAULT_RESULTTYPE;
import static eu.dnetlib.dhp.schema.common.ModelConstants.USER_CLAIM;
import static eu.dnetlib.dhp.schema.oaf.OafMapperUtils.*;
import static eu.dnetlib.dhp.schema.oaf.OafMapperUtils.asString;
import static eu.dnetlib.dhp.schema.oaf.OafMapperUtils.createOpenaireId;
import static eu.dnetlib.dhp.schema.oaf.OafMapperUtils.dataInfo;
import static eu.dnetlib.dhp.schema.oaf.OafMapperUtils.field;
import static eu.dnetlib.dhp.schema.oaf.OafMapperUtils.journal;
import static eu.dnetlib.dhp.schema.oaf.OafMapperUtils.listFields;
import static eu.dnetlib.dhp.schema.oaf.OafMapperUtils.listKeyValues;
import static eu.dnetlib.dhp.schema.oaf.OafMapperUtils.qualifier;
import static eu.dnetlib.dhp.schema.oaf.OafMapperUtils.structuredProperty;
import java.io.Closeable;
import java.io.IOException;
@ -462,44 +470,48 @@ public class MigrateDbEntitiesApplication extends AbstractMigrationApplication i
return Arrays.asList(r);
} else {
final String validationDate = rs.getString("curation_date");
final String sourceId = createOpenaireId(rs.getString(SOURCE_TYPE), rs.getString("source_id"), false);
final String targetId = createOpenaireId(rs.getString(TARGET_TYPE), rs.getString("target_id"), false);
final Relation r1 = new Relation();
final Relation r2 = new Relation();
r1.setValidated(true);
r1.setValidationDate(validationDate);
r1.setCollectedfrom(collectedFrom);
r1.setSource(sourceId);
r1.setTarget(targetId);
r1.setDataInfo(info);
r1.setLastupdatetimestamp(lastUpdateTimestamp);
r2.setValidationDate(validationDate);
r2.setValidated(true);
r2.setCollectedfrom(collectedFrom);
r2.setSource(targetId);
r2.setTarget(sourceId);
r2.setDataInfo(info);
r2.setLastupdatetimestamp(lastUpdateTimestamp);
if (rs.getString(SOURCE_TYPE).equals("project")) {
r1.setCollectedfrom(collectedFrom);
r1.setRelType(RESULT_PROJECT);
r1.setSubRelType(OUTCOME);
r1.setRelClass(PRODUCES);
r2.setCollectedfrom(collectedFrom);
r2.setRelType(RESULT_PROJECT);
r2.setSubRelType(OUTCOME);
r2.setRelClass(IS_PRODUCED_BY);
} else {
r1.setCollectedfrom(collectedFrom);
r1.setRelType(RESULT_RESULT);
r1.setSubRelType(RELATIONSHIP);
r1.setRelClass(IS_RELATED_TO);
r2.setCollectedfrom(collectedFrom);
r2.setRelType(RESULT_RESULT);
r2.setSubRelType(RELATIONSHIP);
r2.setRelClass(IS_RELATED_TO);
}
r1.setSource(sourceId);
r1.setTarget(targetId);
r1.setDataInfo(info);
r1.setLastupdatetimestamp(lastUpdateTimestamp);
r2.setSource(targetId);
r2.setTarget(sourceId);
r2.setDataInfo(info);
r2.setLastupdatetimestamp(lastUpdateTimestamp);
return Arrays.asList(r1, r2);
}

@ -11,7 +11,8 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.SequenceFile;
import org.apache.hadoop.io.Text;
import org.codehaus.jackson.map.ObjectMapper;
import com.fasterxml.jackson.databind.ObjectMapper;
import eu.dnetlib.dhp.schema.oaf.Oaf;

@ -1 +1 @@
SELECT source_type, source_id, target_type, target_id, semantics FROM claim WHERE approved=TRUE;
SELECT source_type, source_id, target_type, target_id, semantics, curation_date::text FROM claim WHERE approved=TRUE;

@ -35,7 +35,7 @@ public class MigrateDbEntitiesApplicationTest {
private MigrateDbEntitiesApplication app;
@Mock
@Mock(lenient = true)
private ResultSet rs;
@Mock

@ -6,29 +6,29 @@
xmlns:dc="http://purl.org/dc/elements/1.1/"
xmlns:dri="http://www.driver-repository.eu/namespace/dri"
xmlns:prov="http://www.openarchives.org/OAI/2.0/provenance" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
<dri:objIdentifier>r3f52792889d::000051aa1f61d77d2c0b340091f8024e</dri:objIdentifier>
<dri:recordIdentifier>textgrid:q9cv.0</dri:recordIdentifier>
<dri:dateOfCollection>2020-11-17T09:34:11.128+01:00</dri:dateOfCollection>
<dri:objIdentifier>r3f52792889d::00002412cb25f2f3047712d00ab2c8eb</dri:objIdentifier>
<dri:recordIdentifier>hdl:11858/00-1734-0000-0003-EE73-2</dri:recordIdentifier>
<dri:dateOfCollection>2020-12-16T10:04:03.148Z</dri:dateOfCollection>
<oaf:datasourceprefix>r3f52792889d</oaf:datasourceprefix>
<identifier xmlns="http://www.openarchives.org/OAI/2.0/">textgrid:q9cv.0</identifier>
<datestamp xmlns="http://www.openarchives.org/OAI/2.0/">2012-01-21T13:35:20Z</datestamp>
<dr:dateOfTransformation>2020-11-17T19:08:56.703+01:00</dr:dateOfTransformation>
<identifier xmlns="http://www.openarchives.org/OAI/2.0/">textgrid:rn8z.0</identifier>
<datestamp xmlns="http://www.openarchives.org/OAI/2.0/">2012-01-29T20:54:12Z</datestamp>
<dr:dateOfTransformation>2020-12-16T16:02:37.562Z</dr:dateOfTransformation>
</oai:header>
<metadata>
<datacite:resource xmlns="http://www.openarchives.org/OAI/2.0/"
xmlns:dc="http://purl.org/dc/elements/1.1/"
xmlns:dri="http://www.driver-repository.eu/namespace/dri"
xmlns:prov="http://www.openarchives.org/OAI/2.0/provenance" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
<datacite:identifier identifierType="Handle">hdl:11858/00-1734-0000-0003-7664-F</datacite:identifier>
<datacite:identifier identifierType="Handle">hdl:11858/00-1734-0000-0003-EE73-2</datacite:identifier>
<datacite:creators>
<datacite:creator>
<datacite:creatorName>Hoffmann von Fallersleben, August Heinrich</datacite:creatorName>
<datacite:nameIdentifier nameIdentifierScheme="pnd" schemeURI="https://de.dariah.eu/pnd-service">118552589</datacite:nameIdentifier>
<datacite:creatorName>Liliencron, Detlev von</datacite:creatorName>
<datacite:nameIdentifier nameIdentifierScheme="pnd" schemeURI="https://ref.de.dariah.eu/pndsearch/pndquery.xql?id=">118572954</datacite:nameIdentifier>
</datacite:creator>
</datacite:creators>
<datacite:titles>
<datacite:title titleType="Other">Mailied</datacite:title>
<datacite:title titleType="Other">August Heinrich Hoffmann von Fallersleben: Unpolitische Lieder von Hoffmann von Fallersleben, 1. + 2. Theil, 1. Theil, Hamburg: Hoffmann und Campe, 1841.</datacite:title>
<datacite:title titleType="Other">Auf dem Trocknen</datacite:title>
<datacite:title titleType="Other">Detlev von Liliencron: Gute Nacht. Hinterlassene Gedichte, Berlin: Schuster &amp; Loeffler, 1909.</datacite:title>
</datacite:titles>
<datacite:publisher>TextGrid</datacite:publisher>
<datacite:publicationYear>2012</datacite:publicationYear>
@ -38,21 +38,21 @@
</datacite:contributor>
<datacite:contributor contributorType="Other">
<datacite:contributorName>Digitale Bibliothek</datacite:contributorName>
<datacite:nameIdentifier nameIdentifierScheme="textgrid">TGPR-372fe6dc-57f2-6cd4-01b5-2c4bbefcfd3c</datacite:nameIdentifier>
<datacite:nameIdentifier nameIdentifierScheme="textgrid" schemeURI="http://www.textgridlab.org/schema/textgrid-metadata_2010.xsd">TGPR-372fe6dc-57f2-6cd4-01b5-2c4bbefcfd3c</datacite:nameIdentifier>
</datacite:contributor>
</datacite:contributors>
<datacite:dates>
<datacite:date dateType="Created">2012-01-21T13:35:20Z</datacite:date>
<datacite:date dateType="Issued">2012-01-21T13:35:20Z</datacite:date>
<datacite:date dateType="Updated">2012-01-21T13:35:20Z</datacite:date>
<datacite:date dateType="Created">2012-01-29T20:54:12Z</datacite:date>
<datacite:date dateType="Issued">2012-01-29T20:54:12Z</datacite:date>
<datacite:date dateType="Updated">2012-01-29T20:54:12Z</datacite:date>
</datacite:dates>
<datacite:resourceType resourceTypeGeneral="Dataset"/>
<alternateIdentifiers xmlns="http://datacite.org/schema/kernel-3">
<datacite:alternateIdentifier alternateIdentifierType="URI" xmlns="http://www.openarchives.org/OAI/2.0/">textgrid:q9cv.0</datacite:alternateIdentifier>
<alternateIdentifier alternateIdentifierType="URL">http://hdl.handle.net/hdl:11858/00-1734-0000-0003-7664-F</alternateIdentifier>
<datacite:alternateIdentifier alternateIdentifierType="URI" xmlns="http://www.openarchives.org/OAI/2.0/">textgrid:rn8z.0</datacite:alternateIdentifier>
<alternateIdentifier alternateIdentifierType="URL">http://hdl.handle.net/hdl:11858/00-1734-0000-0003-EE73-2</alternateIdentifier>
</alternateIdentifiers>
<datacite:relatedIdentifiers>
<datacite:relatedIdentifier relatedIdentifierType="Handle" relationType="IsPartOf">hdl:11858/00-1734-0000-0003-7666-B</datacite:relatedIdentifier>
<datacite:relatedIdentifier relatedIdentifierType="Handle" relationType="IsPartOf">hdl:11858/00-1734-0000-0003-EE72-4</datacite:relatedIdentifier>
</datacite:relatedIdentifiers>
<datacite:sizes>
<datacite:size>527 Bytes</datacite:size>
@ -77,17 +77,18 @@
<datacite:geoLocations>
<datacite:geoLocation>
<datacite:geoLocationPlace
xmlns:xs="http://www.w3.org/2001/XMLSchema" xsi:type="xs:string">Hamburg</datacite:geoLocationPlace>
xmlns:xs="http://www.w3.org/2001/XMLSchema" xsi:type="xs:string">Berlin</datacite:geoLocationPlace>
</datacite:geoLocation>
</datacite:geoLocations>
</datacite:resource>
<oaf:identifier identifierType="handle">hdl:11858/00-1734-0000-0003-7664-F</oaf:identifier>
<oaf:identifier identifierType="handle">hdl:11858/00-1734-0000-0003-EE73-2</oaf:identifier>
<dr:CobjCategory type="dataset">0021</dr:CobjCategory>
<oaf:refereed>0002</oaf:refereed>
<oaf:dateAccepted>2012-01-21</oaf:dateAccepted>
<oaf:dateAccepted>2012-01-29</oaf:dateAccepted>
<oaf:accessrights>OPEN</oaf:accessrights>
<oaf:license>http://creativecommons.org/licenses/by/3.0/de/legalcode</oaf:license>
<oaf:language>und</oaf:language>
<oaf:country>DE</oaf:country>
<oaf:hostedBy id="re3data_____::r3d100011365" name="TextGrid Repository"/>
<oaf:collectedFrom id="re3data_____::r3d100011365" name="TextGrid Repository"/>
</metadata>
@ -95,11 +96,11 @@
xmlns:dri="http://www.driver-repository.eu/namespace/dri"
xmlns:prov="http://www.openarchives.org/OAI/2.0/provenance" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
<provenance xmlns="http://www.openarchives.org/OAI/2.0/provenance" xsi:schemaLocation="http://www.openarchives.org/OAI/2.0/provenance http://www.openarchives.org/OAI/2.0/provenance.xsd">
<originDescription altered="true" harvestDate="2020-11-17T09:34:11.128+01:00">
<originDescription altered="true" harvestDate="2020-12-16T10:04:03.148Z">
<baseURL>https%3A%2F%2Fdev.textgridlab.org%2F1.0%2Ftgoaipmh%2Foai</baseURL>
<identifier>textgrid:q9cv.0</identifier>
<datestamp>2012-01-21T13:35:20Z</datestamp>
<metadataNamespace>http://schema.datacite.org/oai/oai-1.0/</metadataNamespace>
<identifier>textgrid:rn8z.0</identifier>
<datestamp>2012-01-29T20:54:12Z</datestamp>
<metadataNamespace/>
</originDescription>
</provenance>
<oaf:datainfo>
@ -107,9 +108,10 @@
<oaf:deletedbyinference>false</oaf:deletedbyinference>
<oaf:trust>0.9</oaf:trust>
<oaf:inferenceprovenance/>
<oaf:provenanceaction classid="sysimport:crosswalk"
classname="sysimport:crosswalk"
<oaf:provenanceaction classid="sysimport:crosswalk:datasetarchive"
classname="sysimport:crosswalk:datasetarchive"
schemeid="dnet:provenanceActions" schemename="dnet:provenanceActions"/>
</oaf:datainfo>
</about>
</record>
</record>

@ -1,15 +1,15 @@
package eu.dnetlib.dhp.export
import com.fasterxml.jackson.databind.ObjectMapper
import java.time.LocalDateTime
import java.time.format.DateTimeFormatter
import eu.dnetlib.dhp.common.PacePerson
import eu.dnetlib.dhp.schema.action.AtomicAction
import eu.dnetlib.dhp.schema.oaf.{Author, Dataset, ExternalReference, Field, Instance, KeyValue, Oaf, Publication, Qualifier, Relation, Result, StructuredProperty}
import eu.dnetlib.dhp.schema.oaf.{Author, Dataset, ExternalReference, Field, Instance, KeyValue, Oaf, Publication, Qualifier, Relation, Result, StructuredProperty}
import eu.dnetlib.dhp.schema.scholexplorer.{DLIDataset, DLIPublication}
import eu.dnetlib.dhp.utils.DHPUtils
import org.apache.commons.lang3.StringUtils
import org.codehaus.jackson.map.ObjectMapper
import eu.dnetlib.dhp.schema.scholexplorer.OafUtils._
import scala.collection.JavaConverters._

@ -1,27 +1,21 @@
package eu.dnetlib.dhp.`export`
import eu.dnetlib.dhp.application.ArgumentApplicationParser
import eu.dnetlib.dhp.schema.oaf.{Instance, Publication, Relation, Dataset => OafDataset}
import eu.dnetlib.dhp.schema.oaf.{Publication, Relation, Dataset => OafDataset}
import eu.dnetlib.dhp.schema.scholexplorer.{DLIDataset, DLIPublication}
import org.apache.commons.io.IOUtils
import org.apache.hadoop.io.Text
import org.apache.hadoop.io.compress.GzipCodec
import org.apache.hadoop.mapred.SequenceFileOutputFormat
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.{Dataset, Encoder, Encoders, SaveMode, SparkSession}
import org.apache.spark.sql.functions._
import org.apache.spark.sql.expressions.Window
import org.apache.spark.{SparkConf, SparkContext}
import org.codehaus.jackson.map.ObjectMapper
import org.apache.spark.SparkConf
import scala.collection.mutable.ArrayBuffer
import scala.collection.JavaConverters._
object SparkExportContentForOpenAire {
def main(args: Array[String]): Unit = {
val conf: SparkConf = new SparkConf()
val parser = new ArgumentApplicationParser(IOUtils.toString(SparkExportContentForOpenAire.getClass.getResourceAsStream("input_export_content_parameters.json")))
@ -178,11 +172,4 @@ object SparkExportContentForOpenAire {
fRels.union(fpubs).union(fdats).rdd.map(s => (new Text(s._1), new Text(s._2))).saveAsHadoopFile(s"$workingPath/export/rawset", classOf[Text], classOf[Text], classOf[SequenceFileOutputFormat[Text,Text]], classOf[GzipCodec])
}
}

@ -10,7 +10,8 @@ import org.apache.http.client.methods.HttpPut;
import org.apache.http.entity.StringEntity;
import org.apache.http.impl.client.CloseableHttpClient;
import org.apache.http.impl.client.HttpClients;
import org.codehaus.jackson.map.ObjectMapper;
import com.fasterxml.jackson.databind.ObjectMapper;
import eu.dnetlib.dhp.application.ArgumentApplicationParser;

@ -1,5 +1,6 @@
package eu.dnetlib.dhp.provision
import com.fasterxml.jackson.databind.ObjectMapper
import eu.dnetlib.dhp.application.ArgumentApplicationParser
import eu.dnetlib.dhp.provision.scholix.Scholix
import eu.dnetlib.dhp.provision.scholix.summary.ScholixSummary
@ -7,7 +8,6 @@ import org.apache.commons.io.IOUtils
import org.apache.hadoop.io.compress.GzipCodec
import org.apache.spark.SparkConf
import org.apache.spark.sql.{Encoder, Encoders, SparkSession}
import org.codehaus.jackson.map.ObjectMapper
object SparkConvertDatasetToJson {

@ -259,7 +259,7 @@
<action name="Step17">
<hive2 xmlns="uri:oozie:hive2-action:0.1">
<jdbc-url>${hive_jdbc_url}</jdbc-url>
<script>scripts/updateProductionViews.sql</script>
<script>scripts/step17.sql</script>
<param>stats_db_name=${stats_db_name}</param>
<param>stats_db_shadow_name=${stats_db_shadow_name}</param>
</hive2>
@ -287,8 +287,8 @@
<name-node>${nameNode}</name-node>
<exec>impala-shell.sh</exec>
<argument>${stats_db_shadow_name}</argument>
<argument>computeProductionStats.sql</argument>
<argument>${wf:appPath()}/scripts/computeProductionStats.sql</argument>
<argument>step19.sql</argument>
<argument>${wf:appPath()}/scripts/step19.sql</argument>
<file>impala-shell.sh</file>
</shell>
<ok to="Step20"/>

Loading…
Cancel
Save