Merge pull request 'UnknowHostException handling for orcid collector api' (#141) from enrico.ottonello/dnet-hadoop:beta into beta

Reviewed-on: D-Net/dnet-hadoop#141
This commit is contained in:
Claudio Atzori 2021-09-22 11:51:13 +02:00
commit df15a4dc9f
5 changed files with 445 additions and 115 deletions

View File

@ -12,10 +12,6 @@ import org.apache.commons.io.IOUtils;
import org.apache.commons.lang3.StringUtils;
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.JavaSparkContext;
@ -25,8 +21,12 @@ import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
import eu.dnetlib.dhp.common.collection.CollectorException;
import eu.dnetlib.dhp.common.collection.HttpClientParams;
import eu.dnetlib.doiboost.orcid.model.DownloadedRecordData;
import eu.dnetlib.doiboost.orcid.util.DownloadsReport;
import eu.dnetlib.doiboost.orcid.util.HDFSUtil;
import eu.dnetlib.doiboost.orcid.util.MultiAttemptsHttpConnector;
import scala.Tuple2;
public class SparkDownloadOrcidAuthors {
@ -72,16 +72,12 @@ public class SparkDownloadOrcidAuthors {
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 errorHTTP404Acc = spark.sparkContext().longAccumulator("error_HTTP_404");
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");
LongAccumulator errorsAcc = spark.sparkContext().longAccumulator("errors");
logger.info("Retrieving data from lamda sequence file");
String lambdaFilePath = workingPath + lambdaFileName;
logger.info("Retrieving data from lamda sequence file: " + lambdaFilePath);
JavaPairRDD<Text, Text> lamdaFileRDD = sc
.sequenceFile(workingPath + lambdaFileName, Text.class, Text.class);
.sequenceFile(lambdaFilePath, Text.class, Text.class);
final long lamdaFileRDDCount = lamdaFileRDD.count();
logger.info("Data retrieved: {}", lamdaFileRDDCount);
@ -102,47 +98,44 @@ public class SparkDownloadOrcidAuthors {
final DownloadedRecordData downloaded = new DownloadedRecordData();
downloaded.setOrcidId(orcidId);
downloaded.setLastModifiedDate(lastModifiedDate);
CloseableHttpClient client = HttpClients.createDefault();
HttpGet httpGet = new HttpGet("https://api.orcid.org/v3.0/" + orcidId + "/record");
httpGet.addHeader("Accept", "application/vnd.orcid+xml");
httpGet.addHeader("Authorization", String.format("Bearer %s", token));
final HttpClientParams clientParams = new HttpClientParams();
MultiAttemptsHttpConnector httpConnector = new MultiAttemptsHttpConnector(clientParams);
httpConnector.setAuthMethod(MultiAttemptsHttpConnector.BEARER);
httpConnector.setAcceptHeaderValue("application/vnd.orcid+xml");
httpConnector.setAuthToken(token);
String apiUrl = "https://api.orcid.org/v3.0/" + orcidId + "/record";
DownloadsReport report = new DownloadsReport();
long startReq = System.currentTimeMillis();
CloseableHttpResponse response = client.execute(httpGet);
boolean downloadCompleted = false;
String record = "";
try {
record = httpConnector.getInputSource(apiUrl, report);
downloadCompleted = true;
} catch (CollectorException ce) {
if (!report.isEmpty()) {
int errCode = report.keySet().stream().findFirst().get();
report.forEach((k, v) -> {
logger.error(k + " " + v);
});
downloaded.setStatusCode(errCode);
} else {
downloaded.setStatusCode(-4);
}
errorsAcc.add(1);
}
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);
break;
case 404:
errorHTTP404Acc.add(1);
break;
case 409:
errorHTTP409Acc.add(1);
break;
case 503:
errorHTTP503Acc.add(1);
break;
case 525:
errorHTTP525Acc.add(1);
break;
default:
errorHTTPGenericAcc.add(1);
}
return downloaded.toTuple2();
if (downloadCompleted) {
downloaded.setStatusCode(200);
downloadedRecordsAcc.add(1);
downloaded
.setCompressedData(
ArgumentApplicationParser
.compressArgument(record));
}
downloadedRecordsAcc.add(1);
downloaded
.setCompressedData(
ArgumentApplicationParser
.compressArgument(IOUtils.toString(response.getEntity().getContent())));
client.close();
return downloaded.toTuple2();
};
@ -153,26 +146,17 @@ public class SparkDownloadOrcidAuthors {
long authorsModifiedCount = authorsModifiedRDD.count();
logger.info("Authors modified count: {}", authorsModifiedCount);
logger.info("Start downloading ...");
final JavaPairRDD<Text, Text> pairRDD = authorsModifiedRDD
.repartition(100)
.map(downloadRecordFn)
.mapToPair(t -> new Tuple2<>(new Text(t._1()), new Text(t._2())));
saveAsSequenceFile(workingPath, outputPath, sc, pairRDD);
logger.info("parsedRecordsAcc: {}", parsedRecordsAcc.value());
logger.info("modifiedRecordsAcc: {}", modifiedRecordsAcc.value());
logger.info("downloadedRecordsAcc: {}", downloadedRecordsAcc.value());
logger.info("errorHTTP403Acc: {}", errorHTTP403Acc.value());
logger.info("errorHTTP404Acc: {}", errorHTTP404Acc.value());
logger.info("errorHTTP409Acc: {}", errorHTTP409Acc.value());
logger.info("errorHTTP503Acc: {}", errorHTTP503Acc.value());
logger.info("errorHTTP525Acc: {}", errorHTTP525Acc.value());
logger.info("errorHTTPGenericAcc: {}", errorHTTPGenericAcc.value());
logger.info("errorsAcc: {}", errorsAcc.value());
});
}
private static void saveAsSequenceFile(String workingPath, String outputPath, JavaSparkContext sc,

View File

@ -11,10 +11,6 @@ import org.apache.commons.io.IOUtils;
import org.apache.commons.lang3.StringUtils;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.io.compress.GzipCodec;
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.JavaSparkContext;
@ -28,8 +24,12 @@ import com.google.gson.JsonElement;
import com.google.gson.JsonParser;
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
import eu.dnetlib.dhp.common.collection.CollectorException;
import eu.dnetlib.dhp.common.collection.HttpClientParams;
import eu.dnetlib.doiboost.orcid.model.DownloadedRecordData;
import eu.dnetlib.doiboost.orcid.util.DownloadsReport;
import eu.dnetlib.doiboost.orcid.util.HDFSUtil;
import eu.dnetlib.doiboost.orcid.util.MultiAttemptsHttpConnector;
import eu.dnetlib.doiboost.orcid.xml.XMLRecordParser;
import scala.Tuple2;
@ -90,12 +90,7 @@ public class SparkDownloadOrcidWorks {
.sparkContext()
.longAccumulator("error_parsing_xml_found");
LongAccumulator downloadedRecordsAcc = spark.sparkContext().longAccumulator("downloaded_records");
LongAccumulator errorHTTP403Acc = spark.sparkContext().longAccumulator("error_HTTP_403");
LongAccumulator errorHTTP404Acc = spark.sparkContext().longAccumulator("error_HTTP_404");
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");
LongAccumulator errorsAcc = spark.sparkContext().longAccumulator("errors");
JavaPairRDD<Text, Text> updatedAuthorsRDD = sc
.sequenceFile(workingPath + "downloads/updated_authors/*", Text.class, Text.class);
@ -149,51 +144,44 @@ public class SparkDownloadOrcidWorks {
final DownloadedRecordData downloaded = new DownloadedRecordData();
downloaded.setOrcidId(orcidId);
downloaded.setLastModifiedDate(lastUpdateValue);
CloseableHttpClient client = HttpClients.createDefault();
HttpGet httpGet = new HttpGet("https://api.orcid.org/v3.0/" + relativeWorkUrl);
httpGet.addHeader("Accept", "application/vnd.orcid+xml");
httpGet.addHeader("Authorization", String.format("Bearer %s", token));
final HttpClientParams clientParams = new HttpClientParams();
MultiAttemptsHttpConnector httpConnector = new MultiAttemptsHttpConnector(clientParams);
httpConnector.setAuthMethod(MultiAttemptsHttpConnector.BEARER);
httpConnector.setAcceptHeaderValue("application/vnd.orcid+xml");
httpConnector.setAuthToken(token);
String apiUrl = "https://api.orcid.org/v3.0/" + relativeWorkUrl;
DownloadsReport report = new DownloadsReport();
long startReq = System.currentTimeMillis();
CloseableHttpResponse response = client.execute(httpGet);
boolean downloadCompleted = false;
String record = "";
try {
record = httpConnector.getInputSource(apiUrl, report);
downloadCompleted = true;
} catch (CollectorException ce) {
if (!report.isEmpty()) {
int errCode = report.keySet().stream().findFirst().get();
report.forEach((k, v) -> {
logger.error(k + " " + v);
});
downloaded.setStatusCode(errCode);
} else {
downloaded.setStatusCode(-4);
}
errorsAcc.add(1);
}
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);
break;
case 404:
errorHTTP404Acc.add(1);
break;
case 409:
errorHTTP409Acc.add(1);
break;
case 503:
errorHTTP503Acc.add(1);
break;
case 525:
errorHTTP525Acc.add(1);
break;
default:
errorHTTPGenericAcc.add(1);
logger
.info(
"Downloading {} status code: {}", orcidId,
response.getStatusLine().getStatusCode());
}
return downloaded.toTuple2();
if (downloadCompleted) {
downloaded.setStatusCode(200);
downloadedRecordsAcc.add(1);
downloaded
.setCompressedData(
ArgumentApplicationParser
.compressArgument(record));
}
downloadedRecordsAcc.add(1);
downloaded
.setCompressedData(
ArgumentApplicationParser
.compressArgument(IOUtils.toString(response.getEntity().getContent())));
client.close();
return downloaded.toTuple2();
};
@ -214,11 +202,7 @@ public class SparkDownloadOrcidWorks {
logger.info("errorLoadingXMLFoundAcc: {}", errorLoadingXMLFoundAcc.value());
logger.info("errorParsingXMLFoundAcc: {}", errorParsingXMLFoundAcc.value());
logger.info("downloadedRecordsAcc: {}", downloadedRecordsAcc.value());
logger.info("errorHTTP403Acc: {}", errorHTTP403Acc.value());
logger.info("errorHTTP409Acc: {}", errorHTTP409Acc.value());
logger.info("errorHTTP503Acc: {}", errorHTTP503Acc.value());
logger.info("errorHTTP525Acc: {}", errorHTTP525Acc.value());
logger.info("errorHTTPGenericAcc: {}", errorHTTPGenericAcc.value());
logger.info("errorsAcc: {}", errorsAcc.value());
});
}

