oozie workflow for migrating the native records from the mongodb-based mdstores to the hadoop-based ones, testing phase

This commit is contained in:
Claudio Atzori 2022-04-08 16:44:12 +02:00
parent fe34928d54
commit 548886fe91
3 changed files with 200 additions and 180 deletions

View File

@ -6,6 +6,8 @@ import java.util.Arrays;
import java.util.stream.Collectors;
import org.apache.commons.io.IOUtils;
import org.apache.http.HttpStatus;
import org.apache.http.client.methods.CloseableHttpResponse;
import org.apache.http.client.methods.HttpGet;
import org.apache.http.client.methods.HttpPost;
import org.apache.http.client.methods.HttpUriRequest;
@ -65,7 +67,18 @@ public class DNetRestClient {
.map(h -> h.getName() + ":" + h.getValue())
.collect(Collectors.joining(",")));
return IOUtils.toString(client.execute(r).getEntity().getContent());
try (final CloseableHttpResponse response = client.execute(r)) {
final int statusCode = response.getStatusLine().getStatusCode();
if (HttpStatus.SC_OK != statusCode) {
final String err = IOUtils.toString(response.getEntity().getContent());
throw new IOException(String.format("got http response: %s, response body: %s", statusCode, err));
}
final String res = IOUtils.toString(response.getEntity().getContent());
log.info("got response: {}", res);
return res;
}
}
}

View File

