orcid-no-doi #43

Merged
claudio.atzori merged 45 commits from enrico.ottonello/dnet-hadoop:orcid-no-doi into master 2020-12-02 10:55:12 +01:00
13 changed files with 608 additions and 443 deletions
Showing only changes of commit 97c8111847 - Show all commits

View File

@ -1,14 +1,15 @@
package eu.dnetlib.doiboost.orcid;
import java.io.BufferedReader;
import java.io.IOException;
import java.io.InputStreamReader;
import java.io.*;
import java.text.SimpleDateFormat;
import java.util.Arrays;
import java.util.Date;
import java.util.List;
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.io.IOUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataInputStream;
@ -16,6 +17,7 @@ import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.SequenceFile;
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;
@ -27,10 +29,10 @@ import eu.dnetlib.dhp.application.ArgumentApplicationParser;
public class OrcidDownloader extends OrcidDSManager {
static final int REQ_LIMIT = 24;
// static final int REQ_MAX_TEST = 100;
static final int RECORD_PARSED_COUNTER_LOG_INTERVAL = 10000;
static final int REQ_MAX_TEST = -1;
static final int RECORD_PARSED_COUNTER_LOG_INTERVAL = 500;
static final String DATE_FORMAT = "yyyy-MM-dd HH:mm:ss";
static final String lastUpdate = "2019-09-30 00:00:00";
static final String lastUpdate = "2020-09-29 00:00:00";
private String lambdaFileName;
private String outputPath;
private String token;
@ -41,7 +43,7 @@ public class OrcidDownloader extends OrcidDSManager {
orcidDownloader.parseLambdaFile();
}
private String downloadRecord(String orcidId) {
private String downloadRecord(String orcidId) throws IOException {
try (CloseableHttpClient client = HttpClients.createDefault()) {
HttpGet httpGet = new HttpGet("https://api.orcid.org/v3.0/" + orcidId + "/record");
httpGet.addHeader("Accept", "application/vnd.orcid+xml");
@ -49,17 +51,23 @@ public class OrcidDownloader extends OrcidDSManager {
CloseableHttpResponse response = client.execute(httpGet);
if (response.getStatusLine().getStatusCode() != 200) {
Log
.warn(
.info(
"Downloading " + orcidId + " status code: " + response.getStatusLine().getStatusCode());
return new String("");
}
return IOUtils.toString(response.getEntity().getContent());
} catch (Throwable e) {
Log.warn("Downloading " + orcidId, e.getMessage());
// return IOUtils.toString(response.getEntity().getContent());
return xmlStreamToString(response.getEntity().getContent());
}
return new String("");
}
private String xmlStreamToString(InputStream xmlStream) throws IOException {
BufferedReader br = new BufferedReader(new InputStreamReader(xmlStream));
String line;
StringBuffer buffer = new StringBuffer();
while ((line = br.readLine()) != null) {
buffer.append(line);
}
return buffer.toString();
}
public void parseLambdaFile() throws Exception {
@ -76,90 +84,87 @@ public class OrcidDownloader extends OrcidDSManager {
hdfsServerUri
.concat(workingPath)
.concat(outputPath)
.concat("orcid_records.seq"));
try (SequenceFile.Writer writer = SequenceFile
.createWriter(
conf,
SequenceFile.Writer.file(hdfsoutputPath),
SequenceFile.Writer.keyClass(Text.class),
SequenceFile.Writer.valueClass(Text.class))) {
try (BufferedReader br = new BufferedReader(new InputStreamReader(lambdaFileStream))) {
String line;
int nReqTmp = 0;
.concat("updated_xml_authors.seq"));
try (TarArchiveInputStream tais = new TarArchiveInputStream(
new GzipCompressorInputStream(lambdaFileStream))) {
TarArchiveEntry entry = null;
StringBuilder sb = new StringBuilder();
try (SequenceFile.Writer writer = SequenceFile
.createWriter(
conf,
SequenceFile.Writer.file(hdfsoutputPath),
SequenceFile.Writer.keyClass(Text.class),
SequenceFile.Writer.valueClass(Text.class),
SequenceFile.Writer.compression(SequenceFile.CompressionType.BLOCK, new GzipCodec()))) {
startDownload = System.currentTimeMillis();
long startReqTmp = System.currentTimeMillis();
while ((line = br.readLine()) != null) {
parsedRecordsCounter++;
// skip headers line
if (parsedRecordsCounter == 1) {
continue;
}
String[] values = line.split(",");
List<String> recordInfo = Arrays.asList(values);
String orcidId = recordInfo.get(0);
if (isModified(orcidId, recordInfo.get(3))) {
String record = downloadRecord(orcidId);
downloadedRecordsCounter++;
if (!record.isEmpty()) {
String compressRecord = ArgumentApplicationParser.compressArgument(record);
final Text key = new Text(recordInfo.get(0));
final Text value = new Text(compressRecord);
try {
while ((entry = tais.getNextTarEntry()) != null) {
BufferedReader br = new BufferedReader(new InputStreamReader(tais)); // Read directly from tarInput
String line;
while ((line = br.readLine()) != null) {
String[] values = line.split(",");
List<String> recordInfo = Arrays.asList(values);
int nReqTmp = 0;
long startReqTmp = System.currentTimeMillis();
// skip headers line
if (parsedRecordsCounter == 0) {
parsedRecordsCounter++;
continue;
}
parsedRecordsCounter++;
String orcidId = recordInfo.get(0);
if (isModified(orcidId, recordInfo.get(3))) {
String record = downloadRecord(orcidId);
downloadedRecordsCounter++;
if (!record.isEmpty()) {
// String compressRecord = ArgumentApplicationParser.compressArgument(record);
final Text key = new Text(recordInfo.get(0));
final Text value = new Text(record);
writer.append(key, value);
savedRecordsCounter++;
} catch (IOException e) {
Log.warn("Writing to sequence file: " + e.getMessage());
Log.warn(e);
throw new RuntimeException(e);
}
} else {
break;
}
long endReq = System.currentTimeMillis();
nReqTmp++;
if (nReqTmp == REQ_LIMIT) {
long reqSessionDuration = endReq - startReqTmp;
if (reqSessionDuration <= 1000) {
Log
.info(
"\nreqSessionDuration: "
+ reqSessionDuration
+ " nReqTmp: "
+ nReqTmp
+ " wait ....");
Thread.sleep(1000 - reqSessionDuration);
} else {
nReqTmp = 0;
startReqTmp = System.currentTimeMillis();
}
}
if ((parsedRecordsCounter % RECORD_PARSED_COUNTER_LOG_INTERVAL) == 0) {
Log
.info(
"Current parsed: "
+ parsedRecordsCounter
+ " downloaded: "
+ downloadedRecordsCounter
+ " saved: "
+ savedRecordsCounter);
if (REQ_MAX_TEST != -1 && parsedRecordsCounter > REQ_MAX_TEST) {
break;
}
}
}
long endReq = System.currentTimeMillis();
nReqTmp++;
if (nReqTmp == REQ_LIMIT) {
long reqSessionDuration = endReq - startReqTmp;
if (reqSessionDuration <= 1000) {
Log
.warn(
"\nreqSessionDuration: "
+ reqSessionDuration
+ " nReqTmp: "
+ nReqTmp
+ " wait ....");
Thread.sleep(1000 - reqSessionDuration);
} else {
nReqTmp = 0;
startReqTmp = System.currentTimeMillis();
}
}
// if (parsedRecordsCounter > REQ_MAX_TEST) {
// break;
// }
if ((parsedRecordsCounter % RECORD_PARSED_COUNTER_LOG_INTERVAL) == 0) {
Log
.info(
"Current parsed: "
+ parsedRecordsCounter
+ " downloaded: "
+ downloadedRecordsCounter
+ " saved: "
+ savedRecordsCounter);
// if (parsedRecordsCounter > REQ_MAX_TEST) {
// break;
// }
}
long endDownload = System.currentTimeMillis();
long downloadTime = endDownload - startDownload;
Log.info("Download time: " + ((downloadTime / 1000) / 60) + " minutes");
}
long endDownload = System.currentTimeMillis();
long downloadTime = endDownload - startDownload;
Log.info("Download time: " + ((downloadTime / 1000) / 60) + " minutes");
}
}
lambdaFileStream.close();
Log.info("Download started at: " + new Date(startDownload).toString());
Log.info("Download ended at: " + new Date(System.currentTimeMillis()).toString());
Log.info("Parsed Records Counter: " + parsedRecordsCounter);
Log.info("Downloaded Records Counter: " + downloadedRecordsCounter);
Log.info("Saved Records Counter: " + savedRecordsCounter);
@ -185,7 +190,7 @@ public class OrcidDownloader extends OrcidDSManager {
token = parser.get("token");
}
private boolean isModified(String orcidId, String modifiedDate) {
public boolean isModified(String orcidId, String modifiedDate) {
Date modifiedDateDt = null;
Date lastUpdateDt = null;
try {
@ -195,7 +200,7 @@ public class OrcidDownloader extends OrcidDSManager {
modifiedDateDt = new SimpleDateFormat(DATE_FORMAT).parse(modifiedDate);
lastUpdateDt = new SimpleDateFormat(DATE_FORMAT).parse(lastUpdate);
} catch (Exception e) {
Log.warn("[" + orcidId + "] Parsing date: ", e.getMessage());
Log.info("[" + orcidId + "] Parsing date: ", e.getMessage());
return true;
}
return modifiedDateDt.after(lastUpdateDt);

View File

@ -0,0 +1,166 @@
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;
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 eu.dnetlib.dhp.application.ArgumentApplicationParser;
import eu.dnetlib.doiboost.orcid.model.DownloadedRecordData;
import scala.Tuple2;
public class SparkDownloadOrcidAuthors {
static Logger logger = LoggerFactory.getLogger(SparkDownloadOrcidAuthors.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(
SparkDownloadOrcidAuthors.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");
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("parsedRecords");
LongAccumulator modifiedRecordsAcc = spark.sparkContext().longAccumulator("modifiedRecords");
LongAccumulator downloadedRecordsAcc = spark.sparkContext().longAccumulator("downloadedRecords");
logger.info("Retrieving data from lamda sequence file");
JavaPairRDD<Text, Text> lamdaFileRDD = sc
.sequenceFile(workingPath + lambdaFileName, Text.class, Text.class);
logger.info("Data retrieved: " + lamdaFileRDD.count());
Function<Tuple2<Text, Text>, Boolean> isModifiedAfterFilter = data -> {
String orcidId = data._1().toString();
String lastModifiedDate = data._2().toString();
parsedRecordsAcc.add(1);
if (isModified(orcidId, lastModifiedDate)) {
modifiedRecordsAcc.add(1);
return true;
}
return false;
};
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 + "/record");
httpGet.addHeader("Accept", "application/vnd.orcid+xml");
httpGet.addHeader("Authorization", String.format("Bearer %s", token));
CloseableHttpResponse response = client.execute(httpGet);
int statusCode = response.getStatusLine().getStatusCode();
downloaded.setStatusCode(statusCode);
if (statusCode != 200) {
logger
.info(
"Downloading " + orcidId + " status code: "
+ response.getStatusLine().getStatusCode());
return downloaded.toTuple2();
}
downloadedRecordsAcc.add(1);
long currentDownloaded = downloadedRecordsAcc.value();
if ((currentDownloaded % 10000) == 0) {
logger.info("Current downloaded: " + currentDownloaded);
}
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 execution ...");
// List<Tuple2<Text, Text>> sampleList = lamdaFileRDD.take(500);
// JavaRDD<Tuple2<Text, Text>> sampleRDD = sc.parallelize(sampleList);
// sampleRDD
JavaPairRDD<Text, Text> authorsModifiedRDD = lamdaFileRDD
.filter(isModifiedAfterFilter);
logger.info("Authors modified count: " + authorsModifiedRDD.count());
logger.info("Start downloading ...");
authorsModifiedRDD
.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());
});
}
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);
}
}

View File

@ -0,0 +1,99 @@
package eu.dnetlib.doiboost.orcid;
import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession;
import java.io.BufferedReader;
import java.io.IOException;
import java.io.InputStreamReader;
import java.net.URI;
import java.util.Arrays;
import java.util.List;
import java.util.Optional;
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.io.IOUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.SequenceFile;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.io.compress.GzipCodec;
import org.apache.spark.SparkConf;
import org.mortbay.log.Log;
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
public class SparkGenLastModifiedSeq {
private static String hdfsServerUri;
private static String workingPath;
private static String outputPath;
private static String lambdaFileName;
public static void main(String[] args) throws IOException, Exception {
final ArgumentApplicationParser parser = new ArgumentApplicationParser(
IOUtils
.toString(
SparkGenLastModifiedSeq.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);
hdfsServerUri = parser.get("hdfsServerUri");
workingPath = parser.get("workingPath");
outputPath = parser.get("outputPath");
lambdaFileName = parser.get("lambdaFileName");
String lambdaFileUri = hdfsServerUri.concat(workingPath).concat(lambdaFileName);
SparkConf sparkConf = new SparkConf();
runWithSparkSession(
sparkConf,
isSparkSessionManaged,
spark -> {
int rowsNum = 0;
Path output = new Path(
hdfsServerUri
.concat(workingPath)
.concat(outputPath));
Path hdfsreadpath = new Path(lambdaFileUri);
Configuration conf = new Configuration();
conf.set("fs.defaultFS", hdfsServerUri.concat(workingPath));
conf.set("fs.hdfs.impl", org.apache.hadoop.hdfs.DistributedFileSystem.class.getName());
conf.set("fs.file.impl", org.apache.hadoop.fs.LocalFileSystem.class.getName());
FileSystem fs = FileSystem.get(URI.create(hdfsServerUri.concat(workingPath)), conf);
FSDataInputStream lambdaFileStream = fs.open(hdfsreadpath);
try (TarArchiveInputStream tais = new TarArchiveInputStream(
new GzipCompressorInputStream(lambdaFileStream))) {
TarArchiveEntry entry = null;
try (SequenceFile.Writer writer = SequenceFile
.createWriter(
conf,
SequenceFile.Writer.file(output),
SequenceFile.Writer.keyClass(Text.class),
SequenceFile.Writer.valueClass(Text.class),
SequenceFile.Writer.compression(SequenceFile.CompressionType.BLOCK, new GzipCodec()))) {
while ((entry = tais.getNextTarEntry()) != null) {
BufferedReader br = new BufferedReader(new InputStreamReader(tais));
String line;
while ((line = br.readLine()) != null) {
String[] values = line.split(",");
List<String> recordInfo = Arrays.asList(values);
String orcidId = recordInfo.get(0);
final Text key = new Text(orcidId);
final Text value = new Text(recordInfo.get(3));
writer.append(key, value);
rowsNum++;
}
}
}
}
Log.info("Saved rows from lamda csv tar file: " + rowsNum);
});
}
}

View File

@ -1,165 +0,0 @@
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;
import org.apache.hadoop.io.Text;
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.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.api.java.function.Function;
import org.apache.spark.sql.Encoders;
import org.apache.spark.sql.SaveMode;
import org.apache.spark.util.LongAccumulator;
import org.mortbay.log.Log;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
import eu.dnetlib.doiboost.orcid.model.DownloadedRecordData;
import scala.Tuple2;
public class SparkOrcidGenerateAuthors {
static final String DATE_FORMAT = "yyyy-MM-dd HH:mm:ss";
static final String lastUpdate = "2019-09-30 00:00:00";
public static void main(String[] args) throws IOException, Exception {
Logger logger = LoggerFactory.getLogger(SparkOrcidGenerateAuthors.class);
logger.info("[ SparkOrcidGenerateAuthors STARTED]");
final ArgumentApplicationParser parser = new ArgumentApplicationParser(
IOUtils
.toString(
SparkOrcidGenerateAuthors.class
.getResourceAsStream(
"/eu/dnetlib/dhp/doiboost/gen_orcid_authors_parameters.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 outputAuthorsPath = parser.get("outputAuthorsPath");
logger.info("outputAuthorsPath: ", outputAuthorsPath);
final String token = parser.get("token");
SparkConf conf = new SparkConf();
runWithSparkSession(
conf,
isSparkSessionManaged,
spark -> {
JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext());
LongAccumulator parsedRecordsAcc = sc.sc().longAccumulator("parsedRecords");
LongAccumulator modifiedRecordsAcc = sc.sc().longAccumulator("modifiedRecords");
LongAccumulator downloadedRecordsAcc = sc.sc().longAccumulator("downloadedRecords");
LongAccumulator alreadyDownloadedRecords = sc.sc().longAccumulator("alreadyDownloadedRecords");
JavaRDD<String> lamdaFileRDD = sc.textFile(workingPath + "lamdafiles");
JavaRDD<String> downloadedRDD = sc.textFile(workingPath + "downloaded");
Function<String, String> getOrcidIdFunction = line -> {
try {
String[] values = line.split(",");
return values[0].substring(1);
} catch (Exception e) {
return new String("");
}
};
List<String> downloadedRecords = downloadedRDD.map(getOrcidIdFunction).collect();
Function<String, Boolean> isModifiedAfterFilter = line -> {
String[] values = line.split(",");
String orcidId = values[0];
parsedRecordsAcc.add(1);
if (isModified(orcidId, values[3])) {
modifiedRecordsAcc.add(1);
return true;
}
return false;
};
Function<String, Boolean> isNotDownloadedFilter = line -> {
String[] values = line.split(",");
String orcidId = values[0];
if (downloadedRecords.contains(orcidId)) {
alreadyDownloadedRecords.add(1);
return false;
}
return true;
};
Function<String, Tuple2<String, String>> downloadRecordFunction = line -> {
String[] values = line.split(",");
String orcidId = values[0];
String modifiedDate = values[3];
return downloadRecord(orcidId, modifiedDate, token, downloadedRecordsAcc);
};
lamdaFileRDD
.filter(isModifiedAfterFilter)
.filter(isNotDownloadedFilter)
.map(downloadRecordFunction)
.rdd()
.saveAsTextFile(workingPath.concat(outputAuthorsPath));
});
}
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) {
Log.warn("[" + orcidId + "] Parsing date: ", e.getMessage());
return true;
}
return modifiedDateDt.after(lastUpdateDt);
}
private static Tuple2<String, String> downloadRecord(String orcidId, String modifiedDate, String token,
LongAccumulator downloadedRecordsAcc) {
final DownloadedRecordData data = new DownloadedRecordData();
data.setOrcidId(orcidId);
data.setModifiedDate(modifiedDate);
try (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));
CloseableHttpResponse response = client.execute(httpGet);
int statusCode = response.getStatusLine().getStatusCode();
data.setStatusCode(statusCode);
if (statusCode != 200) {
Log
.warn(
"Downloading " + orcidId + " status code: " + response.getStatusLine().getStatusCode());
return data.toTuple2();
}
downloadedRecordsAcc.add(1);
data
.setCompressedData(
ArgumentApplicationParser.compressArgument(IOUtils.toString(response.getEntity().getContent())));
} catch (Throwable e) {
Log.warn("Downloading " + orcidId, e.getMessage());
data.setErrorMessage(e.getMessage());
return data.toTuple2();
}
return data.toTuple2();
}
}

View File

@ -1,50 +0,0 @@
package eu.dnetlib.doiboost.orcid;
import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession;
import java.io.IOException;
import java.util.Optional;
import org.apache.commons.io.IOUtils;
import org.apache.spark.SparkConf;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
public class SparkPartitionLambdaFile {
public static void main(String[] args) throws IOException, Exception {
Logger logger = LoggerFactory.getLogger(SparkOrcidGenerateAuthors.class);
final ArgumentApplicationParser parser = new ArgumentApplicationParser(
IOUtils
.toString(
SparkOrcidGenerateAuthors.class
.getResourceAsStream(
"/eu/dnetlib/dhp/doiboost/gen_orcid_authors_parameters.json")));
parser.parseArgument(args);
Boolean isSparkSessionManaged = Optional
.ofNullable(parser.get("isSparkSessionManaged"))
.map(Boolean::valueOf)
.orElse(Boolean.TRUE);
final String workingPath = parser.get("workingPath");
SparkConf conf = new SparkConf();
runWithSparkSession(
conf,
isSparkSessionManaged,
spark -> {
JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext());
JavaRDD<String> lamdaFileRDD = sc.textFile(workingPath + "last_modified.csv");
lamdaFileRDD
.repartition(20)
.saveAsTextFile(workingPath.concat("lamdafiles"));
});
}
}

View File

@ -3,8 +3,6 @@ package eu.dnetlib.doiboost.orcid.model;
import java.io.Serializable;
import org.apache.hadoop.io.Text;
import com.google.gson.JsonObject;
import scala.Tuple2;
@ -12,7 +10,7 @@ import scala.Tuple2;
public class DownloadedRecordData implements Serializable {
private String orcidId;
private String modifiedDate;
private String lastModifiedDate;
private String statusCode;
private String compressedData;
private String errorMessage;
@ -20,7 +18,7 @@ public class DownloadedRecordData implements Serializable {
public Tuple2<String, String> toTuple2() {
JsonObject data = new JsonObject();
data.addProperty("statusCode", getStatusCode());
data.addProperty("modifiedDate", getModifiedDate());
data.addProperty("lastModifiedDate", getLastModifiedDate());
if (getCompressedData() != null) {
data.addProperty("compressedData", getCompressedData());
}
@ -66,11 +64,11 @@ public class DownloadedRecordData implements Serializable {
this.compressedData = compressedData;
}
public String getModifiedDate() {
return modifiedDate;
public String getLastModifiedDate() {
return lastModifiedDate;
}
public void setModifiedDate(String modifiedDate) {
this.modifiedDate = modifiedDate;
public void setLastModifiedDate(String lastModifiedDate) {
this.lastModifiedDate = lastModifiedDate;
}
}

View File

@ -1,4 +0,0 @@
[{"paramName":"w", "paramLongName":"workingPath", "paramDescription": "the working path", "paramRequired": true},
{"paramName":"t", "paramLongName":"token", "paramDescription": "token to grant access", "paramRequired": true},
{"paramName":"o", "paramLongName":"outputAuthorsPath", "paramDescription": "the relative folder of the sequencial file to write the authors data", "paramRequired": true}
]

View File

@ -1,22 +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.use.system.libpath</name>
<value>true</value>
</property>
<property>
<name>oozie.action.sharelib.for.spark</name>
<value>spark2</value>
</property>
</configuration>

View File

@ -1,83 +0,0 @@
<workflow-app name="Gen Orcid Authors" xmlns="uri:oozie:workflow:0.5">
<parameters>
<property>
<name>workingPath</name>
<description>the working dir base path</description>
</property>
<property>
<name>token</name>
<description>access token</description>
</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>
<property>
<name>outputPath</name>
<description>the working dir base path</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}/authors'/>
</fs>
<ok to="Gen_Orcid_Authors"/>
<error to="Kill"/>
</action>
<action name="Split_Lambda_File">
<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>Split_Lambda_File</name>
<class>eu.dnetlib.doiboost.orcid.SparkPartitionLambdaFile</class>
<jar>dhp-doiboost-1.2.1-SNAPSHOT.jar</jar>
<spark-opts>--num-executors 24 --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>authors/</arg>
<arg>-t</arg><arg>${token}</arg>
</spark>
<ok to="End"/>
<error to="Kill"/>
</action>
<action name="Gen_Orcid_Authors">
<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_Orcid_Authors</name>
<class>eu.dnetlib.doiboost.orcid.SparkOrcidGenerateAuthors</class>
<jar>dhp-doiboost-1.2.1-SNAPSHOT.jar</jar>
<spark-opts>--num-executors 20 --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>authors/</arg>
<arg>-t</arg><arg>${token}</arg>
</spark>
<ok to="End"/>
<error to="Kill"/>
</action>
<end name="End"/>
</workflow-app>

View File

@ -14,9 +14,63 @@
</value>
<description>the shell command that downloads the lambda file from orcid containing last orcid update informations</description>
</property>
<property>
<name>sparkExecutorNumber</name>
<value>20</value>
</property>
<property>
<name>sparkDriverMemory</name>
<value>7G</value>
<description>memory for driver process</description>
</property>
<property>
<name>sparkExecutorMemory</name>
<value>2G</value>
<description>memory for individual executor</description>
</property>
<property>
<name>sparkExecutorCores</name>
<value>1</value>
<description>number of cores used by single executor</description>
</property>
<property>
<name>spark2MaxExecutors</name>
<value>20</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>
</parameters>
<start to="ResetWorkingPath"/>
<global>
<configuration>
<property>
<name>oozie.action.sharelib.for.spark</name>
<value>${oozieActionShareLibForSpark2}</value>
</property>
</configuration>
</global>
<start to="DownloadOrcidAuthors"/>
<kill name="Kill">
@ -26,6 +80,7 @@
<action name="ResetWorkingPath">
<fs>
<delete path='${workingPath}/downloads'/>
<delete path='${workingPath}/last_modified.csv.tar'/>
<mkdir path='${workingPath}/downloads'/>
</fs>
<ok to="DownloadLambdaFile"/>
@ -41,24 +96,77 @@
<argument>${shell_cmd}</argument>
<capture-output/>
</shell>
<ok to="End"/>
<ok to="DownloadUpdatedXMLAuthors"/>
<error to="Kill"/>
</action>
<action name="DownloadOrcidData">
<action name="DownloadUpdatedXMLAuthors">
<java>
<job-tracker>${jobTracker}</job-tracker>
<name-node>${nameNode}</name-node>
<main-class>eu.dnetlib.doiboost.orcid.OrcidDownloader</main-class>
<arg>-d</arg><arg>${workingPathOrcid}/</arg>
<arg>-w</arg><arg>${workingPath}/</arg>
<arg>-n</arg><arg>${nameNode}</arg>
<arg>-f</arg><arg>last_modified.csv</arg>
<arg>-o</arg><arg>download/</arg>
<arg>-f</arg><arg>last_modified.csv.tar</arg>
<arg>-o</arg><arg>downloads/</arg>
<arg>-t</arg><arg>${token}</arg>
</java>
<ok to="End"/>
<error to="Kill"/>
</action>
<action name="GenLastModifiedSeq">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn-cluster</master>
<mode>cluster</mode>
<name>GenLastModifiedSeq</name>
<class>eu.dnetlib.doiboost.orcid.SparkGenLastModifiedSeq</class>
<jar>dhp-doiboost-${projectVersion}.jar</jar>
<spark-opts>
--executor-memory=${sparkExecutorMemory}
--executor-cores=${sparkExecutorCores}
--driver-memory=${sparkDriverMemory}
--conf spark.extraListeners=${spark2ExtraListeners}
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
</spark-opts>
<arg>-w</arg><arg>${workingPath}/</arg>
<arg>-n</arg><arg>${nameNode}</arg>
<arg>-f</arg><arg>last_modified.csv.tar</arg>
<arg>-o</arg><arg>last_modified.seq</arg>
<arg>-t</arg><arg>-</arg>
</spark>
<ok to="End"/>
<error to="Kill"/>
</action>
<action name="DownloadOrcidAuthors">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn-cluster</master>
<mode>cluster</mode>
<name>DownloadOrcidAuthors</name>
<class>eu.dnetlib.doiboost.orcid.SparkDownloadOrcidAuthors</class>
<jar>dhp-doiboost-${projectVersion}.jar</jar>
<spark-opts>
--num-executors=${sparkExecutorNumber}
--executor-memory=${sparkExecutorMemory}
--executor-cores=${sparkExecutorCores}
--driver-memory=${sparkDriverMemory}
--conf spark.extraListeners=${spark2ExtraListeners}
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
</spark-opts>
<arg>-w</arg><arg>${workingPath}/</arg>
<arg>-n</arg><arg>${nameNode}</arg>
<arg>-f</arg><arg>last_modified.seq</arg>
<arg>-o</arg><arg>downloads/updated_authors</arg>
<arg>-t</arg><arg>${token}</arg>
</spark>
<ok to="End"/>
<error to="Kill"/>
</action>
<end name="End"/>
</workflow-app>

View File

@ -5,17 +5,24 @@ import static org.junit.jupiter.api.Assertions.assertTrue;
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;
import java.util.Arrays;
import java.util.Date;
import java.util.List;
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.io.IOUtils;
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.sql.catalyst.expressions.objects.AssertNotNull;
import org.junit.jupiter.api.Test;
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
@ -37,12 +44,49 @@ public class OrcidClientTest {
// 'https://api.orcid.org/v3.0/0000-0001-7291-3210/record'
@Test
public void downloadTest() throws Exception {
String record = testDownloadRecord("0000-0001-6163-2042");
File f = new File("/tmp/downloaded_0000-0001-6163-2042.xml");
private void multipleDownloadTest() throws Exception {
int toDownload = 1;
long start = System.currentTimeMillis();
OrcidDownloader downloader = new OrcidDownloader();
TarArchiveInputStream input = new TarArchiveInputStream(
new GzipCompressorInputStream(new FileInputStream("/tmp/last_modified.csv.tar")));
TarArchiveEntry entry = input.getNextTarEntry();
BufferedReader br = null;
StringBuilder sb = new StringBuilder();
int rowNum = 0;
int entryNum = 0;
int modified = 0;
while (entry != null) {
br = new BufferedReader(new InputStreamReader(input)); // Read directly from tarInput
String line;
while ((line = br.readLine()) != null) {
String[] values = line.toString().split(",");
List<String> recordInfo = Arrays.asList(values);
String orcidId = recordInfo.get(0);
if (downloader.isModified(orcidId, recordInfo.get(3))) {
downloadTest(orcidId);
modified++;
}
rowNum++;
if (modified > toDownload) {
break;
}
}
entryNum++;
entry = input.getNextTarEntry();
}
long end = System.currentTimeMillis();
logToFile("start test: " + new Date(start).toString());
logToFile("end test: " + new Date(end).toString());
}
@Test
private void downloadTest(String orcid) throws Exception {
String record = testDownloadRecord(orcid);
String filename = "/tmp/downloaded_".concat(orcid).concat(".xml");
File f = new File(filename);
OutputStream outStream = new FileOutputStream(f);
IOUtils.write(record.getBytes(), outStream);
System.out.println("saved to tmp");
}
private String testDownloadRecord(String orcidId) throws Exception {
@ -50,7 +94,9 @@ public class OrcidClientTest {
HttpGet httpGet = new HttpGet("https://api.orcid.org/v3.0/" + orcidId + "/record");
httpGet.addHeader("Accept", "application/vnd.orcid+xml");
httpGet.addHeader("Authorization", "Bearer 78fdb232-7105-4086-8570-e153f4198e3d");
logToFile("start connection: " + new Date(System.currentTimeMillis()).toString());
CloseableHttpResponse response = client.execute(httpGet);
logToFile("end connection: " + new Date(System.currentTimeMillis()).toString());
if (response.getStatusLine().getStatusCode() != 200) {
System.out
.println("Downloading " + orcidId + " status code: " + response.getStatusLine().getStatusCode());
@ -62,7 +108,7 @@ public class OrcidClientTest {
return new String("");
}
// @Test
// @Test
private void testLambdaFileParser() throws Exception {
try (BufferedReader br = new BufferedReader(
new InputStreamReader(this.getClass().getResourceAsStream("last_modified.csv")))) {
@ -108,7 +154,7 @@ public class OrcidClientTest {
}
}
// @Test
// @Test
private void getRecordDatestamp() throws ParseException {
Date toRetrieveDateDt = new SimpleDateFormat(DATE_FORMAT).parse(toRetrieveDate);
Date toNotRetrieveDateDt = new SimpleDateFormat(DATE_FORMAT).parse(toNotRetrieveDate);
@ -126,7 +172,7 @@ public class OrcidClientTest {
System.out.println(valueDt.toString());
}
// @Test
// @Test
@Ignore
private void testModifiedDate() throws ParseException {
testDate(toRetrieveDate);
@ -134,14 +180,81 @@ public class OrcidClientTest {
testDate(shortDate);
}
// @Test
@Ignore
private void testReadBase64CompressedRecord() throws Exception {
@Test
public void testReadBase64CompressedRecord() throws Exception {
final String base64CompressedRecord = IOUtils
.toString(getClass().getResourceAsStream("0000-0001-6645-509X.compressed.base64"));
.toString(getClass().getResourceAsStream("0000-0003-3028-6161.compressed.base64"));
final String recordFromSeqFile = ArgumentApplicationParser.decompressValue(base64CompressedRecord);
System.out.println(recordFromSeqFile);
final String downloadedRecord = testDownloadRecord("0000-0001-6645-509X");
logToFile("\n\ndownloaded \n\n" + recordFromSeqFile);
final String downloadedRecord = testDownloadRecord("0000-0003-3028-6161");
assertTrue(recordFromSeqFile.equals(downloadedRecord));
}
@Test
private void lambdaFileReaderTest() throws Exception {
TarArchiveInputStream input = new TarArchiveInputStream(
new GzipCompressorInputStream(new FileInputStream("/develop/last_modified.csv.tar")));
TarArchiveEntry entry = input.getNextTarEntry();
BufferedReader br = null;
StringBuilder sb = new StringBuilder();
int rowNum = 0;
int entryNum = 0;
while (entry != null) {
br = new BufferedReader(new InputStreamReader(input)); // Read directly from tarInput
String line;
while ((line = br.readLine()) != null) {
String[] values = line.toString().split(",");
List<String> recordInfo = Arrays.asList(values);
assertTrue(recordInfo.size() == 4);
rowNum++;
if (rowNum == 1) {
assertTrue(recordInfo.get(3).equals("last_modified"));
} else if (rowNum == 2) {
assertTrue(recordInfo.get(0).equals("0000-0002-0499-7333"));
}
}
entryNum++;
assertTrue(entryNum == 1);
entry = input.getNextTarEntry();
}
}
@Test
private void lambdaFileCounterTest() throws Exception {
final String lastUpdate = "2020-09-29 00:00:00";
OrcidDownloader downloader = new OrcidDownloader();
TarArchiveInputStream input = new TarArchiveInputStream(
new GzipCompressorInputStream(new FileInputStream("/tmp/last_modified.csv.tar")));
TarArchiveEntry entry = input.getNextTarEntry();
BufferedReader br = null;
StringBuilder sb = new StringBuilder();
int rowNum = 0;
int entryNum = 0;
int modified = 0;
while (entry != null) {
br = new BufferedReader(new InputStreamReader(input)); // Read directly from tarInput
String line;
while ((line = br.readLine()) != null) {
String[] values = line.toString().split(",");
List<String> recordInfo = Arrays.asList(values);
String orcidId = recordInfo.get(0);
if (downloader.isModified(orcidId, recordInfo.get(3))) {
modified++;
}
rowNum++;
}
entryNum++;
entry = input.getNextTarEntry();
}
logToFile("rowNum: " + rowNum);
logToFile("modified: " + modified);
}
private void logToFile(String log)
throws IOException {
log = log.concat("\n");
Path path = Paths.get("/tmp/orcid_log.txt");
Files.write(path, log.getBytes(), StandardOpenOption.APPEND);
}
}

View File

@ -1 +0,0 @@
H4sIAAAAAAAAAO1a227bOBB9z1cIepd18SW24aho0wTbAgEWjRdY9I2RaJtbSdSSkhP165eURIm6kHa2SbCLNkBiWDxzhhxyZg7tbN49xZFxhIQinFyZ7sQxDZgEOETJ/sr8Y3trLU2DZiAJQYQTeGUWkJrv/IsNgQEm4bp6MVKQHa5M22E/Fvt1rcViNrfmzupP02AOErpGSQZJAqIr85Bl6dq2Hx8fJ5gEKGR/93ZCbYEQFjDMA5CV01KZNBBhEyKaoSTQW0mgxg6mbCUgg6HGrMEIK5wdILESEEO1VYsRVjGMH1i8DyhVW7WYJhqEYKKJBB8W2ADHsS4A1bhAV1uoRlfjAp2yaWG2S1YIM4AiqrbrIwXDN1g8ah3WgGblMbPWrJwPN9in6gxZKIRJhnYI6mI2BAueXZ5UGaCyrQFNVAjcQcISB+oC0oKEHQhDAqnGpga0WXRE7ABaKaZIf8j7SMHAIvtNbcVHBfLA0gSTQg2uAe0+pREuYhZK3WYJjLD6OwcRC/2pTO/AhC2F5IgCTfLVgO7ZPXVim71hFYLFEOm2tMW02UQhIAFP+pxojm0X186QvSfwiOCjbpoNSNg95JFmV/lof36MgOKc6KI3gJr+hcF+NlX9WJdgKXmqURmRE+RzdsroW+qRLrGxJYsBDe8uvs6qBAzMDphmfuO2AZePq4XY2pVspISVM1zyJCMiHIAI+jDZ2COPa4dayk2dUSL1JEdiJCCwTAErhtkBh/5d2SiskonAcGOrgEMqmj/EiPK+b4Wsq/me464sZ2l53tadrmeLtXc58ZbLry1n32IQ8QjQzIqZeGBBDAWrx7Ztbrnu1puu59P11JksPfdrE/sRm5FlRwDFMPQzkkNpjfXTIZ4Jmoqv7A49s96gxjolKAak0LN0QfU+j+7kpiowdR3SiCZRieSTVplyIWEcEUUPKEIZK85p/hChwKzJxgRYSyJvVXk+2k0abv187rWb1EGP8o1u/QlW3dZLi24lxHqPjjAp1RT1twgkRb4Z6IwO6ATfDsQoKkqs/xmBETIZ0e6GLW2H9LgVe5I2pLqNlmCmLTF120Ovq2gZe9AOa3lEK0Gl5ag0lWxZ6xAhWPSLEqJFJqhFnVB/WnuB6c59qNbG5J5+XSN44aTZ0+qlftg2eEkPWDSPecprY9Aqg2fUyZnlTLfObD2brZ3pZHm5OLNOStOUbjfaWMi47la3XM39Sh/VBqXkaWTfiWPXwFRMte7W0giMiqMvjbVkA7CKtb2yafkkmIpJ0ndaKhmn4uroZi1bF6niG2jCs2pRi1bx1kpdyyYwKg5+edESlABFP3zplOxPbk9wnnaHX9u9zC9VPjpEKZDjQAXYyooU+iFGzfwGg8+iO4Ioh77rTFzXWdnvr69v7u8nPCYTb7X0PNcZ9VNZPctRgknMjv53GBoZAQlF5Q2Wiz2zcQ8Cdu7oafct1/PmwDp1c1FiISyvSc9dOud4llMCoyrZWTHyKYx2o7Qd1PjJGTEbOYkjqJGjuOFJWqZy22XzzApwyG6qly67kCxWjnkqy+0WOSaWWe9LI1BYKAnhE1PNpj4lelqZp+XUmjpbz1szYTt3JjP38hyt3Od9raSXfVR19/TBqHBWEPHjr8192Wr8gl+RSJuzWi5nlrtyp+P3fJ2H3t1/yNS9++uoTn4eMGpsPztAvZCWd4Rrgillt/Q+XfcCoXGsAJXZkqEsOmOLK9g9K1CR9ZFdnBN+kzdu2WnNCTTuQEbQk3HNMp3VvlIXGnflZwfGDhPjI6y+FDC+wBQyJnbHMm7Ze0iMO3yElba7JTg2biIYZATzzzXSA4jwnoDYuEd7lvK0WZRmyhv71KLOb2oK9Hnn5YWam4ryVRqcytlbNznVPF690akcv1SzK/nPangq5An99W8jpIxKXSP4Gf2LlRI+CUAyFERQZJry+DZFuOyb1eeJ6pYjWxRM95fNrJlf+UQfpPPcVOsRS6nKxKebmxvjfXl+60V1x0fUyEBn9LS7rRfvP6rt64/GVlt3vnYXa8ebLJz5T6jt53ObB8OeLl2m2WZvJurP8fviav4cpz+BjF+4znzqzd3TMr5FvryMP5GBPyjjXyC/ZR+/ZPwvGd+Rzh8IQIl1jWOWVkyDf+L/PLMDATSuDyBJYGTdQ67DuYq/ZxUwg/vC+AAoq4fsyXuWtwVF1MA74+bIA/GFlwc2+BHSIgkOBCfoe1kvjC1OuYRPD4WBSi78DRq/szGu+H/p+ddqaiovb9bYVBN4veam8vj/l+6q0PwnNbu7OkOzy3bslxf3ZWNWPThpF4LC91or/va17gefq3e83v0GQZQdAkCgcZPsUQIhQcn+DW4NnbHyqwjxxaP2S0b/YmN3/tnSv/gH9+klwrUpAAA=