View File

@ -0,0 +1,10 @@
package eu.dnetlib.doiboost.orcid.util;
import java.util.LinkedHashMap;
public class DownloadsReport extends LinkedHashMap<Integer, String> {
public DownloadsReport() {
}
}

View File

@ -0,0 +1,272 @@
package eu.dnetlib.doiboost.orcid.util;
import static eu.dnetlib.dhp.utils.DHPUtils.MAPPER;
import java.io.IOException;
import java.io.InputStream;
import java.net.*;
import java.util.List;
import java.util.Map;
import org.apache.commons.io.IOUtils;
import org.apache.commons.lang3.math.NumberUtils;
import org.apache.http.HttpHeaders;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import eu.dnetlib.dhp.common.collection.CollectorException;
import eu.dnetlib.dhp.common.collection.HttpClientParams;
/**
* Derived from eu.dnetlib.dhp.common.collection.HttpConnector2 with custom report and Bearer auth
*
* @author enrico
*/
public class MultiAttemptsHttpConnector {
private static final Logger log = LoggerFactory.getLogger(MultiAttemptsHttpConnector.class);
private HttpClientParams clientParams;
private String responseType = null;
private static final String userAgent = "Mozilla/5.0 (compatible; OAI; +http://www.openaire.eu)";
private String authToken = "";
private String acceptHeaderValue = "";
private String authMethod = "";
public final static String BEARER = "BEARER";
public MultiAttemptsHttpConnector() {
this(new HttpClientParams());
}
public MultiAttemptsHttpConnector(HttpClientParams clientParams) {
this.clientParams = clientParams;
CookieHandler.setDefault(new CookieManager(null, CookiePolicy.ACCEPT_ALL));
}
/**
* Given the URL returns the content via HTTP GET
*
* @param requestUrl the URL
* @param report the list of errors
* @return the content of the downloaded resource
* @throws CollectorException when retrying more than maxNumberOfRetry times
*/
public String getInputSource(final String requestUrl, DownloadsReport report)
throws CollectorException {
return attemptDownloadAsString(requestUrl, 1, report);
}
private String attemptDownloadAsString(final String requestUrl, final int retryNumber,
final DownloadsReport report) throws CollectorException {
try (InputStream s = attemptDownload(requestUrl, retryNumber, report)) {
return IOUtils.toString(s);
} catch (IOException e) {
log.error(e.getMessage(), e);
throw new CollectorException(e);
}
}
private InputStream attemptDownload(final String requestUrl, final int retryNumber,
final DownloadsReport report) throws CollectorException, IOException {
if (retryNumber > getClientParams().getMaxNumberOfRetry()) {
final String msg = String
.format(
"Max number of retries (%s/%s) exceeded, failing.",
retryNumber, getClientParams().getMaxNumberOfRetry());
log.error(msg);
throw new CollectorException(msg);
}
log.info("Request attempt {} [{}]", retryNumber, requestUrl);
InputStream input = null;
try {
if (getClientParams().getRequestDelay() > 0) {
backoffAndSleep(getClientParams().getRequestDelay());
}
final HttpURLConnection urlConn = (HttpURLConnection) new URL(requestUrl).openConnection();
urlConn.setInstanceFollowRedirects(false);
urlConn.setReadTimeout(getClientParams().getReadTimeOut() * 1000);
urlConn.setConnectTimeout(getClientParams().getConnectTimeOut() * 1000);
urlConn.addRequestProperty(HttpHeaders.USER_AGENT, userAgent);
if (!getAcceptHeaderValue().isEmpty()) {
urlConn.addRequestProperty(HttpHeaders.ACCEPT, getAcceptHeaderValue());
}
if (!getAuthToken().isEmpty() && getAuthMethod().equals(BEARER)) {
urlConn.addRequestProperty(HttpHeaders.AUTHORIZATION, String.format("Bearer %s", getAuthToken()));
}
if (log.isDebugEnabled()) {
logHeaderFields(urlConn);
}
int retryAfter = obtainRetryAfter(urlConn.getHeaderFields());
if (is2xx(urlConn.getResponseCode())) {
input = urlConn.getInputStream();
responseType = urlConn.getContentType();
return input;
}
if (is3xx(urlConn.getResponseCode())) {
// REDIRECTS
final String newUrl = obtainNewLocation(urlConn.getHeaderFields());
log.info("The requested url has been moved to {}", newUrl);
report
.put(
urlConn.getResponseCode(),
String.format("Moved to: %s", newUrl));
urlConn.disconnect();
if (retryAfter > 0) {
backoffAndSleep(retryAfter);
}
return attemptDownload(newUrl, retryNumber + 1, report);
}
if (is4xx(urlConn.getResponseCode()) || is5xx(urlConn.getResponseCode())) {
switch (urlConn.getResponseCode()) {
case HttpURLConnection.HTTP_NOT_FOUND:
case HttpURLConnection.HTTP_BAD_GATEWAY:
case HttpURLConnection.HTTP_UNAVAILABLE:
case HttpURLConnection.HTTP_GATEWAY_TIMEOUT:
if (retryAfter > 0) {
log
.warn(
"{} - waiting and repeating request after suggested retry-after {} sec.",
requestUrl, retryAfter);
backoffAndSleep(retryAfter * 1000);
} else {
log
.warn(
"{} - waiting and repeating request after default delay of {} sec.",
requestUrl, getClientParams().getRetryDelay());
backoffAndSleep(retryNumber * getClientParams().getRetryDelay() * 1000);
}
report.put(urlConn.getResponseCode(), requestUrl);
urlConn.disconnect();
return attemptDownload(requestUrl, retryNumber + 1, report);
default:
report
.put(
urlConn.getResponseCode(),
String
.format(
"%s Error: %s", requestUrl, urlConn.getResponseMessage()));
throw new CollectorException(urlConn.getResponseCode() + " error " + report);
}
}
throw new CollectorException(
String
.format(
"Unexpected status code: %s errors: %s", urlConn.getResponseCode(),
MAPPER.writeValueAsString(report)));
} catch (MalformedURLException | UnknownHostException e) {
log.error(e.getMessage(), e);
report.put(-2, e.getMessage());
throw new CollectorException(e.getMessage(), e);
} catch (SocketTimeoutException | SocketException e) {
log.error(e.getMessage(), e);
report.put(-3, e.getMessage());
backoffAndSleep(getClientParams().getRetryDelay() * retryNumber * 1000);
return attemptDownload(requestUrl, retryNumber + 1, report);
}
}
private void logHeaderFields(final HttpURLConnection urlConn) throws IOException {
log.debug("StatusCode: {}", urlConn.getResponseMessage());
for (Map.Entry<String, List<String>> e : urlConn.getHeaderFields().entrySet()) {
if (e.getKey() != null) {
for (String v : e.getValue()) {
log.debug(" key: {} - value: {}", e.getKey(), v);
}
}
}
}
private void backoffAndSleep(int sleepTimeMs) throws CollectorException {
log.info("I'm going to sleep for {}ms", sleepTimeMs);
try {
Thread.sleep(sleepTimeMs);
} catch (InterruptedException e) {
log.error(e.getMessage(), e);
throw new CollectorException(e);
}
}
private int obtainRetryAfter(final Map<String, List<String>> headerMap) {
for (String key : headerMap.keySet()) {
if ((key != null) && key.equalsIgnoreCase(HttpHeaders.RETRY_AFTER) && (!headerMap.get(key).isEmpty())
&& NumberUtils.isCreatable(headerMap.get(key).get(0))) {
return Integer.parseInt(headerMap.get(key).get(0)) + 10;
}
}
return -1;
}
private String obtainNewLocation(final Map<String, List<String>> headerMap) throws CollectorException {
for (String key : headerMap.keySet()) {
if ((key != null) && key.equalsIgnoreCase(HttpHeaders.LOCATION) && (headerMap.get(key).size() > 0)) {
return headerMap.get(key).get(0);
}
}
throw new CollectorException("The requested url has been MOVED, but 'location' param is MISSING");
}
private boolean is2xx(final int statusCode) {
return statusCode >= 200 && statusCode <= 299;
}
private boolean is4xx(final int statusCode) {
return statusCode >= 400 && statusCode <= 499;
}
private boolean is3xx(final int statusCode) {
return statusCode >= 300 && statusCode <= 399;
}
private boolean is5xx(final int statusCode) {
return statusCode >= 500 && statusCode <= 599;
}
public String getResponseType() {
return responseType;
}
public HttpClientParams getClientParams() {
return clientParams;
}
public void setClientParams(HttpClientParams clientParams) {
this.clientParams = clientParams;
}
public void setAuthToken(String authToken) {
this.authToken = authToken;
}
private String getAuthToken() {
return authToken;
}
public String getAcceptHeaderValue() {
return acceptHeaderValue;
}
public void setAcceptHeaderValue(String acceptHeaderValue) {
this.acceptHeaderValue = acceptHeaderValue;
}
public String getAuthMethod() {
return authMethod;
}
public void setAuthMethod(String authMethod) {
this.authMethod = authMethod;
}
}