@ -1,10 +1,18 @@
package eu.dnetlib.dhp.migration;
import com.github.sisyphsu.dateparser.DateParserUtils;
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
import eu.dnetlib.dhp.schema.mdstore.MDStoreVersion;
import eu.dnetlib.dhp.schema.mdstore.MetadataRecord;
import eu.dnetlib.dhp.schema.mdstore.Provenance;
import static eu.dnetlib.dhp.common.Constants.*;
import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession;
import static eu.dnetlib.dhp.utils.DHPUtils.*;
import java.io.ByteArrayInputStream;
import java.io.IOException;
import java.nio.charset.StandardCharsets;
import java.time.LocalDate;
import java.time.ZoneId;
import java.util.Objects;
import java.util.Optional;
import org.apache.commons.io.IOUtils;
import org.apache.hadoop.io.IntWritable;
import org.apache.hadoop.io.Text;
@ -22,132 +30,128 @@ import org.dom4j.io.SAXReader;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.ByteArrayInputStream;
import java.io.IOException;
import java.nio.charset.StandardCharsets;
import java.time.LocalDate;
import java.time.ZoneId;
import java.util.Objects;
import java.util.Optional;
import com.github.sisyphsu.dateparser.DateParserUtils;
import static eu.dnetlib.dhp.common.Constants.*;
import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession;
import static eu.dnetlib.dhp.utils.DHPUtils.*;
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
import eu.dnetlib.dhp.schema.mdstore.MDStoreVersion;
import eu.dnetlib.dhp.schema.mdstore.MetadataRecord;
import eu.dnetlib.dhp.schema.mdstore.Provenance;
public class MigrateNativeStoreSparkJob {
private static final Logger log = LoggerFactory.getLogger(MigrateNativeStoreSparkJob.class);
private static final Logger log = LoggerFactory.getLogger(MigrateNativeStoreSparkJob.class);
public static void main(String[] args) throws Exception {
public static void main(String[] args) throws Exception {
final ArgumentApplicationParser parser = new ArgumentApplicationParser(
IOUtils
.toString(
MigrateNativeStoreSparkJob.class
.getResourceAsStream(
"/eu/dnetlib/dhp/collection/migrate_native_input_parameters.json")));
parser.parseArgument(args);
final ArgumentApplicationParser parser = new ArgumentApplicationParser(
IOUtils
.toString(
MigrateNativeStoreSparkJob.class
.getResourceAsStream(
"/eu/dnetlib/dhp/collection/migrate_native_input_parameters.json")));
parser.parseArgument(args);
final String encoding = parser.get("encoding");
log.info("encoding is {}", encoding);
final String encoding = parser.get("encoding");
log.info("encoding is {}", encoding);
String mdStoreVersion = parser.get("mdStoreVersion");
log.info("mdStoreVersion is {}", mdStoreVersion);
String mdStoreVersion = parser.get("mdStoreVersion");
log.info("mdStoreVersion is {}", mdStoreVersion);
final String datasourceId = parser.get("datasourceId");
log.info("datasourceId is {}", datasourceId);
final String datasourceId = parser.get("datasourceId");
log.info("datasourceId is {}", datasourceId);
final String datasourceName = parser.get("datasourceName");
log.info("datasourceName is {}", datasourceName);
final String datasourceName = parser.get("datasourceName");
log.info("datasourceName is {}", datasourceName);
final String nsPrefix = parser.get("nsPrefix");
log.info("nsPrefix is {}", nsPrefix);
final String nsPrefix = parser.get("nsPrefix");
log.info("nsPrefix is {}", nsPrefix);
final Provenance provenance = new Provenance(datasourceId, datasourceName, nsPrefix);
final Provenance provenance = new Provenance(datasourceId, datasourceName, nsPrefix);
final MDStoreVersion currentVersion = MAPPER.readValue(mdStoreVersion, MDStoreVersion.class);
final MDStoreVersion currentVersion = MAPPER.readValue(mdStoreVersion, MDStoreVersion.class);
Boolean isSparkSessionManaged = Optional
.ofNullable(parser.get("isSparkSessionManaged"))
.map(Boolean::valueOf)
.orElse(Boolean.TRUE);
log.info("isSparkSessionManaged: {}", isSparkSessionManaged);
Boolean isSparkSessionManaged = Optional
.ofNullable(parser.get("isSparkSessionManaged"))
.map(Boolean::valueOf)
.orElse(Boolean.TRUE);
log.info("isSparkSessionManaged: {}", isSparkSessionManaged);
SparkConf conf = new SparkConf();
runWithSparkSession(
conf,
isSparkSessionManaged,
spark -> migrateNativeMDStore(
spark, provenance, encoding, currentVersion));
}
SparkConf conf = new SparkConf();
runWithSparkSession(
conf,
isSparkSessionManaged,
spark -> migrateNativeMDStore(
spark, provenance, encoding, currentVersion));
}
private static void migrateNativeMDStore(SparkSession spark,
Provenance provenance,
String encoding,
MDStoreVersion currentVersion) throws IOException {
final JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext());
private static void migrateNativeMDStore(SparkSession spark,
Provenance provenance,
String encoding,
MDStoreVersion currentVersion) throws IOException {
final JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext());
final LongAccumulator totalItems = sc.sc().longAccumulator(CONTENT_TOTALITEMS);
final LongAccumulator invalidRecords = sc.sc().longAccumulator(CONTENT_INVALIDRECORDS);
final LongAccumulator totalItems = sc.sc().longAccumulator(CONTENT_TOTALITEMS);
final LongAccumulator invalidRecords = sc.sc().longAccumulator(CONTENT_INVALIDRECORDS);
final String seqFilePath = currentVersion.getHdfsPath() + SEQUENCE_FILE_NAME;
final String seqFilePath = currentVersion.getHdfsPath() + SEQUENCE_FILE_NAME;
final JavaRDD<MetadataRecord> nativeStore = sc
.sequenceFile(seqFilePath, IntWritable.class, Text.class)
.map(
item -> parseRecord(
item._2().toString(),
encoding,
provenance,
totalItems,
invalidRecords))
.filter(Objects::nonNull)
.distinct();
final JavaRDD<MetadataRecord> nativeStore = sc
.sequenceFile(seqFilePath, IntWritable.class, Text.class)
.map(
item -> parseRecord(
item._2().toString(),
encoding,
provenance,
totalItems,
invalidRecords))
.filter(Objects::nonNull)
.distinct();
final Encoder<MetadataRecord> encoder = Encoders.bean(MetadataRecord.class);
final Dataset<MetadataRecord> mdstore = spark.createDataset(nativeStore.rdd(), encoder);
final Encoder<MetadataRecord> encoder = Encoders.bean(MetadataRecord.class);
final Dataset<MetadataRecord> mdstore = spark.createDataset(nativeStore.rdd(), encoder);
final String targetPath = currentVersion.getHdfsPath() + MDSTORE_DATA_PATH;
final String targetPath = currentVersion.getHdfsPath() + MDSTORE_DATA_PATH;
saveDataset(mdstore, targetPath);
saveDataset(mdstore, targetPath);
final Long total = spark.read().load(targetPath).count();
log.info("migrated {} records for datasource '{}'", total, provenance.getDatasourceName());
final Long total = spark.read().load(targetPath).count();
log.info("migrated {} records for datasource '{}'", total, provenance.getDatasourceName());
writeHdfsFile(
spark.sparkContext().hadoopConfiguration(), total.toString(),
currentVersion.getHdfsPath() + MDSTORE_SIZE_PATH);
}
writeHdfsFile(
spark.sparkContext().hadoopConfiguration(), total.toString(),
currentVersion.getHdfsPath() + MDSTORE_SIZE_PATH);
}
public static MetadataRecord parseRecord(
final String input,
final String encoding,
final Provenance provenance,
final LongAccumulator totalItems,
final LongAccumulator invalidRecords) {
public static MetadataRecord parseRecord(
final String input,
final String encoding,
final Provenance provenance,
final LongAccumulator totalItems,
final LongAccumulator invalidRecords) {
if (totalItems != null)
totalItems.add(1);
try {
SAXReader reader = new SAXReader();
reader.setFeature("http://apache.org/xml/features/disallow-doctype-decl", true);
Document document = reader.read(new ByteArrayInputStream(input.getBytes(StandardCharsets.UTF_8)));
document.normalize();
String id = document.valueOf("//dri:objIdentifier/text()");
String dateOfCollection = document.valueOf("//dri:dateOfCollection/text()");
final LocalDate date = DateParserUtils
.parseDate(dateOfCollection.trim())
.toInstant()
.atZone(ZoneId.systemDefault())
.toLocalDate();
if (totalItems != null)
totalItems.add(1);
try {
SAXReader reader = new SAXReader();
reader.setFeature("http://apache.org/xml/features/disallow-doctype-decl", true);
Document document = reader.read(new ByteArrayInputStream(input.getBytes(StandardCharsets.UTF_8)));
document.normalize();
String id = document.valueOf("//dri:objIdentifier/text()");
String dateOfCollection = document.valueOf("//dri:dateOfCollection/text()");
final LocalDate date = DateParserUtils
.parseDate(dateOfCollection.trim())
.toInstant()
.atZone(ZoneId.systemDefault())
.toLocalDate();
final Node nativeRecord = document.selectSingleNode("/*[local-name() = 'record']/*[local-name() = 'metadata']/*");
final Node nativeRecord = document
.selectSingleNode("/*[local-name() = 'record']/*[local-name() = 'metadata']/*");
return new MetadataRecord(id, encoding, provenance, nativeRecord.asXML(), date.toEpochDay());
} catch (Throwable e) {
invalidRecords.add(1);
return null;
}
}
return new MetadataRecord(id, encoding, provenance, nativeRecord.asXML(), date.toEpochDay());
} catch (Throwable e) {
invalidRecords.add(1);
return null;
}
}
}

