wf to generate doi_author_list for doiboost; wf to download updated works

pull/98/head
Enrico Ottonello 3 years ago
parent f2df3ead74
commit 1b1e9ea67c

@ -0,0 +1,224 @@
package eu.dnetlib.doiboost.orcid;
import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession;
import java.io.IOException;
import java.text.SimpleDateFormat;
import java.util.*;
import org.apache.commons.compress.utils.Lists;
import org.apache.commons.io.IOUtils;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat;
import org.apache.http.client.methods.CloseableHttpResponse;
import org.apache.http.client.methods.HttpGet;
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;
import com.google.gson.JsonElement;
import com.google.gson.JsonParser;
import com.ximpleware.NavException;
import com.ximpleware.ParseException;
import com.ximpleware.XPathEvalException;
import com.ximpleware.XPathParseException;
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
import eu.dnetlib.doiboost.orcid.model.DownloadedRecordData;
import eu.dnetlib.doiboost.orcid.model.WorkData;
import eu.dnetlib.doiboost.orcid.xml.XMLRecordParser;
import scala.Tuple2;
public class SparkDownloadOrcidWorks {
static Logger logger = LoggerFactory.getLogger(SparkDownloadOrcidWorks.class);
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 {
final ArgumentApplicationParser parser = new ArgumentApplicationParser(
IOUtils
.toString(
SparkDownloadOrcidWorks.class
.getResourceAsStream(
"/eu/dnetlib/dhp/doiboost/download_orcid_data.json")));
parser.parseArgument(args);
Boolean isSparkSessionManaged = Optional
.ofNullable(parser.get("isSparkSessionManaged"))
.map(Boolean::valueOf)
.orElse(Boolean.TRUE);
logger.info("isSparkSessionManaged: {}", isSparkSessionManaged);
final String workingPath = parser.get("workingPath");
logger.info("workingPath: ", workingPath);
// final String outputPath = parser.get("outputPath");
final String outputPath = "downloads/updated_works";
logger.info("outputPath: ", outputPath);
final String token = parser.get("token");
// final String lambdaFileName = parser.get("lambdaFileName");
// logger.info("lambdaFileName: ", lambdaFileName);
SparkConf conf = new SparkConf();
runWithSparkSession(
conf,
isSparkSessionManaged,
spark -> {
JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext());
LongAccumulator parsedRecordsAcc = spark.sparkContext().longAccumulator("parsed_records");
LongAccumulator modifiedRecordsAcc = spark.sparkContext().longAccumulator("to_download_records");
LongAccumulator downloadedRecordsAcc = spark.sparkContext().longAccumulator("downloaded_records");
LongAccumulator errorHTTP403Acc = spark.sparkContext().longAccumulator("error_HTTP_403");
LongAccumulator errorHTTP409Acc = spark.sparkContext().longAccumulator("error_HTTP_409");
LongAccumulator errorHTTP503Acc = spark.sparkContext().longAccumulator("error_HTTP_503");
LongAccumulator errorHTTP525Acc = spark.sparkContext().longAccumulator("error_HTTP_525");
LongAccumulator errorHTTPGenericAcc = spark.sparkContext().longAccumulator("error_HTTP_Generic");
logger.info("Retrieving updated authors");
JavaPairRDD<Text, Text> updatedAuthorsRDD = sc
.sequenceFile(workingPath + "downloads/updated_authors/*", Text.class, Text.class);
logger.info("Updated authors retrieved: " + updatedAuthorsRDD.count());
Function<Tuple2<Text, Text>, Iterator<String>> retrieveWorkUrlFunction = data -> {
String orcidId = data._1().toString();
String jsonData = data._2().toString();
List<String> orcidIdWorkId = Lists.newArrayList();
Map<String, String> workIdLastModifiedDate = retrieveWorkIdLastModifiedDate(jsonData);
workIdLastModifiedDate.forEach((k, v) -> {
if (isModified(orcidId, v)) {
orcidIdWorkId.add(orcidId.concat("/work/").concat(k));
}
});
Iterator<String> iterator = orcidIdWorkId.iterator();
return iterator;
};
List<Iterator<String>> toDownloadWorksRDD = updatedAuthorsRDD
.map(retrieveWorkUrlFunction)
.take(1000);
sc.parallelize(toDownloadWorksRDD).saveAsTextFile(workingPath.concat("downloads/updated_works_test/"));
Function<Tuple2<Text, Text>, Tuple2<String, String>> downloadRecordFunction = data -> {
String orcidId = data._1().toString();
String lastModifiedDate = data._2().toString();
final DownloadedRecordData downloaded = new DownloadedRecordData();
downloaded.setOrcidId(orcidId);
downloaded.setLastModifiedDate(lastModifiedDate);
try (CloseableHttpClient client = HttpClients.createDefault()) {
HttpGet httpGet = new HttpGet("https://api.orcid.org/v3.0/" + orcidId + "/work");
httpGet.addHeader("Accept", "application/vnd.orcid+xml");
httpGet.addHeader("Authorization", String.format("Bearer %s", token));
long startReq = System.currentTimeMillis();
CloseableHttpResponse response = client.execute(httpGet);
long endReq = System.currentTimeMillis();
long reqTime = endReq - startReq;
if (reqTime < 1000) {
Thread.sleep(1000 - reqTime);
}
int statusCode = response.getStatusLine().getStatusCode();
downloaded.setStatusCode(statusCode);
if (statusCode != 200) {
switch (statusCode) {
case 403:
errorHTTP403Acc.add(1);
case 409:
errorHTTP409Acc.add(1);
case 503:
errorHTTP503Acc.add(1);
throw new RuntimeException("Orcid request rate limit reached (HTTP 503)");
case 525:
errorHTTP525Acc.add(1);
default:
errorHTTPGenericAcc.add(1);
logger
.info(
"Downloading " + orcidId + " status code: "
+ response.getStatusLine().getStatusCode());
}
return downloaded.toTuple2();
}
downloadedRecordsAcc.add(1);
downloaded
.setCompressedData(
ArgumentApplicationParser
.compressArgument(IOUtils.toString(response.getEntity().getContent())));
} catch (Throwable e) {
logger.info("Downloading " + orcidId, e.getMessage());
downloaded.setErrorMessage(e.getMessage());
return downloaded.toTuple2();
}
return downloaded.toTuple2();
};
// sc.hadoopConfiguration().set("mapreduce.output.fileoutputformat.compress", "true");
// logger.info("Start downloading ...");
// updatedAuthorsRDD
// .map(downloadRecordFunction)
// .mapToPair(t -> new Tuple2(new Text(t._1()), new Text(t._2())))
// .saveAsNewAPIHadoopFile(
// workingPath.concat(outputPath),
// Text.class,
// Text.class,
// SequenceFileOutputFormat.class,
// sc.hadoopConfiguration());
// logger.info("parsedRecordsAcc: " + parsedRecordsAcc.value().toString());
// logger.info("modifiedRecordsAcc: " + modifiedRecordsAcc.value().toString());
// logger.info("downloadedRecordsAcc: " + downloadedRecordsAcc.value().toString());
// logger.info("errorHTTP403Acc: " + errorHTTP403Acc.value().toString());
// logger.info("errorHTTP409Acc: " + errorHTTP409Acc.value().toString());
// logger.info("errorHTTP503Acc: " + errorHTTP503Acc.value().toString());
// logger.info("errorHTTP525Acc: " + errorHTTP525Acc.value().toString());
// logger.info("errorHTTPGenericAcc: " + errorHTTPGenericAcc.value().toString());
});
}
private static boolean isModified(String orcidId, String modifiedDate) {
Date modifiedDateDt = null;
Date lastUpdateDt = null;
try {
if (modifiedDate.length() != 19) {
modifiedDate = modifiedDate.substring(0, 19);
}
modifiedDateDt = new SimpleDateFormat(DATE_FORMAT).parse(modifiedDate);
lastUpdateDt = new SimpleDateFormat(DATE_FORMAT).parse(lastUpdate);
} catch (Exception e) {
logger.info("[" + orcidId + "] Parsing date: ", e.getMessage());
return true;
}
return modifiedDateDt.after(lastUpdateDt);
}
private static Map<String, String> retrieveWorkIdLastModifiedDate(String json)
throws XPathEvalException, NavException, XPathParseException, ParseException {
JsonElement jElement = new JsonParser().parse(json);
String statusCode = getJsonValue(jElement, "statusCode");
if (statusCode.equals("200")) {
String compressedData = getJsonValue(jElement, "compressedData");
String authorSummary = ArgumentApplicationParser.decompressValue(compressedData);
return XMLRecordParser.retrieveWorkIdLastModifiedDate(authorSummary.getBytes());
}
return new HashMap<>();
}
private static String getJsonValue(JsonElement jElement, String property) {
if (jElement.getAsJsonObject().has(property)) {
JsonElement name = null;
name = jElement.getAsJsonObject().get(property);
if (name != null && !name.isJsonNull()) {
return name.getAsString();
}
}
return new String("");
}
}