View File

@ -1,13 +1,11 @@
package eu.dnetlib.doiboost.orcid;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertTrue;
import static org.junit.jupiter.api.Assertions.*;
import java.io.*;
import java.nio.file.Files;
import java.nio.file.Path;
import java.nio.file.Paths;
import java.nio.file.StandardOpenOption;
import java.text.ParseException;
import java.text.SimpleDateFormat;
@ -17,7 +15,6 @@ 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.FileUtils;
import org.apache.commons.io.IOUtils;
import org.apache.http.client.methods.CloseableHttpResponse;
import org.apache.http.client.methods.HttpGet;
@ -28,8 +25,11 @@ import org.junit.jupiter.api.Disabled;
import org.junit.jupiter.api.Test;
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
import eu.dnetlib.dhp.common.collection.CollectorException;
import eu.dnetlib.dhp.common.collection.HttpClientParams;
import eu.dnetlib.dhp.schema.orcid.AuthorData;
import eu.dnetlib.doiboost.orcid.xml.XMLRecordParserTest;
import eu.dnetlib.doiboost.orcid.util.DownloadsReport;
import eu.dnetlib.doiboost.orcid.util.MultiAttemptsHttpConnector;
import jdk.nashorn.internal.ir.annotations.Ignore;
public class OrcidClientTest {
@ -49,7 +49,7 @@ public class OrcidClientTest {
@BeforeAll
private static void setUp() throws IOException {
testPath = Files.createTempDirectory(XMLRecordParserTest.class.getName());
testPath = Files.createTempDirectory(OrcidClientTest.class.getName());
System.out.println("using test path: " + testPath);
}
@ -349,4 +349,84 @@ public class OrcidClientTest {
final String work = ArgumentApplicationParser.decompressValue(base64CompressedWork);
logToFile(testPath, "\n\nwork updated \n\n" + work);
}
@Test
void downloadUnknownHostExceptionTest() throws Exception {
logToFile(testPath, "downloadUnknownHostExceptionTest");
final String orcid = "0000-0001-7291-3210";
final HttpClientParams clientParams = new HttpClientParams();
clientParams.setMaxNumberOfRetry(2);
MultiAttemptsHttpConnector httpConnector = new MultiAttemptsHttpConnector(clientParams);
httpConnector.setAuthMethod(MultiAttemptsHttpConnector.BEARER);
httpConnector.setAcceptHeaderValue("application/vnd.orcid+xml");
httpConnector.setAuthToken("78fdb232-7105-4086-8570-e153f4198e3d");
String wrongApiUrl = "https://api.orcid_UNKNOWN.org/v3.0/" + orcid + "/" + REQUEST_TYPE_RECORD;
String url = "UNKNOWN";
DownloadsReport report = new DownloadsReport();
try {
httpConnector.getInputSource(wrongApiUrl, report);
} catch (CollectorException ce) {
logToFile(testPath, "CollectorException downloading: " + ce.getMessage());
} catch (Throwable t) {
logToFile(testPath, "Throwable downloading: " + t.getMessage());
}
}
@Test
void downloadAttemptSuccessTest() throws Exception {
logToFile(testPath, "downloadAttemptSuccessTest");
final String orcid = "0000-0001-7291-3210";
final HttpClientParams clientParams = new HttpClientParams();
clientParams.setMaxNumberOfRetry(2);
MultiAttemptsHttpConnector httpConnector = new MultiAttemptsHttpConnector(clientParams);
httpConnector.setAuthMethod(MultiAttemptsHttpConnector.BEARER);
httpConnector.setAcceptHeaderValue("application/vnd.orcid+xml");
httpConnector.setAuthToken("78fdb232-7105-4086-8570-e153f4198e3d");
String apiUrl = "https://api.orcid.org/v3.0/" + orcid + "/" + REQUEST_TYPE_RECORD;
String url = "UNKNOWN";
DownloadsReport report = new DownloadsReport();
String record = httpConnector.getInputSource(apiUrl, report);
logToFile(testPath, "Downloaded at first attempt record: " + record);
}
@Test
void downloadAttemptNotFoundTest() throws Exception {
logToFile(testPath, "downloadAttemptNotFoundTest");
final HttpClientParams clientParams = new HttpClientParams();
clientParams.setMaxNumberOfRetry(2);
MultiAttemptsHttpConnector httpConnector = new MultiAttemptsHttpConnector(clientParams);
httpConnector.setAuthMethod(MultiAttemptsHttpConnector.BEARER);
httpConnector.setAcceptHeaderValue("application/vnd.orcid+xml");
httpConnector.setAuthToken("78fdb232-7105-4086-8570-e153f4198e3d");
String apiUrl = "https://api.orcid.org/v3.0/NOTFOUND/" + REQUEST_TYPE_RECORD;
DownloadsReport report = new DownloadsReport();
try {
httpConnector.getInputSource(apiUrl, report);
} catch (CollectorException ce) {
}
report.forEach((k, v) -> {
try {
logToFile(testPath, k + " " + v);
} catch (IOException e) {
e.printStackTrace();
}
});
}
@Test
@Ignore
void testDownloadedAuthor() throws Exception {
final String base64CompressedWork = "H4sIAAAAAAAAAI2Yy26jMBSG932KiD0hIe1MiwiVZjGLkWbX2XRHsFOsgs3YJmnefszFFy4+mUhtVPz9P/gcH/vQ9PWrrjYXzAVh9Bjst7tgg2nBEKEfx+DP28/wOdgImVOUV4ziY3DDInjNHlKOC8ZRMnxtmlyWxyDaqU+ofg7h/uX7IYwfn+Ngo25ARUKoxJzm1TEopWySKLper1vGC4LU74+IikgTWoFRW+SyfyyfxCBag4iQhBawyoGMDjdqJrnECJAZRquYLDEPaV5jv8oyWlXj+qTiXZLGr7KMiQbnjAOR6IY1W7C6hgIwjGt6SKGfHsY13ajHYipLIcIyJ5Xw6+akdvjEtyt4wxEwM6+VGph5N2zYr2ENhQRhKsmZYChmS1j7nFs6VIBPOwImKhyfMVeFg6GAWEjrcoQ4FoBmBGwVXYhagGHDBIEX+ZzUDiqyn35VN6rJUpUJ4zc/PAI2T03FbrUKJZQszWjV3zavVOjvVfoE01qB+YUUQPGNwHTt3luxJjdqh1AxJFBKLWOrSeCcF13RtxxYtlPOPqH6m+MLwVfoMQ2kdae2ArLajc6fTxkI1nIoegs0yB426pMO+0fSw07xDKMu0XKSde5C2VvrlVMijRzFwqY7XTJI1QMLWcmEzMxtDdxfHiYSgTNJnYJ1K9y5k0tUrMgrnGGaRiuXxxuClulYUbr0nBvpkYLjvgTCGsuSoex3f1CEvRPHKI184NJKtKeaiO7cD5E61bJ4F+9DFd7d01u8Tw6H5BBvvz8f3q3nXLGIeJULGdaqeVBBRK7rS7h/fNvvk/gpedxt4923dxP7Fc3KtKuc1BhlkrfYmeN4dcmrhmbw60+HmWw2CKgbTuqc32CXKTTmeTWT6bDBjPsQ0DTpnchdaYO0ayQ2FyLIiVREqs25aU8VKYLRbK0BsyZuqvr1MU2Sm/rDdhe/2CRN6FU/b+oBVyj1zqRtC5F8kAumfTclsl+s7EoNQu64nfOaVLeezX60Z3XCULLi6GI2IZGTEeey7fec9lBAuXawIHKcpifE7GABHWfoxLVfpUNPBXoMbZWrHFsR3bPAk9J9i2sw9nW6AQT1mpk++7JhW+v44Hmt8PomJqfD13jRnvFOSxCKtu6qHoyBbQ7cMFo750UEfGaXm6bEeplXIXj2hvL6mA7tzvIwmM9pbJFBG834POZdLGi2gH2u9u0K9HMwn5PTioFWLufzmrS4oNuU9Pkt2rf/2jMs7fMdm2rQTTM+j+49AzToAVuXYA1mD2k0+XdE9vAP+JYR5NcQAAA=";
final String work = ArgumentApplicationParser.decompressValue(base64CompressedWork);
logToFile(testPath, "\n\ndownloaded author \n\n" + work);
}
@Test
@Ignore
void testDownloadedWork() throws Exception {
final String base64CompressedWork = "H4sIAAAAAAAAANVa63LiOBb+z1Oo+LVbhbkGAlTCLE1Id9IhTQV6unr/CVvB2tiWR5Khmal5rX2BfbE9ki3b3Jzt6Y13h6pQSPrOXTo6knL10zffQxvCBWXBdbVVb1YRCWzm0GB9Xf28vLX6VSQkDhzssYBcV3dEVH8aVa62jL8M1RcKI2kBAYwNLnrtXrMPFCGW7nW10YSPBX8dq3XRb1swNGgomkaG3FBBV9SjcnddDaOVR+0qApUCMaSBJDzA3nXVlTIcNhrb7bbOuE0d+F43AtEwCENBnMjGUhtyjiSFGBqHCkkDu5gqB0rpSMgJsCJOAVmKMVRMuoRbAfbJeaoMY6h84q8gQi4Nz1NlmNQbnDNe4Ak1bLA28/0iB8TjBg1GMV5gdzxu0CGoxSBKlkMkpp44T3eINBxeyG5bKDABpJb7QF1guRpOsd/iOWRRhwSSPlNS5LNjsOHzHAXxmjlHmwBSr3DyTDgsNVLkkAxk6LDjcCIKaBJAtoo2FCagFTJBiyf5IdJwUAv2PJUaNUgXlgnju/PgBJDFKfTYzgdXFgXLYAzVLxH2wPWvrfQ9mKEVhG+oXbD4EsD+3H1txqaxgQwBPqRFIc0w2WoSBHNbLfqIF0zbfVymIbQ52VCyLVIzBRm6VeQVRFWNHuoHDASLeJH3jqDVUQXB5yrOH0ObE5UNLQe+R+1mu2U1u1Z7sGy2hq3esN2tt5oXf79qnELv8fGwkJYPmxSswD1uA6vVXrY7w+5g2G3WuxedjNsJmj2escJx33G/ZXsU5iAs/AyRR0WcjpRXBLglc0lM1BjP59bX1qw9Hn/+dH87/dy9vBikeinKkyzVHjoqJNWIk7QuE3KU6pES6O7MwsarJh44QW1KowcWOCxAC9tlzEPsGX3YrYGQICgS0JKzENach2bEoTYNyKEQzaJyQnzSqesKSaV3IhRx92L8tLAm7GerjbZUujSwlFnIobqKkTuth+Q4ED4Vqqypp5JyfK8ah5Ji0f8AZVSGT2TZVGXfBLw/liOyqdRpJqfyXr8ldyEZrehKkm8Jr/2hc3Qb7EVk9DfMJbU98pu3k+6aETXXBebCZpt23tBaBUfSZRxdo98eYmgNfRxrh3zAnldDM/37FvZ+IiWtoQfddgiaEGBIDGCG7btA7jgBP9svAK2h90l4yYqIGop5jgMHXA4J0NB9ksR+YTX0qFtfqACO01jGjDHFPx552AW2W0P3uvGROk4NLfTvCeNS8X9MaDg1rL9Qz6PYh7En3f4ZNmKS6nUfQYFmE6PYe05IYBqPFGaq5wHlYpaoDbYqxokVK+JBerz51z+BIzc+SfSdTHVrTiSYtZzGFNOdGrr5ohsLF2+NUguqppkDoua6/S6yXwAYu44pM+/HiZ1BwEDWMqYbC5fjZ+MEBwMjb4PRLdTFYWrUwiUhJH/H+G3pMl/7fjqJhTGwSwU5lnfLsVDmxIPvmRetbJeCOsvfaxWXbXWxLVziqNky51BLW1OP2JKzgNoASSa7Gk1WAfrLI9mirzBBIUD1r/W/AgrMla7CjEMOzYBJolo30/mnxd0SzadPt5+eZtMb9O7rEN1wNINgEA8Ha+IxNMdrHLCQRR4TFRCudnmB7m6GqD0YDCqW+lQqlfnndw93iw/TJ/RwN5k+TqZDNJkAQyUvUlWvktjrdgbQEeI1EapN8Grd7MOeYJlfajSxWVOMfcIhVQXgfcFsqhcceobVA/U3GjsbDCYrjVSKSz0wHo8Xym6dArRvvjsbAfUGouFr8s5lG9o72DVVSy1saDqMqlarWW+12r2GiIXXMzuAU6AQcLLqWf3mZRf6iOlsNQdda9BudhQnvNNdPWN8XA7BgU5G2k3pLADA75XD3BSnn3y+3M90SbZWGczkxiRVmfSaJrd0V8u0yG3CeYRyht7O07Ste45weuqNmhcpLO44woEPRq1eilLN/f3ntEqGPFfzi2PmudHTO3EOEKf60LdTyUeDr7KIIzKfTfqtdr896JxklQtbES/IQD7UyL+SZIJSXYhLHkHZ9oqEjPR1MRzWu550cDYdCeI9n+S4hzouUU76+UeCQJ0fjkKn0+v3m703i0Eh/z97BCDH/XAAziTIt4rH94j7s4dHbSY/HJ90e3qriBQL+MMxCGETs9j/QxiSQ5PaS63/QsZqdS8vOxdvtj7Oc//fL4dTI2LvDAfVA6erSDKe3+cPxw70j4c5HHZlfLT9iAEZYKjZkxOYKZxymJy659l/t+QZllC5bvVJrzShD5GN0/NkiaZyqNcJh0NrdngtTfp7wviaHB+SS1Ng7O+Sk3h5HodT4S8RyY78pUmGM6eEg1l8tVCa1KnvY/SgrzDKsxRLF46j+uahNKH3BE6lsIb1lUxpUhdS3WUE+u6nPP/qiyAsklumMhMz9SBNqeus0oQ+QXqwIa7m3qy87IhXnBLPI8kVXXlZMaASm5vAEqWuKYkvHMtbPdiPiIdm6dVmeVMZjX+lfnKDWmaRAT7ev6ctTfhEF3RoWnJeXlKfSXcHcsf69rk0wTd4Qx30RV9yl5et2Ipwqe/SS5MJXiU8vbIv2b/qZaC8PZ65AUwj9QJR3vx1mQ9b7VPy1FFebnSpWq7xi0qJuwA+fLYpL7rwJdLXobcSa97kM4Cl35f3YXmofp0+8R9gBc/XeXL9Vn38pH7mLTs27z9T8ky1n7ynlZ0I4le78rYzl6t/woG5krwQlpcRcLDD2UPkH5F73C9G5tFKfY0q/wa1TIHI0CgAAA==";
final String work = ArgumentApplicationParser.decompressValue(base64CompressedWork);
logToFile(testPath, "\n\ndownloaded work \n\n" + work);
}
}