View File

@ -1,6 +1,26 @@
package eu.dnetlib.dhp.migration;
import static eu.dnetlib.dhp.common.Constants.DNET_MESSAGE_MGR_URL;
import static eu.dnetlib.dhp.utils.DHPUtils.MAPPER;
import static eu.dnetlib.dhp.utils.DHPUtils.getHadoopConfiguration;
import java.io.IOException;
import java.util.HashMap;
import java.util.Optional;
import java.util.Spliterator;
import java.util.Spliterators;
import java.util.stream.StreamSupport;
import org.apache.commons.cli.ParseException;
import org.apache.commons.io.IOUtils;
import org.apache.hadoop.fs.FileSystem;
import org.bson.Document;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import com.mongodb.client.MongoCollection;
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
import eu.dnetlib.dhp.collection.ApiDescriptor;
import eu.dnetlib.dhp.collection.CollectorWorker;
@ -12,102 +32,85 @@ import eu.dnetlib.dhp.common.collection.CollectorException;
import eu.dnetlib.dhp.common.collection.HttpClientParams;
import eu.dnetlib.dhp.message.MessageSender;
import eu.dnetlib.dhp.schema.mdstore.MDStoreVersion;
import org.apache.commons.cli.ParseException;
import org.apache.commons.io.IOUtils;
import org.apache.hadoop.fs.FileSystem;
import org.bson.Document;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.IOException;
import java.util.HashMap;
import java.util.Optional;
import java.util.Spliterator;
import java.util.Spliterators;
import java.util.stream.StreamSupport;
import static eu.dnetlib.dhp.common.Constants.DNET_MESSAGE_MGR_URL;
import static eu.dnetlib.dhp.utils.DHPUtils.MAPPER;
import static eu.dnetlib.dhp.utils.DHPUtils.getHadoopConfiguration;
public class MongoDbMDStoreCollectionReader {
private static final Logger log = LoggerFactory.getLogger(MongoDbMDStoreCollectionReader.class);
private static final Logger log = LoggerFactory.getLogger(MongoDbMDStoreCollectionReader.class);
private final FileSystem fileSystem;
private final FileSystem fileSystem;
public MongoDbMDStoreCollectionReader(FileSystem fileSystem) {
this.fileSystem = fileSystem;
}
public MongoDbMDStoreCollectionReader(FileSystem fileSystem) {
this.fileSystem = fileSystem;
}
/**
* @param args
*/
public static void main(final String[] args)
throws ParseException, IOException, UnknownCollectorPluginException, CollectorException {
/**
* @param args
*/
public static void main(final String[] args)
throws ParseException, IOException, UnknownCollectorPluginException, CollectorException {
final ArgumentApplicationParser argumentParser = new ArgumentApplicationParser(
IOUtils
.toString(
MongoDbMDStoreCollectionReader.class
.getResourceAsStream(
"/eu/dnetlib/dhp/migration/native_records_migration_input_parameter.json")));
argumentParser.parseArgument(args);
final ArgumentApplicationParser argumentParser = new ArgumentApplicationParser(
IOUtils
.toString(
MongoDbMDStoreCollectionReader.class
.getResourceAsStream(
"/eu/dnetlib/dhp/migration/native_records_migration_input_parameter.json")));
argumentParser.parseArgument(args);
log.info("Java Xmx: {}m", Runtime.getRuntime().maxMemory() / (1024 * 1024));
log.info("Java Xmx: {}m", Runtime.getRuntime().maxMemory() / (1024 * 1024));
final String hdfsuri = argumentParser.get("namenode");
log.info("hdfsURI is {}", hdfsuri);
final String hdfsuri = argumentParser.get("namenode");
log.info("hdfsURI is {}", hdfsuri);
final String mongoBaseUrl = Optional
.ofNullable(argumentParser.get("mongoDbUri"))
.orElseThrow(
() -> new IOException("missing mongodb baseUrl"));
log.info("mongoBaseUrl: {}", mongoBaseUrl);
final String mongoBaseUrl = Optional
.ofNullable(argumentParser.get("mongoDbUri"))
.orElseThrow(
() -> new IOException("missing mongodb baseUrl"));
log.info("mongoBaseUrl: {}", mongoBaseUrl);
final String dbName = Optional
.ofNullable(argumentParser.get("mongoDbName"))
.orElseThrow(() -> new IOException("missing parameter 'mongoDbName'"));
log.info("dbName: {}", dbName);
final String dbName = Optional
.ofNullable(argumentParser.get("mongoDbName"))
.orElseThrow(() -> new IOException("missing parameter 'mongoDbName'"));
log.info("dbName: {}", dbName);
final String mdId = Optional
.ofNullable(argumentParser.get("sourceMDStoreId"))
.orElseThrow(() -> new IOException("missing parameter 'sourceMDStoreId'"));
log.info("sourceMDStoreId: {}", mdId);
final String mdId = Optional
.ofNullable(argumentParser.get("sourceMDStoreId"))
.orElseThrow(() -> new IOException("missing parameter 'sourceMDStoreId'"));
log.info("sourceMDStoreId: {}", mdId);
final String mdStoreVersion = argumentParser.get("mdStoreVersion");
log.info("mdStoreVersion is {}", mdStoreVersion);
final String mdStoreVersion = argumentParser.get("mdStoreVersion");
log.info("mdStoreVersion is {}", mdStoreVersion);
final String dnetMessageManagerURL = argumentParser.get(DNET_MESSAGE_MGR_URL);
log.info("dnetMessageManagerURL is {}", dnetMessageManagerURL);
final String dnetMessageManagerURL = argumentParser.get(DNET_MESSAGE_MGR_URL);
log.info("dnetMessageManagerURL is {}", dnetMessageManagerURL);
final String workflowId = argumentParser.get("workflowId");
log.info("workflowId is {}", workflowId);
final String workflowId = argumentParser.get("workflowId");
log.info("workflowId is {}", workflowId);
final FileSystem fileSystem = FileSystem.get(getHadoopConfiguration(hdfsuri));
final FileSystem fileSystem = FileSystem.get(getHadoopConfiguration(hdfsuri));
new MongoDbMDStoreCollectionReader(fileSystem)
.run(mongoBaseUrl, dbName, mdId, mdStoreVersion, dnetMessageManagerURL, workflowId);
}
new MongoDbMDStoreCollectionReader(fileSystem)
.run(mongoBaseUrl, dbName, mdId, mdStoreVersion, dnetMessageManagerURL, workflowId);
}
protected void run(String mongoBaseUrl, String dbName, String mdId, String mdStoreVersion,
String dnetMessageManagerURL, String workflowId)
throws IOException, UnknownCollectorPluginException, CollectorException {
protected void run(String mongoBaseUrl, String dbName, String mdId, String mdStoreVersion,
String dnetMessageManagerURL, String workflowId)
throws IOException, UnknownCollectorPluginException, CollectorException {
final ApiDescriptor api = new ApiDescriptor();
api.setProtocol("other");
api.setBaseUrl(mongoBaseUrl);
api.getParams().put("other_plugin_type", "mdstore_mongodb");
api.getParams().put("mongodb_dbname", dbName);
api.getParams().put("mdstore_id", mdId);
final ApiDescriptor api = new ApiDescriptor();
api.setProtocol("other");
api.setBaseUrl(mongoBaseUrl);
api.getParams().put("other_plugin_type", "mdstore_mongodb");
api.getParams().put("mongodb_dbname", dbName);
api.getParams().put("mdstore_id", mdId);
final MDStoreVersion currentVersion = MAPPER.readValue(mdStoreVersion, MDStoreVersion.class);
final MessageSender ms = new MessageSender(dnetMessageManagerURL, workflowId);
final MDStoreVersion currentVersion = MAPPER.readValue(mdStoreVersion, MDStoreVersion.class);
final MessageSender ms = new MessageSender(dnetMessageManagerURL, workflowId);
try (AggregatorReport report = new AggregatorReport(ms)) {
try (AggregatorReport report = new AggregatorReport(ms)) {
new CollectorWorker(api, fileSystem, currentVersion, null, report).collect();
}
}
new CollectorWorker(api, fileSystem, currentVersion, null, report).collect();
}
}
}