@ -3,11 +3,11 @@ package eu.dnetlib.doiboost.orcid;
import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession;
import java.io.BufferedReader;
import java.io.IOException;
import java.util.Arrays;
import java.util.List;
import java.util.Objects;
import java.util.Optional;
import java.io.InputStream;
import java.io.InputStreamReader;
import java.util.*;
import java.util.stream.Collectors;
import java.util.stream.Stream;
@ -19,6 +19,7 @@ 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.api.java.function.MapFunction;
import org.apache.spark.rdd.RDD;
import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Encoders;
import org.slf4j.Logger;
@ -28,10 +29,14 @@ import com.esotericsoftware.minlog.Log;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.gson.JsonElement;
import com.google.gson.JsonParser;
import com.ximpleware.ParseException;
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
import eu.dnetlib.dhp.parser.utility.VtdException;
import eu.dnetlib.dhp.schema.orcid.AuthorData;
import eu.dnetlib.doiboost.orcid.model.WorkData;
import eu.dnetlib.doiboost.orcid.xml.XMLRecordParser;
import eu.dnetlib.doiboost.orcidnodoi.json.JsonWriter;
import scala.Tuple2;
public class SparkGenerateDoiAuthorList {
@ -56,6 +61,10 @@ public class SparkGenerateDoiAuthorList {
logger.info("workingPath: ", workingPath);
final String outputDoiAuthorListPath = parser.get("outputDoiAuthorListPath");
logger.info("outputDoiAuthorListPath: ", outputDoiAuthorListPath);
final String authorsPath = parser.get("authorsPath");
logger.info("authorsPath: ", authorsPath);
final String xmlWorksPath = parser.get("xmlWorksPath");
logger.info("xmlWorksPath: ", xmlWorksPath);
SparkConf conf = new SparkConf();
runWithSparkSession(
@ -65,17 +74,21 @@ public class SparkGenerateDoiAuthorList {
JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext());
JavaPairRDD<Text, Text> summariesRDD = sc
.sequenceFile(workingPath + "../orcid_summaries/output/authors.seq", Text.class, Text.class);
.sequenceFile(workingPath.concat(authorsPath), Text.class, Text.class);
Dataset<AuthorData> summariesDataset = spark
.createDataset(
summariesRDD.map(seq -> loadAuthorFromJson(seq._1(), seq._2())).rdd(),
Encoders.bean(AuthorData.class));
JavaPairRDD<Text, Text> activitiesRDD = sc
.sequenceFile(workingPath + "/output/*.seq", Text.class, Text.class);
JavaPairRDD<Text, Text> xmlWorksRDD = sc
.sequenceFile(workingPath.concat(xmlWorksPath), Text.class, Text.class);
Dataset<WorkData> activitiesDataset = spark
.createDataset(
activitiesRDD.map(seq -> loadWorkFromJson(seq._1(), seq._2())).rdd(),
xmlWorksRDD
.map(seq -> XMLRecordParser.VTDParseWorkData(seq._2().toString().getBytes()))
.filter(work -> work != null && work.getErrorCode() == null && work.isDoiFound())
.rdd(),
Encoders.bean(WorkData.class));
Function<Tuple2<String, AuthorData>, Tuple2<String, List<AuthorData>>> toAuthorListFunction = data -> {
@ -135,12 +148,16 @@ public class SparkGenerateDoiAuthorList {
}
return null;
})
.mapToPair(s -> {
List<AuthorData> authorList = s._2();
Set<String> oidsAlreadySeen = new HashSet<>();
authorList.removeIf(a -> !oidsAlreadySeen.add(a.getOid()));
return new Tuple2<>(s._1(), authorList);
})
.mapToPair(
s -> {
ObjectMapper mapper = new ObjectMapper();
return new Tuple2<>(s._1(), mapper.writeValueAsString(s._2()));
return new Tuple2<>(s._1(), JsonWriter.create(s._2()));
})
.repartition(10)
.saveAsTextFile(workingPath + outputDoiAuthorListPath);
});

@ -1,23 +1,17 @@
package eu.dnetlib.doiboost.orcid.xml;
import java.util.Arrays;
import java.util.List;
import java.util.*;
import org.mortbay.log.Log;
import com.ximpleware.AutoPilot;
import com.ximpleware.EOFException;
import com.ximpleware.EncodingException;
import com.ximpleware.EntityException;
import com.ximpleware.ParseException;
import com.ximpleware.VTDGen;
import com.ximpleware.VTDNav;
import com.ximpleware.*;
import eu.dnetlib.dhp.parser.utility.VtdException;
import eu.dnetlib.dhp.parser.utility.VtdUtilityParser;
import eu.dnetlib.dhp.schema.orcid.AuthorData;
import eu.dnetlib.doiboost.orcid.model.WorkData;
import eu.dnetlib.doiboost.orcidnodoi.model.Contributor;
public class XMLRecordParser {
@ -32,7 +26,8 @@ public class XMLRecordParser {
private static final String NS_RECORD_URL = "http://www.orcid.org/ns/record";
private static final String NS_RECORD = "record";
private static final String NS_ERROR_URL = "http://www.orcid.org/ns/error";
private static final String NS_ACTIVITIES = "activities";
private static final String NS_ACTIVITIES_URL = "http://www.orcid.org/ns/activities";
private static final String NS_WORK = "work";
private static final String NS_WORK_URL = "http://www.orcid.org/ns/work";
@ -139,6 +134,12 @@ public class XMLRecordParser {
return retrieveOrcidId(bytes, defaultValue, NS_WORK, NS_WORK_URL, "//work:work", "put-code");
}
public static String retrieveWorkIdFromSummary(byte[] bytes, String defaultValue)
throws VtdException, ParseException {
return retrieveOrcidId(
bytes, defaultValue, NS_ACTIVITIES, NS_ACTIVITIES_URL, "//work:work-summary", "put-code");
}
private static String retrieveOrcidId(byte[] bytes, String defaultValue, String ns, String nsUrl, String xpath,
String idAttributeName)
throws VtdException, ParseException {
@ -148,6 +149,7 @@ public class XMLRecordParser {
final VTDNav vn = vg.getNav();
final AutoPilot ap = new AutoPilot(vn);
ap.declareXPathNameSpace(ns, nsUrl);
ap.declareXPathNameSpace(NS_WORK, NS_WORK_URL);
List<VtdUtilityParser.Node> recordNodes = VtdUtilityParser
.getTextValuesWithAttributes(
ap, vn, xpath, Arrays.asList(idAttributeName));
@ -157,4 +159,42 @@ public class XMLRecordParser {
Log.info("id not found - default: " + defaultValue);
return defaultValue;
}
public static Map<String, String> retrieveWorkIdLastModifiedDate(byte[] bytes)
throws ParseException, XPathParseException, NavException, XPathEvalException {
final VTDGen vg = new VTDGen();
vg.setDoc(bytes);
vg.parse(true);
final VTDNav vn = vg.getNav();
final AutoPilot ap = new AutoPilot(vn);
ap.declareXPathNameSpace(NS_COMMON, NS_COMMON_URL);
ap.declareXPathNameSpace(NS_PERSON, NS_PERSON_URL);
ap.declareXPathNameSpace(NS_DETAILS, NS_DETAILS_URL);
ap.declareXPathNameSpace(NS_OTHER, NS_OTHER_URL);
ap.declareXPathNameSpace(NS_RECORD, NS_RECORD_URL);
ap.declareXPathNameSpace(NS_ERROR, NS_ERROR_URL);
ap.declareXPathNameSpace(NS_WORK, NS_WORK_URL);
ap.declareXPathNameSpace(NS_ACTIVITIES, NS_ACTIVITIES_URL);
Map<String, String> workIdLastModifiedDate = new HashMap<>();
ap.selectXPath("//work:work-summary");
while (ap.evalXPath() != -1) {
String workId = "";
String lastModifiedDate = "";
int attr = vn.getAttrVal("put-code");
if (attr > -1) {
workId = vn.toNormalizedString(attr);
workIdLastModifiedDate.put(workId, "");
}
if (vn.toElement(VTDNav.FIRST_CHILD, "common:last-modified-date")) {
int val = vn.getText();
if (val != -1) {
lastModifiedDate = vn.toNormalizedString(val);
workIdLastModifiedDate.put(workId, lastModifiedDate);
}
vn.toElement(VTDNav.PARENT);
}
}
return workIdLastModifiedDate;
}
}

@ -37,7 +37,7 @@ import eu.dnetlib.doiboost.orcidnodoi.similarity.AuthorMatcher;
import scala.Tuple2;
/**
* This spark job generates one parquet file, containing orcid publications dataset
* This spark job generates orcid publications no doi dataset
*/
public class SparkGenEnrichedOrcidWorks {

@ -22,6 +22,10 @@ public class JsonWriter {
return OBJECT_MAPPER.writeValueAsString(authorData);
}
public static String create(Object obj) throws JsonProcessingException {
return OBJECT_MAPPER.writeValueAsString(obj);
}
public static String create(WorkData workData) {
JsonObject work = new JsonObject();
work.addProperty("oid", workData.getOid());

@ -31,6 +31,7 @@ public class PublicationToOaf implements Serializable {
static Logger logger = LoggerFactory.getLogger(PublicationToOaf.class);
public static final String ORCID = "ORCID";
public static final String ORCID_PID_TYPE_CLASSNAME = "Open Researcher and Contributor ID";
public final static String orcidPREFIX = "orcid_______";
public static final String OPENAIRE_PREFIX = "openaire____";
public static final String SEPARATOR = "::";
@ -79,10 +80,10 @@ public class PublicationToOaf implements Serializable {
{
put("ark".toLowerCase(), new Pair<>("ark", "ark"));
put("arxiv".toLowerCase(), new Pair<>("arxiv", "arXiv"));
put("pmc".toLowerCase(), new Pair<>("pmc", "pmc"));
put("pmid".toLowerCase(), new Pair<>("pmid", "pmid"));
put("source-work-id".toLowerCase(), new Pair<>("orcidworkid", "orcidworkid"));
put("arxiv".toLowerCase(), new Pair<>("arXiv", "arXiv"));
put("pmc".toLowerCase(), new Pair<>("pmc", "PubMed Central ID"));
put("pmid".toLowerCase(), new Pair<>("pmid", "PubMed ID"));
put("source-work-id".toLowerCase(), new Pair<>("orcidworkid", "orcid workid"));
put("urn".toLowerCase(), new Pair<>("urn", "urn"));
}
};
@ -152,8 +153,8 @@ public class PublicationToOaf implements Serializable {
.keySet()
.stream()
.forEach(jsonExtId -> {
final String classid = externalIds.get(jsonExtId.toLowerCase()).getValue();
final String classname = externalIds.get(jsonExtId.toLowerCase()).getKey();
final String classid = externalIds.get(jsonExtId.toLowerCase()).getKey();
final String classname = externalIds.get(jsonExtId.toLowerCase()).getValue();
final String extId = getStringValue(rootElement, jsonExtId);
if (StringUtils.isNotBlank(extId)) {
publication
@ -522,21 +523,21 @@ public class PublicationToOaf implements Serializable {
sp.setValue(orcidId);
final Qualifier q = new Qualifier();
q.setClassid(ORCID.toLowerCase());
q.setClassname(ORCID.toLowerCase());
q.setClassname(ORCID_PID_TYPE_CLASSNAME);
q.setSchemeid(ModelConstants.DNET_PID_TYPES);
q.setSchemename(ModelConstants.DNET_PID_TYPES);
sp.setQualifier(q);
final DataInfo dataInfo = new DataInfo();
dataInfo.setDeletedbyinference(false);
dataInfo.setInferred(false);
dataInfo.setTrust("0.9");
dataInfo.setTrust("0.91");
dataInfo
.setProvenanceaction(
mapQualifier(
"sysimport:crosswalk:entityregistry",
"Harvested",
"dnet:provenanceActions",
"dnet:provenanceActions"));
.setProvenanceaction(
mapQualifier(
"sysimport:crosswalk:entityregistry",
"Harvested",
"dnet:provenanceActions",
"dnet:provenanceActions"));
sp.setDataInfo(dataInfo);
return sp;
}

@ -1,3 +1,5 @@
[{"paramName":"w", "paramLongName":"workingPath", "paramDescription": "the working path", "paramRequired": true},
{"paramName":"a", "paramLongName":"authorsPath", "paramDescription": "the path of the authors seq file", "paramRequired": true},
{"paramName":"xw", "paramLongName":"xmlWorksPath", "paramDescription": "the path of the works xml seq file", "paramRequired": true},
{"paramName":"o", "paramLongName":"outputDoiAuthorListPath", "paramDescription": "the relative folder of the sequencial file to write the data", "paramRequired": true}
]

@ -1,18 +0,0 @@
<configuration>
<property>
<name>jobTracker</name>
<value>hadoop-rm3.garr-pa1.d4science.org:8032</value>
</property>
<property>
<name>nameNode</name>
<value>hdfs://hadoop-rm1.garr-pa1.d4science.org:8020</value>
</property>
<property>
<name>queueName</name>
<value>default</value>
</property>
<property>
<name>oozie.action.sharelib.for.spark</name>
<value>spark2</value>
</property>
</configuration>

@ -1,55 +1,99 @@
<workflow-app name="Gen_Doi_Author_List_WF" xmlns="uri:oozie:workflow:0.5">
<parameters>
<workflow-app name="gen_doi_author_list" 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>sparkExecutorCores</name>
<description>number of cores used by single executor</description>
</property>
<property>
<name>spark2MaxExecutors</name>
<value>40</value>
</property>
<property>
<name>oozieActionShareLibForSpark2</name>
<description>oozie action sharelib for spark 2.*</description>
</property>
<property>
<name>spark2ExtraListeners</name>
<value>com.cloudera.spark.lineage.NavigatorAppListener</value>
<description>spark 2.* extra listeners classname</description>
</property>
<property>
<name>spark2SqlQueryExecutionListeners</name>
<value>com.cloudera.spark.lineage.NavigatorQueryListener</value>
<description>spark 2.* sql query execution listeners classname</description>
</property>
<property>
<name>spark2YarnHistoryServerAddress</name>
<description>spark 2.* yarn history server address</description>
</property>
<property>
<name>spark2EventLogDir</name>
<description>spark 2.* event log dir location</description>
</property>
<property>
<name>workingPath</name>
<description>the working dir base path</description>
</property>
</parameters>
<global>
<job-tracker>${jobTracker}</job-tracker>
<name-node>${nameNode}</name-node>
<configuration>
<property>
<name>workingPath</name>
<description>the working dir base path</description>
<name>oozie.action.sharelib.for.spark</name>
<value>${oozieActionShareLibForSpark2}</value>
</property>
<property>
<name>sparkDriverMemory</name>
<description>memory for driver process</description>
</property>
<property>
<name>sparkExecutorMemory</name>
<description>memory for individual executor</description>
</property>
<property>
<name>sparkExecutorCores</name>
<description>number of cores used by single executor</description>
</property>
</parameters>
<start to="ResetWorkingPath"/>
<kill name="Kill">
<message>Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}]</message>
</kill>
<action name="ResetWorkingPath">
<fs>
<delete path='${workingPath_activities}/doi_author_list'/>
</fs>
<ok to="Gen_Doi_Author_List"/>
<error to="Kill"/>
</action>
<action name="Gen_Doi_Author_List">
<spark xmlns="uri:oozie:spark-action:0.2">
<job-tracker>${jobTracker}</job-tracker>
<name-node>${nameNode}</name-node>
<master>yarn</master>
<mode>cluster</mode>
<name>Gen_Doi_Author_List</name>
<class>eu.dnetlib.doiboost.orcid.SparkGenerateDoiAuthorList</class>
<jar>dhp-doiboost-1.2.1-SNAPSHOT.jar</jar>
<spark-opts>--num-executors 10 --conf spark.yarn.jars=&quot;hdfs://hadoop-rm1.garr-pa1.d4science.org:8020/user/oozie/share/lib/lib_20180405103059/spark2&quot; --executor-memory=${sparkExecutorMemory} --executor-cores=${sparkExecutorCores} --driver-memory=${sparkDriverMemory}
</spark-opts>
<arg>-w</arg><arg>${workingPath}/</arg>
<arg>-o</arg><arg>doi_author_list/</arg>
</spark>
<ok to="End"/>
<error to="Kill"/>
</action>
<end name="End"/>
</configuration>
</global>
<start to="ResetWorkingPath"/>
<kill name="Kill">
<message>Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}]</message>
</kill>
<action name="ResetWorkingPath">
<fs>
<delete path='${workingPath}/doi_author_list'/>
</fs>
<ok to="GenDoiAuthorList"/>
<error to="Kill"/>
</action>
<action name="GenDoiAuthorList">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn-cluster</master>
<mode>cluster</mode>
<name>GenDoiAuthorList</name>
<class>eu.dnetlib.doiboost.orcid.SparkGenerateDoiAuthorList</class>
<jar>dhp-doiboost-${projectVersion}.jar</jar>
<spark-opts>
--executor-memory=${sparkExecutorMemory}
--driver-memory=${sparkDriverMemory}
--conf spark.extraListeners=${spark2ExtraListeners}
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
--conf spark.dynamicAllocation.enabled=true
--conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors}
</spark-opts>
<arg>-w</arg><arg>${workingPath}/</arg>
<arg>-a</arg><arg>authors/authors.seq</arg>
<arg>-xw</arg><arg>xml/works/*.seq</arg>
<arg>-o</arg><arg>doi_author_list/</arg>
</spark>
<ok to="End"/>
<error to="Kill"/>
</action>
<end name="End"/>
</workflow-app>

@ -13,13 +13,13 @@ import java.text.SimpleDateFormat;
import java.time.Duration;
import java.time.LocalDateTime;
import java.time.temporal.TemporalUnit;
import java.util.Arrays;
import java.util.Date;
import java.util.List;
import java.util.*;
import java.util.stream.Collectors;
import org.apache.commons.compress.archivers.tar.TarArchiveEntry;
import org.apache.commons.compress.archivers.tar.TarArchiveInputStream;
import org.apache.commons.compress.compressors.gzip.GzipCompressorInputStream;
import org.apache.commons.compress.utils.Lists;
import org.apache.commons.io.IOUtils;
import org.apache.http.client.methods.CloseableHttpResponse;
import org.apache.http.client.methods.HttpGet;
@ -30,6 +30,7 @@ import org.junit.jupiter.api.Test;
import org.mortbay.log.Log;
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
import eu.dnetlib.dhp.schema.orcid.AuthorData;
import jdk.nashorn.internal.ir.annotations.Ignore;
public class OrcidClientTest {
@ -42,6 +43,9 @@ public class OrcidClientTest {
String toNotRetrieveDate = "2019-09-29 23:59:59.000000";
String lastUpdate = "2019-09-30 00:00:00";
String shortDate = "2020-05-06 16:06:11";
final String REQUEST_TYPE_RECORD = "record";
final String REQUEST_TYPE_WORK = "work/47652866";
final String REQUEST_TYPE_WORKS = "works";
// curl -i -H "Accept: application/vnd.orcid+xml"
// -H 'Authorization: Bearer 78fdb232-7105-4086-8570-e153f4198e3d'
@ -86,25 +90,25 @@ public class OrcidClientTest {
@Test
private void downloadTest(String orcid) throws Exception {
String record = testDownloadRecord(orcid);
String filename = "/tmp/downloaded_".concat(orcid).concat(".xml");
String record = testDownloadRecord(orcid, REQUEST_TYPE_RECORD);
String filename = "/tmp/downloaded_record_".concat(orcid).concat(".xml");
File f = new File(filename);
OutputStream outStream = new FileOutputStream(f);
IOUtils.write(record.getBytes(), outStream);
}
private String testDownloadRecord(String orcidId) throws Exception {
private String testDownloadRecord(String orcidId, String dataType) throws Exception {
try (CloseableHttpClient client = HttpClients.createDefault()) {
HttpGet httpGet = new HttpGet("https://api.orcid.org/v3.0/" + orcidId + "/record");
HttpGet httpGet = new HttpGet("https://api.orcid.org/v3.0/" + orcidId + "/" + dataType);
httpGet.addHeader("Accept", "application/vnd.orcid+xml");
httpGet.addHeader("Authorization", "Bearer 78fdb232-7105-4086-8570-e153f4198e3d");
logToFile("start connection: " + new Date(System.currentTimeMillis()).toString());
long start = System.currentTimeMillis();
CloseableHttpResponse response = client.execute(httpGet);
logToFile("end connection: " + new Date(System.currentTimeMillis()).toString());
long end = System.currentTimeMillis();
if (response.getStatusLine().getStatusCode() != 200) {
System.out
.println("Downloading " + orcidId + " status code: " + response.getStatusLine().getStatusCode());
logToFile("Downloading " + orcidId + " status code: " + response.getStatusLine().getStatusCode());
}
logToFile(orcidId + " " + dataType + " " + (end - start) / 1000 + " seconds");
return IOUtils.toString(response.getEntity().getContent());
} catch (Throwable e) {
e.printStackTrace();
@ -129,7 +133,7 @@ public class OrcidClientTest {
}
String[] values = line.split(",");
List<String> recordInfo = Arrays.asList(values);
testDownloadRecord(recordInfo.get(0));
testDownloadRecord(recordInfo.get(0), REQUEST_TYPE_RECORD);
long endReq = System.currentTimeMillis();
nReqTmp++;
if (nReqTmp == REQ_LIMIT) {
@ -190,7 +194,7 @@ public class OrcidClientTest {
.toString(getClass().getResourceAsStream("0000-0003-3028-6161.compressed.base64"));
final String recordFromSeqFile = ArgumentApplicationParser.decompressValue(base64CompressedRecord);
logToFile("\n\ndownloaded \n\n" + recordFromSeqFile);
final String downloadedRecord = testDownloadRecord("0000-0003-3028-6161");
final String downloadedRecord = testDownloadRecord("0000-0003-3028-6161", REQUEST_TYPE_RECORD);
assertTrue(recordFromSeqFile.equals(downloadedRecord));
}
@ -255,7 +259,7 @@ public class OrcidClientTest {
logToFile("modified: " + modified);
}
private void logToFile(String log)
public static void logToFile(String log)
throws IOException {
log = log.concat("\n");
Path path = Paths.get("/tmp/orcid_log.txt");
@ -298,4 +302,72 @@ public class OrcidClientTest {
}
return new String("");
}
@Test
private void downloadWorkTest() throws Exception {
String orcid = "0000-0003-0015-1952";
String record = testDownloadRecord(orcid, REQUEST_TYPE_WORK);
String filename = "/tmp/downloaded_work_".concat(orcid).concat(".xml");
File f = new File(filename);
OutputStream outStream = new FileOutputStream(f);
IOUtils.write(record.getBytes(), outStream);
}
@Test
private void downloadRecordTest() throws Exception {
String orcid = "0000-0001-5004-5918";
String record = testDownloadRecord(orcid, REQUEST_TYPE_RECORD);
String filename = "/tmp/downloaded_record_".concat(orcid).concat(".xml");
File f = new File(filename);
OutputStream outStream = new FileOutputStream(f);
IOUtils.write(record.getBytes(), outStream);
}
@Test
private void downloadWorksTest() throws Exception {
String orcid = "0000-0001-5004-5918";
String record = testDownloadRecord(orcid, REQUEST_TYPE_WORKS);
String filename = "/tmp/downloaded_works_".concat(orcid).concat(".xml");
File f = new File(filename);
OutputStream outStream = new FileOutputStream(f);
IOUtils.write(record.getBytes(), outStream);
}
@Test
private void downloadSingleWorkTest() throws Exception {
String orcid = "0000-0001-5004-5918";
String record = testDownloadRecord(orcid, REQUEST_TYPE_WORK);
String filename = "/tmp/downloaded_work_47652866_".concat(orcid).concat(".xml");
File f = new File(filename);
OutputStream outStream = new FileOutputStream(f);
IOUtils.write(record.getBytes(), outStream);
}
@Test
public void cleanAuthorListTest() throws Exception {
AuthorData a1 = new AuthorData();
a1.setOid("1");
a1.setName("n1");
a1.setSurname("s1");
a1.setCreditName("c1");
AuthorData a2 = new AuthorData();
a2.setOid("1");
a2.setName("n1");
a2.setSurname("s1");
a2.setCreditName("c1");
AuthorData a3 = new AuthorData();
a3.setOid("3");
a3.setName("n3");
a3.setSurname("s3");
a3.setCreditName("c3");
List<AuthorData> list = Lists.newArrayList();
list.add(a1);
list.add(a2);
list.add(a3);
Set<String> namesAlreadySeen = new HashSet<>();
assertTrue(list.size() == 3);
list.removeIf(a -> !namesAlreadySeen.add(a.getOid()));
assertTrue(list.size() == 2);
}
}

@ -4,10 +4,13 @@ package eu.dnetlib.doiboost.orcid.xml;
import static org.junit.jupiter.api.Assertions.assertNotNull;
import static org.junit.jupiter.api.Assertions.assertTrue;
import java.util.Map;
import org.apache.commons.io.IOUtils;
import org.junit.jupiter.api.Test;
import eu.dnetlib.dhp.schema.orcid.AuthorData;
import eu.dnetlib.doiboost.orcid.OrcidClientTest;
import eu.dnetlib.doiboost.orcid.model.WorkData;
import eu.dnetlib.doiboost.orcidnodoi.json.JsonWriter;
@ -59,7 +62,7 @@ public class XMLRecordParserTest {
}
@Test
public void testOrcidOtherNamesXMLParser() throws Exception {
private void testOrcidOtherNamesXMLParser() throws Exception {
String xml = IOUtils
.toString(
@ -74,4 +77,17 @@ public class XMLRecordParserTest {
String jsonData = JsonWriter.create(authorData);
assertNotNull(jsonData);
}
@Test
public void testWorkIdLastModifiedDateXMLParser() throws Exception {
String xml = IOUtils
.toString(
this.getClass().getResourceAsStream("record_8888-8888-8888-8880.xml"));
// Map<String, String> workIdLastModifiedDate = XMLRecordParser.retrieveWorkIdLastModifiedDate(xml.getBytes());
// String LastModifiedDate = workIdLastModifiedDate.get(0);
// OrcidClientTest.logToFile(LastModifiedDate + " -- " + workIdLastModifiedDate.get(LastModifiedDate));
String result = XMLRecordParser.retrieveWorkIdFromSummary(xml.getBytes(), "empty");
OrcidClientTest.logToFile(result);
}
}

Loading…
Cancel
Save