Solr JSON payload #399
|
@ -0,0 +1,52 @@
|
||||||
|
/*
|
||||||
|
* Copyright (c) 2024.
|
||||||
|
* SPDX-FileCopyrightText: © 2023 Consiglio Nazionale delle Ricerche
|
||||||
|
* SPDX-License-Identifier: AGPL-3.0-or-later
|
||||||
|
*/
|
||||||
|
|
||||||
|
package eu.dnetlib.dhp.oa.provision;
|
||||||
|
|
||||||
|
import java.io.StringReader;
|
||||||
|
import java.io.StringWriter;
|
||||||
|
|
||||||
|
import javax.xml.transform.Transformer;
|
||||||
|
import javax.xml.transform.TransformerException;
|
||||||
|
import javax.xml.transform.stream.StreamResult;
|
||||||
|
import javax.xml.transform.stream.StreamSource;
|
||||||
|
|
||||||
|
import eu.dnetlib.dhp.utils.saxon.SaxonTransformerFactory;
|
||||||
|
|
||||||
|
public abstract class AbstractSolrRecordTransformJob {
|
||||||
|
|
||||||
|
protected static String toIndexRecord(Transformer tr, final String xmlRecord) {
|
||||||
|
final StreamResult res = new StreamResult(new StringWriter());
|
||||||
|
try {
|
||||||
|
tr.transform(new StreamSource(new StringReader(xmlRecord)), res);
|
||||||
|
return res.getWriter().toString();
|
||||||
|
} catch (TransformerException e) {
|
||||||
|
throw new IllegalArgumentException("XPathException on record: \n" + xmlRecord, e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Creates the XSLT responsible for building the index xml records.
|
||||||
|
*
|
||||||
|
* @param format Metadata format name (DMF|TMF)
|
||||||
|
* @param xslt xslt for building the index record transformer
|
||||||
|
* @param fields the list of fields
|
||||||
|
* @return the javax.xml.transform.Transformer
|
||||||
|
* @throws TransformerException could happen
|
||||||
|
*/
|
||||||
|
protected static String getLayoutTransformer(String format, String fields, String xslt)
|
||||||
|
throws TransformerException {
|
||||||
|
|
||||||
|
final Transformer layoutTransformer = SaxonTransformerFactory.newInstance(xslt);
|
||||||
|
final StreamResult layoutToXsltXslt = new StreamResult(new StringWriter());
|
||||||
|
|
||||||
|
layoutTransformer.setParameter("format", format);
|
||||||
|
layoutTransformer.transform(new StreamSource(new StringReader(fields)), layoutToXsltXslt);
|
||||||
|
|
||||||
|
return layoutToXsltXslt.getWriter().toString();
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -62,18 +62,10 @@ public class SolrAdminApplication implements Closeable {
|
||||||
final String collection = ProvisionConstants.getCollectionName(format);
|
final String collection = ProvisionConstants.getCollectionName(format);
|
||||||
log.info("collection: {}", collection);
|
log.info("collection: {}", collection);
|
||||||
|
|
||||||
final boolean shouldIndex = Optional
|
|
||||||
.ofNullable(parser.get("shouldIndex"))
|
|
||||||
.map(Boolean::valueOf)
|
|
||||||
.orElse(false);
|
|
||||||
log.info("shouldIndex: {}", shouldIndex);
|
|
||||||
|
|
||||||
if (shouldIndex) {
|
|
||||||
try (SolrAdminApplication app = new SolrAdminApplication(zkHost)) {
|
try (SolrAdminApplication app = new SolrAdminApplication(zkHost)) {
|
||||||
app.execute(action, collection, query, commit);
|
app.execute(action, collection, query, commit);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
|
||||||
|
|
||||||
public SolrAdminApplication(String zkHost) {
|
public SolrAdminApplication(String zkHost) {
|
||||||
final ZkServers zk = ZkServers.newInstance(zkHost);
|
final ZkServers zk = ZkServers.newInstance(zkHost);
|
||||||
|
|
|
@ -0,0 +1,144 @@
|
||||||
|
|
||||||
|
package eu.dnetlib.dhp.oa.provision;
|
||||||
|
|
||||||
|
import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession;
|
||||||
|
|
||||||
|
import java.util.Optional;
|
||||||
|
|
||||||
|
import javax.xml.transform.TransformerException;
|
||||||
|
|
||||||
|
import org.apache.commons.io.IOUtils;
|
||||||
|
import org.apache.commons.lang3.StringUtils;
|
||||||
|
import org.apache.solr.common.SolrInputDocument;
|
||||||
|
import org.apache.spark.SparkConf;
|
||||||
|
import org.apache.spark.api.java.function.MapFunction;
|
||||||
|
import org.apache.spark.sql.Encoder;
|
||||||
|
import org.apache.spark.sql.Encoders;
|
||||||
|
import org.apache.spark.sql.SaveMode;
|
||||||
|
import org.apache.spark.sql.SparkSession;
|
||||||
|
import org.slf4j.Logger;
|
||||||
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
|
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
|
||||||
|
import eu.dnetlib.dhp.oa.provision.model.SerializableSolrInputDocument;
|
||||||
|
import eu.dnetlib.dhp.oa.provision.model.TupleWrapper;
|
||||||
|
import eu.dnetlib.dhp.oa.provision.utils.ISLookupClient;
|
||||||
|
import eu.dnetlib.dhp.oa.provision.utils.StreamingInputDocumentFactory;
|
||||||
|
import eu.dnetlib.dhp.utils.ISLookupClientFactory;
|
||||||
|
import eu.dnetlib.dhp.utils.saxon.SaxonTransformerFactory;
|
||||||
|
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException;
|
||||||
|
|
||||||
|
public class SolrRecordDumpJob extends AbstractSolrRecordTransformJob {
|
||||||
|
|
||||||
|
private static final Logger log = LoggerFactory.getLogger(SolrRecordDumpJob.class);
|
||||||
|
|
||||||
|
private static final Integer DEFAULT_BATCH_SIZE = 1000;
|
||||||
|
|
||||||
|
private final String inputPath;
|
||||||
|
|
||||||
|
private final String format;
|
||||||
|
|
||||||
|
private final String outputPath;
|
||||||
|
|
||||||
|
private final SparkSession spark;
|
||||||
|
|
||||||
|
public static void main(String[] args) throws Exception {
|
||||||
|
|
||||||
|
final ArgumentApplicationParser parser = new ArgumentApplicationParser(
|
||||||
|
IOUtils
|
||||||
|
.toString(
|
||||||
|
SolrRecordDumpJob.class
|
||||||
|
.getResourceAsStream(
|
||||||
|
"/eu/dnetlib/dhp/oa/provision/input_params_solr_record_dump.json")));
|
||||||
|
parser.parseArgument(args);
|
||||||
|
|
||||||
|
Boolean isSparkSessionManaged = Optional
|
||||||
|
.ofNullable(parser.get("isSparkSessionManaged"))
|
||||||
|
.map(Boolean::valueOf)
|
||||||
|
.orElse(Boolean.TRUE);
|
||||||
|
log.info("isSparkSessionManaged: {}", isSparkSessionManaged);
|
||||||
|
|
||||||
|
final String inputPath = parser.get("inputPath");
|
||||||
|
log.info("inputPath: {}", inputPath);
|
||||||
|
|
||||||
|
final String format = parser.get("format");
|
||||||
|
log.info("format: {}", format);
|
||||||
|
|
||||||
|
final String outputPath = Optional
|
||||||
|
.ofNullable(parser.get("outputPath"))
|
||||||
|
.map(StringUtils::trim)
|
||||||
|
.orElse(null);
|
||||||
|
log.info("outputPath: {}", outputPath);
|
||||||
|
|
||||||
|
final Integer batchSize = Optional
|
||||||
|
.ofNullable(parser.get("batchSize"))
|
||||||
|
.map(Integer::valueOf)
|
||||||
|
.orElse(DEFAULT_BATCH_SIZE);
|
||||||
|
log.info("batchSize: {}", batchSize);
|
||||||
|
|
||||||
|
final boolean shouldIndex = Optional
|
||||||
|
.ofNullable(parser.get("shouldIndex"))
|
||||||
|
.map(Boolean::valueOf)
|
||||||
|
.orElse(false);
|
||||||
|
log.info("shouldIndex: {}", shouldIndex);
|
||||||
|
|
||||||
|
final SparkConf conf = new SparkConf();
|
||||||
|
|
||||||
|
conf.registerKryoClasses(new Class[] {
|
||||||
|
SerializableSolrInputDocument.class
|
||||||
|
});
|
||||||
|
|
||||||
|
runWithSparkSession(
|
||||||
|
conf,
|
||||||
|
isSparkSessionManaged,
|
||||||
|
spark -> {
|
||||||
|
final String isLookupUrl = parser.get("isLookupUrl");
|
||||||
|
log.info("isLookupUrl: {}", isLookupUrl);
|
||||||
|
final ISLookupClient isLookup = new ISLookupClient(ISLookupClientFactory.getLookUpService(isLookupUrl));
|
||||||
|
new SolrRecordDumpJob(spark, inputPath, format, outputPath).run(isLookup);
|
||||||
|
});
|
||||||
|
}
|
||||||
|
|
||||||
|
public SolrRecordDumpJob(SparkSession spark, String inputPath, String format, String outputPath) {
|
||||||
|
this.spark = spark;
|
||||||
|
this.inputPath = inputPath;
|
||||||
|
this.format = format;
|
||||||
|
this.outputPath = outputPath;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void run(ISLookupClient isLookup) throws ISLookUpException, TransformerException {
|
||||||
|
final String fields = isLookup.getLayoutSource(format);
|
||||||
|
log.info("fields: {}", fields);
|
||||||
|
|
||||||
|
final String xslt = isLookup.getLayoutTransformer();
|
||||||
|
|
||||||
|
final String dsId = isLookup.getDsId(format);
|
||||||
|
log.info("dsId: {}", dsId);
|
||||||
|
|
||||||
|
final String indexRecordXslt = getLayoutTransformer(format, fields, xslt);
|
||||||
|
log.info("indexRecordTransformer {}", indexRecordXslt);
|
||||||
|
|
||||||
|
final Encoder<TupleWrapper> encoder = Encoders.bean(TupleWrapper.class);
|
||||||
|
spark
|
||||||
|
.read()
|
||||||
|
.schema(encoder.schema())
|
||||||
|
.json(inputPath)
|
||||||
|
.as(encoder)
|
||||||
|
.map(
|
||||||
|
(MapFunction<TupleWrapper, TupleWrapper>) t -> new TupleWrapper(
|
||||||
|
toIndexRecord(SaxonTransformerFactory.newInstance(indexRecordXslt), t.getXml()),
|
||||||
|
t.getJson()),
|
||||||
|
Encoders.bean(TupleWrapper.class))
|
||||||
|
.map(
|
||||||
|
(MapFunction<TupleWrapper, SerializableSolrInputDocument>) t -> {
|
||||||
|
SolrInputDocument s = new StreamingInputDocumentFactory()
|
||||||
|
.parseDocument(t.getXml(), t.getJson());
|
||||||
|
return new SerializableSolrInputDocument(s);
|
||||||
|
},
|
||||||
|
Encoders.kryo(SerializableSolrInputDocument.class))
|
||||||
|
.write()
|
||||||
|
.mode(SaveMode.Overwrite)
|
||||||
|
.parquet(outputPath);
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -3,26 +3,18 @@ package eu.dnetlib.dhp.oa.provision;
|
||||||
|
|
||||||
import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession;
|
import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession;
|
||||||
|
|
||||||
import java.io.StringReader;
|
|
||||||
import java.io.StringWriter;
|
|
||||||
import java.text.SimpleDateFormat;
|
|
||||||
import java.util.Date;
|
|
||||||
import java.util.Optional;
|
import java.util.Optional;
|
||||||
|
|
||||||
import javax.xml.transform.Transformer;
|
|
||||||
import javax.xml.transform.TransformerException;
|
import javax.xml.transform.TransformerException;
|
||||||
import javax.xml.transform.stream.StreamResult;
|
|
||||||
import javax.xml.transform.stream.StreamSource;
|
|
||||||
|
|
||||||
import org.apache.commons.io.IOUtils;
|
import org.apache.commons.io.IOUtils;
|
||||||
import org.apache.commons.lang3.StringUtils;
|
|
||||||
import org.apache.hadoop.io.Text;
|
|
||||||
import org.apache.solr.common.SolrInputDocument;
|
import org.apache.solr.common.SolrInputDocument;
|
||||||
import org.apache.spark.SparkConf;
|
import org.apache.spark.SparkConf;
|
||||||
import org.apache.spark.api.java.JavaRDD;
|
import org.apache.spark.api.java.JavaRDD;
|
||||||
import org.apache.spark.api.java.JavaSparkContext;
|
|
||||||
import org.apache.spark.api.java.function.MapFunction;
|
import org.apache.spark.api.java.function.MapFunction;
|
||||||
import org.apache.spark.sql.*;
|
import org.apache.spark.sql.Encoder;
|
||||||
|
import org.apache.spark.sql.Encoders;
|
||||||
|
import org.apache.spark.sql.SparkSession;
|
||||||
import org.slf4j.Logger;
|
import org.slf4j.Logger;
|
||||||
import org.slf4j.LoggerFactory;
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
|
@ -36,32 +28,19 @@ import eu.dnetlib.dhp.oa.provision.utils.StreamingInputDocumentFactory;
|
||||||
import eu.dnetlib.dhp.utils.ISLookupClientFactory;
|
import eu.dnetlib.dhp.utils.ISLookupClientFactory;
|
||||||
import eu.dnetlib.dhp.utils.saxon.SaxonTransformerFactory;
|
import eu.dnetlib.dhp.utils.saxon.SaxonTransformerFactory;
|
||||||
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException;
|
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException;
|
||||||
import scala.Tuple2;
|
|
||||||
|
|
||||||
public class XmlIndexingJob {
|
public class XmlIndexingJob extends AbstractSolrRecordTransformJob {
|
||||||
|
|
||||||
private static final Logger log = LoggerFactory.getLogger(XmlIndexingJob.class);
|
private static final Logger log = LoggerFactory.getLogger(XmlIndexingJob.class);
|
||||||
|
|
||||||
public enum OutputFormat {
|
|
||||||
SOLR, HDFS
|
|
||||||
}
|
|
||||||
|
|
||||||
private static final Integer DEFAULT_BATCH_SIZE = 1000;
|
private static final Integer DEFAULT_BATCH_SIZE = 1000;
|
||||||
|
|
||||||
protected static final String DATE_FORMAT = "yyyy-MM-dd'T'hh:mm:ss'Z'";
|
|
||||||
|
|
||||||
private final String inputPath;
|
private final String inputPath;
|
||||||
|
|
||||||
private final String format;
|
private final String format;
|
||||||
|
|
||||||
private final int batchSize;
|
private final int batchSize;
|
||||||
|
|
||||||
private final OutputFormat outputFormat;
|
|
||||||
|
|
||||||
private final String outputPath;
|
|
||||||
|
|
||||||
private boolean shouldIndex;
|
|
||||||
|
|
||||||
private final SparkSession spark;
|
private final SparkSession spark;
|
||||||
|
|
||||||
public static void main(String[] args) throws Exception {
|
public static void main(String[] args) throws Exception {
|
||||||
|
@ -86,30 +65,12 @@ public class XmlIndexingJob {
|
||||||
final String format = parser.get("format");
|
final String format = parser.get("format");
|
||||||
log.info("format: {}", format);
|
log.info("format: {}", format);
|
||||||
|
|
||||||
final String outputPath = Optional
|
|
||||||
.ofNullable(parser.get("outputPath"))
|
|
||||||
.map(StringUtils::trim)
|
|
||||||
.orElse(null);
|
|
||||||
log.info("outputPath: {}", outputPath);
|
|
||||||
|
|
||||||
final Integer batchSize = Optional
|
final Integer batchSize = Optional
|
||||||
.ofNullable(parser.get("batchSize"))
|
.ofNullable(parser.get("batchSize"))
|
||||||
.map(Integer::valueOf)
|
.map(Integer::valueOf)
|
||||||
.orElse(DEFAULT_BATCH_SIZE);
|
.orElse(DEFAULT_BATCH_SIZE);
|
||||||
log.info("batchSize: {}", batchSize);
|
log.info("batchSize: {}", batchSize);
|
||||||
|
|
||||||
final OutputFormat outputFormat = Optional
|
|
||||||
.ofNullable(parser.get("outputFormat"))
|
|
||||||
.map(OutputFormat::valueOf)
|
|
||||||
.orElse(OutputFormat.SOLR);
|
|
||||||
log.info("outputFormat: {}", outputFormat);
|
|
||||||
|
|
||||||
final boolean shouldIndex = Optional
|
|
||||||
.ofNullable(parser.get("shouldIndex"))
|
|
||||||
.map(Boolean::valueOf)
|
|
||||||
.orElse(false);
|
|
||||||
log.info("shouldIndex: {}", shouldIndex);
|
|
||||||
|
|
||||||
final SparkConf conf = new SparkConf();
|
final SparkConf conf = new SparkConf();
|
||||||
|
|
||||||
conf.registerKryoClasses(new Class[] {
|
conf.registerKryoClasses(new Class[] {
|
||||||
|
@ -123,20 +84,16 @@ public class XmlIndexingJob {
|
||||||
final String isLookupUrl = parser.get("isLookupUrl");
|
final String isLookupUrl = parser.get("isLookupUrl");
|
||||||
log.info("isLookupUrl: {}", isLookupUrl);
|
log.info("isLookupUrl: {}", isLookupUrl);
|
||||||
final ISLookupClient isLookup = new ISLookupClient(ISLookupClientFactory.getLookUpService(isLookupUrl));
|
final ISLookupClient isLookup = new ISLookupClient(ISLookupClientFactory.getLookUpService(isLookupUrl));
|
||||||
new XmlIndexingJob(spark, inputPath, format, batchSize, outputFormat, shouldIndex, outputPath)
|
new XmlIndexingJob(spark, inputPath, format, batchSize)
|
||||||
.run(isLookup);
|
.run(isLookup);
|
||||||
});
|
});
|
||||||
}
|
}
|
||||||
|
|
||||||
public XmlIndexingJob(SparkSession spark, String inputPath, String format, Integer batchSize,
|
public XmlIndexingJob(SparkSession spark, String inputPath, String format, Integer batchSize) {
|
||||||
OutputFormat outputFormat, boolean shouldIndex, String outputPath) {
|
|
||||||
this.spark = spark;
|
this.spark = spark;
|
||||||
this.inputPath = inputPath;
|
this.inputPath = inputPath;
|
||||||
this.format = format;
|
this.format = format;
|
||||||
this.batchSize = batchSize;
|
this.batchSize = batchSize;
|
||||||
this.outputFormat = outputFormat;
|
|
||||||
this.shouldIndex = shouldIndex;
|
|
||||||
this.outputPath = outputPath;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
public void run(ISLookupClient isLookup) throws ISLookUpException, TransformerException {
|
public void run(ISLookupClient isLookup) throws ISLookUpException, TransformerException {
|
||||||
|
@ -148,6 +105,9 @@ public class XmlIndexingJob {
|
||||||
final String dsId = isLookup.getDsId(format);
|
final String dsId = isLookup.getDsId(format);
|
||||||
log.info("dsId: {}", dsId);
|
log.info("dsId: {}", dsId);
|
||||||
|
|
||||||
|
final String collection = ProvisionConstants.getCollectionName(format);
|
||||||
|
log.info("collection: {}", collection);
|
||||||
|
|
||||||
final String zkHost = isLookup.getZkHost();
|
final String zkHost = isLookup.getZkHost();
|
||||||
log.info("zkHost: {}", zkHost);
|
log.info("zkHost: {}", zkHost);
|
||||||
|
|
||||||
|
@ -155,7 +115,8 @@ public class XmlIndexingJob {
|
||||||
log.info("indexRecordTransformer {}", indexRecordXslt);
|
log.info("indexRecordTransformer {}", indexRecordXslt);
|
||||||
|
|
||||||
final Encoder<TupleWrapper> encoder = Encoders.bean(TupleWrapper.class);
|
final Encoder<TupleWrapper> encoder = Encoders.bean(TupleWrapper.class);
|
||||||
final Dataset<TupleWrapper> records = spark
|
|
||||||
|
JavaRDD<SolrInputDocument> docs = spark
|
||||||
.read()
|
.read()
|
||||||
.schema(encoder.schema())
|
.schema(encoder.schema())
|
||||||
.json(inputPath)
|
.json(inputPath)
|
||||||
|
@ -164,80 +125,11 @@ public class XmlIndexingJob {
|
||||||
(MapFunction<TupleWrapper, TupleWrapper>) t -> new TupleWrapper(
|
(MapFunction<TupleWrapper, TupleWrapper>) t -> new TupleWrapper(
|
||||||
toIndexRecord(SaxonTransformerFactory.newInstance(indexRecordXslt), t.getXml()),
|
toIndexRecord(SaxonTransformerFactory.newInstance(indexRecordXslt), t.getXml()),
|
||||||
t.getJson()),
|
t.getJson()),
|
||||||
Encoders.bean(TupleWrapper.class));
|
Encoders.bean(TupleWrapper.class))
|
||||||
|
|
||||||
switch (outputFormat) {
|
|
||||||
case SOLR:
|
|
||||||
if (shouldIndex) {
|
|
||||||
final String collection = ProvisionConstants.getCollectionName(format);
|
|
||||||
|
|
||||||
// SparkSolr >= 4
|
|
||||||
// com.lucidworks.spark.BatchSizeType bt = com.lucidworks.spark.BatchSizeType.NUM_DOCS;
|
|
||||||
// SolrSupport.indexDocs(zkHost, collection, batchSize, bt, docs.rdd());
|
|
||||||
// SparkSolr < 4
|
|
||||||
JavaRDD<SolrInputDocument> docs = records
|
|
||||||
.javaRDD()
|
.javaRDD()
|
||||||
.map(
|
.map(
|
||||||
t -> new StreamingInputDocumentFactory().parseDocument(t.getXml(), t.getJson()));
|
t -> new StreamingInputDocumentFactory().parseDocument(t.getXml(), t.getJson()));
|
||||||
SolrSupport.indexDocs(zkHost, collection, batchSize, docs.rdd());
|
SolrSupport.indexDocs(zkHost, collection, batchSize, docs.rdd());
|
||||||
}
|
}
|
||||||
break;
|
|
||||||
case HDFS:
|
|
||||||
records
|
|
||||||
.map(
|
|
||||||
(MapFunction<TupleWrapper, SerializableSolrInputDocument>) t -> {
|
|
||||||
SolrInputDocument s = new StreamingInputDocumentFactory()
|
|
||||||
.parseDocument(t.getXml(), t.getJson());
|
|
||||||
return new SerializableSolrInputDocument(s);
|
|
||||||
},
|
|
||||||
Encoders.kryo(SerializableSolrInputDocument.class))
|
|
||||||
.write()
|
|
||||||
.mode(SaveMode.Overwrite)
|
|
||||||
.parquet(outputPath);
|
|
||||||
break;
|
|
||||||
default:
|
|
||||||
throw new IllegalArgumentException("invalid outputFormat: " + outputFormat);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
protected static String toIndexRecord(Transformer tr, final String xmlRecord) {
|
|
||||||
final StreamResult res = new StreamResult(new StringWriter());
|
|
||||||
try {
|
|
||||||
tr.transform(new StreamSource(new StringReader(xmlRecord)), res);
|
|
||||||
return res.getWriter().toString();
|
|
||||||
} catch (TransformerException e) {
|
|
||||||
throw new IllegalArgumentException("XPathException on record: \n" + xmlRecord, e);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Creates the XSLT responsible for building the index xml records.
|
|
||||||
*
|
|
||||||
* @param format Metadata format name (DMF|TMF)
|
|
||||||
* @param xslt xslt for building the index record transformer
|
|
||||||
* @param fields the list of fields
|
|
||||||
* @return the javax.xml.transform.Transformer
|
|
||||||
* @throws TransformerException could happen
|
|
||||||
*/
|
|
||||||
protected static String getLayoutTransformer(String format, String fields, String xslt)
|
|
||||||
throws TransformerException {
|
|
||||||
|
|
||||||
final Transformer layoutTransformer = SaxonTransformerFactory.newInstance(xslt);
|
|
||||||
final StreamResult layoutToXsltXslt = new StreamResult(new StringWriter());
|
|
||||||
|
|
||||||
layoutTransformer.setParameter("format", format);
|
|
||||||
layoutTransformer.transform(new StreamSource(new StringReader(fields)), layoutToXsltXslt);
|
|
||||||
|
|
||||||
return layoutToXsltXslt.getWriter().toString();
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* method return a solr-compatible string representation of a date, used to mark all records as indexed today
|
|
||||||
*
|
|
||||||
* @return the parsed date
|
|
||||||
*/
|
|
||||||
public static String getRecordDatestamp() {
|
|
||||||
return new SimpleDateFormat(DATE_FORMAT).format(new Date());
|
|
||||||
}
|
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -63,27 +63,32 @@ public class ProvisionModelSupport {
|
||||||
}
|
}
|
||||||
|
|
||||||
public static SolrRecord transform(JoinedEntity je, ContextMapper contextMapper, VocabularyGroup vocs) {
|
public static SolrRecord transform(JoinedEntity je, ContextMapper contextMapper, VocabularyGroup vocs) {
|
||||||
SolrRecord s = new SolrRecord();
|
SolrRecord record = new SolrRecord();
|
||||||
final OafEntity e = je.getEntity();
|
final OafEntity e = je.getEntity();
|
||||||
s
|
final RecordType type = RecordType.valueOf(e.getClass().getSimpleName().toLowerCase());
|
||||||
|
final Boolean deletedbyinference = Optional
|
||||||
|
.ofNullable(e.getDataInfo())
|
||||||
|
.map(DataInfo::getDeletedbyinference)
|
||||||
|
.orElse(null);
|
||||||
|
record
|
||||||
.setHeader(
|
.setHeader(
|
||||||
SolrRecordHeader
|
SolrRecordHeader
|
||||||
.newInstance(
|
.newInstance(
|
||||||
e.getId(), e.getOriginalId(), RecordType.valueOf(e.getClass().getSimpleName().toLowerCase())));
|
e.getId(), e.getOriginalId(), type, deletedbyinference));
|
||||||
s.setCollectedfrom(asProvenance(e.getCollectedfrom()));
|
record.setCollectedfrom(asProvenance(e.getCollectedfrom()));
|
||||||
s.setContext(asContext(e.getContext(), contextMapper));
|
record.setContext(asContext(e.getContext(), contextMapper));
|
||||||
s.setPid(asPid(e.getPid()));
|
record.setPid(asPid(e.getPid()));
|
||||||
|
|
||||||
if (e instanceof eu.dnetlib.dhp.schema.oaf.Result) {
|
if (e instanceof eu.dnetlib.dhp.schema.oaf.Result) {
|
||||||
s.setResult(mapResult((eu.dnetlib.dhp.schema.oaf.Result) e));
|
record.setResult(mapResult((eu.dnetlib.dhp.schema.oaf.Result) e));
|
||||||
} else if (e instanceof eu.dnetlib.dhp.schema.oaf.Datasource) {
|
} else if (e instanceof eu.dnetlib.dhp.schema.oaf.Datasource) {
|
||||||
s.setDatasource(mapDatasource((eu.dnetlib.dhp.schema.oaf.Datasource) e));
|
record.setDatasource(mapDatasource((eu.dnetlib.dhp.schema.oaf.Datasource) e));
|
||||||
} else if (e instanceof eu.dnetlib.dhp.schema.oaf.Organization) {
|
} else if (e instanceof eu.dnetlib.dhp.schema.oaf.Organization) {
|
||||||
s.setOrganization(mapOrganization((eu.dnetlib.dhp.schema.oaf.Organization) e));
|
record.setOrganization(mapOrganization((eu.dnetlib.dhp.schema.oaf.Organization) e));
|
||||||
} else if (e instanceof eu.dnetlib.dhp.schema.oaf.Project) {
|
} else if (e instanceof eu.dnetlib.dhp.schema.oaf.Project) {
|
||||||
s.setProject(mapProject((eu.dnetlib.dhp.schema.oaf.Project) e, vocs));
|
record.setProject(mapProject((eu.dnetlib.dhp.schema.oaf.Project) e, vocs));
|
||||||
}
|
}
|
||||||
s
|
record
|
||||||
.setLinks(
|
.setLinks(
|
||||||
Optional
|
Optional
|
||||||
.ofNullable(je.getLinks())
|
.ofNullable(je.getLinks())
|
||||||
|
@ -94,7 +99,7 @@ public class ProvisionModelSupport {
|
||||||
.collect(Collectors.toList()))
|
.collect(Collectors.toList()))
|
||||||
.orElse(null));
|
.orElse(null));
|
||||||
|
|
||||||
return s;
|
return record;
|
||||||
}
|
}
|
||||||
|
|
||||||
private static RelatedRecord mapRelatedRecord(RelatedEntityWrapper rew, VocabularyGroup vocs) {
|
private static RelatedRecord mapRelatedRecord(RelatedEntityWrapper rew, VocabularyGroup vocs) {
|
||||||
|
|
|
@ -10,6 +10,7 @@ import java.io.Serializable;
|
||||||
|
|
||||||
public class TupleWrapper implements Serializable {
|
public class TupleWrapper implements Serializable {
|
||||||
|
|
||||||
|
private static final long serialVersionUID = -1418439827125577822L;
|
||||||
private String xml;
|
private String xml;
|
||||||
|
|
||||||
private String json;
|
private String json;
|
||||||
|
|
|
@ -0,0 +1,26 @@
|
||||||
|
[
|
||||||
|
{
|
||||||
|
"paramName": "is",
|
||||||
|
"paramLongName": "isLookupUrl",
|
||||||
|
"paramDescription": "URL of the isLookUp Service",
|
||||||
|
"paramRequired": true
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"paramName": "i",
|
||||||
|
"paramLongName": "inputPath",
|
||||||
|
"paramDescription": "the path of the sequence file to read the XML records",
|
||||||
|
"paramRequired": true
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"paramName": "f",
|
||||||
|
"paramLongName": "format",
|
||||||
|
"paramDescription": "MDFormat name found in the IS profile",
|
||||||
|
"paramRequired": true
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"paramName": "op",
|
||||||
|
"paramLongName": "outputPath",
|
||||||
|
"paramDescription": "path on hdfs activating an alternative output for the SolrInputDocuments",
|
||||||
|
"paramRequired": false
|
||||||
|
}
|
||||||
|
]
|
|
@ -22,23 +22,5 @@
|
||||||
"paramLongName": "batchSize",
|
"paramLongName": "batchSize",
|
||||||
"paramDescription": "size of the batch of documents sent to solr",
|
"paramDescription": "size of the batch of documents sent to solr",
|
||||||
"paramRequired": false
|
"paramRequired": false
|
||||||
},
|
|
||||||
{
|
|
||||||
"paramName": "of",
|
|
||||||
"paramLongName": "outputFormat",
|
|
||||||
"paramDescription": "decides the job output format, SOLR | HDFS",
|
|
||||||
"paramRequired": false
|
|
||||||
},
|
|
||||||
{
|
|
||||||
"paramName": "si",
|
|
||||||
"paramLongName": "shouldIndex",
|
|
||||||
"paramDescription": "should the action actually index the records?",
|
|
||||||
"paramRequired": true
|
|
||||||
},
|
|
||||||
{
|
|
||||||
"paramName": "op",
|
|
||||||
"paramLongName": "outputPath",
|
|
||||||
"paramDescription": "path on hdfs activating an alternative output for the SolrInputDocuments",
|
|
||||||
"paramRequired": false
|
|
||||||
}
|
}
|
||||||
]
|
]
|
||||||
|
|
|
@ -28,11 +28,5 @@
|
||||||
"paramLongName": "commit",
|
"paramLongName": "commit",
|
||||||
"paramDescription": "should the action be followed by a commit?",
|
"paramDescription": "should the action be followed by a commit?",
|
||||||
"paramRequired": false
|
"paramRequired": false
|
||||||
},
|
|
||||||
{
|
|
||||||
"paramName": "i",
|
|
||||||
"paramLongName": "shouldIndex",
|
|
||||||
"paramDescription": "should the action actually index the records?",
|
|
||||||
"paramRequired": true
|
|
||||||
}
|
}
|
||||||
]
|
]
|
|
@ -591,18 +591,23 @@
|
||||||
--conf spark.sql.shuffle.partitions=3840
|
--conf spark.sql.shuffle.partitions=3840
|
||||||
--conf spark.network.timeout=${sparkNetworkTimeout}
|
--conf spark.network.timeout=${sparkNetworkTimeout}
|
||||||
</spark-opts>
|
</spark-opts>
|
||||||
<!--<arg>- -inputPath</arg><arg>${workingDir}/join_entities</arg>-->
|
<arg>--inputPath</arg><arg>${workingDir}/join_entities</arg>
|
||||||
<arg>--inputPath</arg><arg>/user/claudio.atzori/data/provision/join_entities</arg>
|
<arg>--outputPath</arg><arg>${workingDir}/xml_json</arg>
|
||||||
|
|
||||||
<!--<arg>- -outputPath</arg><arg>${workingDir}/xml_json</arg>-->
|
|
||||||
<arg>--outputPath</arg><arg>/user/claudio.atzori/data/provision/xml_json_test</arg>
|
|
||||||
<arg>--contextApiBaseUrl</arg><arg>${contextApiBaseUrl}</arg>
|
<arg>--contextApiBaseUrl</arg><arg>${contextApiBaseUrl}</arg>
|
||||||
<arg>--isLookupUrl</arg><arg>${isLookupUrl}</arg>
|
<arg>--isLookupUrl</arg><arg>${isLookupUrl}</arg>
|
||||||
</spark>
|
</spark>
|
||||||
<ok to="drop_solr_collection"/>
|
<ok to="should_index"/>
|
||||||
<error to="Kill"/>
|
<error to="Kill"/>
|
||||||
</action>
|
</action>
|
||||||
|
|
||||||
|
<decision name="should_index">
|
||||||
|
<switch>
|
||||||
|
<case to="drop_solr_collection">${wf:conf('shouldIndex') eq 'true'}</case>
|
||||||
|
<case to="dump_solr_records_hdfs">${wf:conf('shouldIndex') eq 'false'}</case>
|
||||||
|
<default to="dump_solr_records_hdfs"/>
|
||||||
|
</switch>
|
||||||
|
</decision>
|
||||||
|
|
||||||
<action name="drop_solr_collection">
|
<action name="drop_solr_collection">
|
||||||
<java>
|
<java>
|
||||||
<configuration>
|
<configuration>
|
||||||
|
@ -617,7 +622,6 @@
|
||||||
<arg>--action</arg><arg>DELETE_BY_QUERY</arg>
|
<arg>--action</arg><arg>DELETE_BY_QUERY</arg>
|
||||||
<arg>--query</arg><arg>${solrDeletionQuery}</arg>
|
<arg>--query</arg><arg>${solrDeletionQuery}</arg>
|
||||||
<arg>--commit</arg><arg>true</arg>
|
<arg>--commit</arg><arg>true</arg>
|
||||||
<arg>--shouldIndex</arg><arg>${shouldIndex}</arg>
|
|
||||||
</java>
|
</java>
|
||||||
<ok to="to_solr_index"/>
|
<ok to="to_solr_index"/>
|
||||||
<error to="Kill"/>
|
<error to="Kill"/>
|
||||||
|
@ -643,17 +647,10 @@
|
||||||
--conf spark.hadoop.mapreduce.map.speculative=false
|
--conf spark.hadoop.mapreduce.map.speculative=false
|
||||||
--conf spark.hadoop.mapreduce.reduce.speculative=false
|
--conf spark.hadoop.mapreduce.reduce.speculative=false
|
||||||
</spark-opts>
|
</spark-opts>
|
||||||
<!--<arg>- -inputPath</arg><arg>${workingDir}/xml_json</arg>-->
|
<arg>--inputPath</arg><arg>${workingDir}/xml_json</arg>
|
||||||
<arg>--inputPath</arg><arg>/user/claudio.atzori/data/provision/xml_json_test</arg>
|
|
||||||
<arg>--isLookupUrl</arg><arg>${isLookupUrl}</arg>
|
<arg>--isLookupUrl</arg><arg>${isLookupUrl}</arg>
|
||||||
<arg>--format</arg><arg>${format}</arg>
|
<arg>--format</arg><arg>${format}</arg>
|
||||||
<arg>--batchSize</arg><arg>${batchSize}</arg>
|
<arg>--batchSize</arg><arg>${batchSize}</arg>
|
||||||
<arg>--outputFormat</arg><arg>${outputFormat}</arg>
|
|
||||||
|
|
||||||
<!--<arg>- -outputPath</arg><arg>${workingDir}/solr_documents</arg>-->
|
|
||||||
<arg>--outputPath</arg><arg>/user/claudio.atzori/data/provision/solr_documents</arg>
|
|
||||||
|
|
||||||
<arg>--shouldIndex</arg><arg>${shouldIndex}</arg>
|
|
||||||
</spark>
|
</spark>
|
||||||
<ok to="commit_solr_collection"/>
|
<ok to="commit_solr_collection"/>
|
||||||
<error to="Kill"/>
|
<error to="Kill"/>
|
||||||
|
@ -671,11 +668,35 @@
|
||||||
<arg>--isLookupUrl</arg><arg>${isLookupUrl}</arg>
|
<arg>--isLookupUrl</arg><arg>${isLookupUrl}</arg>
|
||||||
<arg>--format</arg><arg>${format}</arg>
|
<arg>--format</arg><arg>${format}</arg>
|
||||||
<arg>--action</arg><arg>COMMIT</arg>
|
<arg>--action</arg><arg>COMMIT</arg>
|
||||||
<arg>--shouldIndex</arg><arg>${shouldIndex}</arg>
|
|
||||||
</java>
|
</java>
|
||||||
<ok to="End"/>
|
<ok to="End"/>
|
||||||
<error to="Kill"/>
|
<error to="Kill"/>
|
||||||
</action>
|
</action>
|
||||||
|
|
||||||
|
<action name="dump_solr_records_hdfs">
|
||||||
|
<spark xmlns="uri:oozie:spark-action:0.2">
|
||||||
|
<master>yarn</master>
|
||||||
|
<mode>cluster</mode>
|
||||||
|
<name>dump_solr_records_hdfs</name>
|
||||||
|
<class>eu.dnetlib.dhp.oa.provision.SolrRecordDumpJob</class>
|
||||||
|
<jar>dhp-graph-provision-${projectVersion}.jar</jar>
|
||||||
|
<spark-opts>
|
||||||
|
--executor-cores=${sparkExecutorCoresForJoining}
|
||||||
|
--executor-memory=${sparkExecutorMemoryForJoining}
|
||||||
|
--driver-memory=${sparkDriverMemoryForJoining}
|
||||||
|
--conf spark.extraListeners=${spark2ExtraListeners}
|
||||||
|
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
|
||||||
|
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
|
||||||
|
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
|
||||||
|
</spark-opts>
|
||||||
|
<arg>--inputPath</arg><arg>${workingDir}/xml_json</arg>
|
||||||
|
<arg>--isLookupUrl</arg><arg>${isLookupUrl}</arg>
|
||||||
|
<arg>--format</arg><arg>${format}</arg>
|
||||||
|
<arg>--outputPath</arg><arg>${workingDir}/solr_documents</arg>
|
||||||
|
</spark>
|
||||||
|
<ok to="End"/>
|
||||||
|
<error to="Kill"/>
|
||||||
|
</action>
|
||||||
|
|
||||||
<end name="End"/>
|
<end name="End"/>
|
||||||
</workflow-app>
|
</workflow-app>
|
|
@ -86,8 +86,7 @@ public class SolrConfigExploreTest extends SolrExploreTest {
|
||||||
|
|
||||||
String inputPath = "src/test/resources/eu/dnetlib/dhp/oa/provision/xml";
|
String inputPath = "src/test/resources/eu/dnetlib/dhp/oa/provision/xml";
|
||||||
|
|
||||||
new XmlIndexingJob(spark, inputPath, FORMAT, batchSize, XmlIndexingJob.OutputFormat.SOLR, true, null)
|
new XmlIndexingJob(spark, inputPath, FORMAT, batchSize).run(isLookupClient);
|
||||||
.run(isLookupClient);
|
|
||||||
Assertions.assertEquals(0, miniCluster.getSolrClient().commit().getStatus());
|
Assertions.assertEquals(0, miniCluster.getSolrClient().commit().getStatus());
|
||||||
|
|
||||||
String[] queryStrings = {
|
String[] queryStrings = {
|
||||||
|
|
|
@ -95,7 +95,7 @@ public class SolrConfigTest extends SolrTest {
|
||||||
|
|
||||||
String inputPath = "src/test/resources/eu/dnetlib/dhp/oa/provision/xml";
|
String inputPath = "src/test/resources/eu/dnetlib/dhp/oa/provision/xml";
|
||||||
|
|
||||||
new XmlIndexingJob(spark, inputPath, FORMAT, batchSize, XmlIndexingJob.OutputFormat.SOLR, false, null)
|
new XmlIndexingJob(spark, inputPath, FORMAT, batchSize)
|
||||||
.run(isLookupClient);
|
.run(isLookupClient);
|
||||||
Assertions.assertEquals(0, miniCluster.getSolrClient().commit().getStatus());
|
Assertions.assertEquals(0, miniCluster.getSolrClient().commit().getStatus());
|
||||||
|
|
||||||
|
|
|
@ -0,0 +1,170 @@
|
||||||
|
/*
|
||||||
|
* Copyright (c) 2024.
|
||||||
|
* SPDX-FileCopyrightText: © 2023 Consiglio Nazionale delle Ricerche
|
||||||
|
* SPDX-License-Identifier: AGPL-3.0-or-later
|
||||||
|
*/
|
||||||
|
|
||||||
|
package eu.dnetlib.dhp.oa.provision;
|
||||||
|
|
||||||
|
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||||
|
import static org.junit.jupiter.api.Assertions.assertNotNull;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.io.StringReader;
|
||||||
|
import java.nio.file.Path;
|
||||||
|
|
||||||
|
import org.apache.commons.io.FileUtils;
|
||||||
|
import org.apache.commons.io.IOUtils;
|
||||||
|
import org.apache.solr.common.SolrInputField;
|
||||||
|
import org.apache.spark.SparkConf;
|
||||||
|
import org.apache.spark.api.java.function.MapFunction;
|
||||||
|
import org.apache.spark.sql.Dataset;
|
||||||
|
import org.apache.spark.sql.Encoders;
|
||||||
|
import org.apache.spark.sql.SparkSession;
|
||||||
|
import org.dom4j.io.SAXReader;
|
||||||
|
import org.junit.jupiter.api.AfterAll;
|
||||||
|
import org.junit.jupiter.api.BeforeAll;
|
||||||
|
import org.junit.jupiter.api.BeforeEach;
|
||||||
|
import org.junit.jupiter.api.Test;
|
||||||
|
import org.junit.jupiter.api.extension.ExtendWith;
|
||||||
|
import org.junit.jupiter.api.io.TempDir;
|
||||||
|
import org.mockito.Mock;
|
||||||
|
import org.mockito.Mockito;
|
||||||
|
import org.mockito.junit.jupiter.MockitoExtension;
|
||||||
|
import org.slf4j.Logger;
|
||||||
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
|
import eu.dnetlib.dhp.oa.provision.model.SerializableSolrInputDocument;
|
||||||
|
import eu.dnetlib.dhp.oa.provision.model.TupleWrapper;
|
||||||
|
import eu.dnetlib.dhp.oa.provision.utils.ISLookupClient;
|
||||||
|
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException;
|
||||||
|
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService;
|
||||||
|
|
||||||
|
@ExtendWith(MockitoExtension.class)
|
||||||
|
class SolrRecordDumpJobTest {
|
||||||
|
|
||||||
|
protected static final Logger log = LoggerFactory.getLogger(SolrRecordDumpJobTest.class);
|
||||||
|
|
||||||
|
protected static SparkSession spark;
|
||||||
|
|
||||||
|
protected static final String FORMAT = "test";
|
||||||
|
|
||||||
|
@Mock
|
||||||
|
private ISLookUpService isLookUpService;
|
||||||
|
|
||||||
|
@Mock
|
||||||
|
private ISLookupClient isLookupClient;
|
||||||
|
|
||||||
|
@TempDir
|
||||||
|
public static Path workingDir;
|
||||||
|
|
||||||
|
@BeforeAll
|
||||||
|
public static void before() {
|
||||||
|
|
||||||
|
SparkConf conf = new SparkConf();
|
||||||
|
conf.setAppName(SolrRecordDumpJobTest.class.getSimpleName());
|
||||||
|
|
||||||
|
conf.registerKryoClasses(new Class[] {
|
||||||
|
SerializableSolrInputDocument.class
|
||||||
|
});
|
||||||
|
|
||||||
|
conf.setMaster("local[1]");
|
||||||
|
conf.set("spark.driver.host", "localhost");
|
||||||
|
conf.set("hive.metastore.local", "true");
|
||||||
|
conf.set("spark.ui.enabled", "false");
|
||||||
|
conf.set("spark.sql.warehouse.dir", workingDir.resolve("spark").toString());
|
||||||
|
|
||||||
|
spark = SparkSession
|
||||||
|
.builder()
|
||||||
|
.appName(SolrRecordDumpJobTest.class.getSimpleName())
|
||||||
|
.config(conf)
|
||||||
|
.getOrCreate();
|
||||||
|
}
|
||||||
|
|
||||||
|
@AfterAll
|
||||||
|
public static void shutDown() throws Exception {
|
||||||
|
spark.stop();
|
||||||
|
FileUtils.deleteDirectory(workingDir.toFile());
|
||||||
|
}
|
||||||
|
|
||||||
|
@BeforeEach
|
||||||
|
public void prepareMocks() throws ISLookUpException, IOException {
|
||||||
|
isLookupClient.setIsLookup(isLookUpService);
|
||||||
|
|
||||||
|
Mockito
|
||||||
|
.when(isLookupClient.getDsId(Mockito.anyString()))
|
||||||
|
.thenReturn("313f0381-23b6-466f-a0b8-c72a9679ac4b_SW5kZXhEU1Jlc291cmNlcy9JbmRleERTUmVzb3VyY2VUeXBl");
|
||||||
|
Mockito
|
||||||
|
.when(isLookupClient.getLayoutSource(Mockito.anyString()))
|
||||||
|
.thenReturn(IOUtils.toString(getClass().getResourceAsStream("fields.xml")));
|
||||||
|
Mockito
|
||||||
|
.when(isLookupClient.getLayoutTransformer())
|
||||||
|
.thenReturn(IOUtils.toString(getClass().getResourceAsStream("layoutToRecordTransformer.xsl")));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
void testXmlIndexingJob_saveOnHDFS() throws Exception {
|
||||||
|
final String ID_XPATH = "//*[local-name()='header']/*[local-name()='objIdentifier']";
|
||||||
|
|
||||||
|
String inputPath = "src/test/resources/eu/dnetlib/dhp/oa/provision/xml";
|
||||||
|
// String inputPath = "/Users/claudio/workspace/data/index";
|
||||||
|
|
||||||
|
Dataset<TupleWrapper> records = spark
|
||||||
|
.read()
|
||||||
|
.schema(Encoders.bean(TupleWrapper.class).schema())
|
||||||
|
.json(inputPath)
|
||||||
|
.as(Encoders.bean(TupleWrapper.class));
|
||||||
|
|
||||||
|
records.printSchema();
|
||||||
|
|
||||||
|
long nRecord = records.count();
|
||||||
|
log.info("found {} records", nRecord);
|
||||||
|
|
||||||
|
final Dataset<String> ids = records
|
||||||
|
.map((MapFunction<TupleWrapper, String>) TupleWrapper::getXml, Encoders.STRING())
|
||||||
|
.map(
|
||||||
|
(MapFunction<String, String>) s -> new SAXReader().read(new StringReader(s)).valueOf(ID_XPATH),
|
||||||
|
Encoders.STRING());
|
||||||
|
|
||||||
|
log.info("found {} ids", ids.count());
|
||||||
|
|
||||||
|
long xmlIdUnique = ids
|
||||||
|
.distinct()
|
||||||
|
.count();
|
||||||
|
|
||||||
|
log.info("found {} unique ids", xmlIdUnique);
|
||||||
|
|
||||||
|
assertEquals(nRecord, xmlIdUnique, "IDs should be unique among input records");
|
||||||
|
|
||||||
|
final String outputPath = workingDir.resolve("outputPath").toAbsolutePath().toString();
|
||||||
|
new SolrRecordDumpJob(spark, inputPath, FORMAT, outputPath).run(isLookupClient);
|
||||||
|
|
||||||
|
final Dataset<SerializableSolrInputDocument> solrDocs = spark
|
||||||
|
.read()
|
||||||
|
.load(outputPath)
|
||||||
|
.as(Encoders.kryo(SerializableSolrInputDocument.class));
|
||||||
|
|
||||||
|
solrDocs.foreach(doc -> {
|
||||||
|
assertNotNull(doc.get("__result"));
|
||||||
|
assertNotNull(doc.get("__json"));
|
||||||
|
});
|
||||||
|
|
||||||
|
long docIdUnique = solrDocs.map((MapFunction<SerializableSolrInputDocument, String>) doc -> {
|
||||||
|
final SolrInputField id = doc.getField("__indexrecordidentifier");
|
||||||
|
return id.getFirstValue().toString();
|
||||||
|
}, Encoders.STRING())
|
||||||
|
.distinct()
|
||||||
|
.count();
|
||||||
|
assertEquals(xmlIdUnique, docIdUnique, "IDs should be unique among the output XML records");
|
||||||
|
|
||||||
|
long jsonUnique = solrDocs
|
||||||
|
.map(
|
||||||
|
(MapFunction<SerializableSolrInputDocument, String>) je -> (String) je.getField("__json").getValue(),
|
||||||
|
Encoders.STRING())
|
||||||
|
.distinct()
|
||||||
|
.count();
|
||||||
|
|
||||||
|
assertEquals(jsonUnique, docIdUnique, "IDs should be unique among the output JSON records");
|
||||||
|
|
||||||
|
}
|
||||||
|
}
|
|
@ -92,7 +92,7 @@ public abstract class SolrTest {
|
||||||
FileUtils.deleteDirectory(workingDir.toFile());
|
FileUtils.deleteDirectory(workingDir.toFile());
|
||||||
}
|
}
|
||||||
|
|
||||||
protected static NamedList<Object> createCollection(CloudSolrClient client, String name, int numShards,
|
public static NamedList<Object> createCollection(CloudSolrClient client, String name, int numShards,
|
||||||
int replicationFactor, int maxShardsPerNode, String configName) throws Exception {
|
int replicationFactor, int maxShardsPerNode, String configName) throws Exception {
|
||||||
ModifiableSolrParams modParams = new ModifiableSolrParams();
|
ModifiableSolrParams modParams = new ModifiableSolrParams();
|
||||||
modParams.set(CoreAdminParams.ACTION, CollectionParams.CollectionAction.CREATE.name());
|
modParams.set(CoreAdminParams.ACTION, CollectionParams.CollectionAction.CREATE.name());
|
||||||
|
|
|
@ -1,24 +1,21 @@
|
||||||
|
|
||||||
package eu.dnetlib.dhp.oa.provision;
|
package eu.dnetlib.dhp.oa.provision;
|
||||||
|
|
||||||
import static org.junit.jupiter.api.Assertions.*;
|
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||||
|
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.io.StringReader;
|
|
||||||
import java.net.URI;
|
import java.net.URI;
|
||||||
import java.util.Optional;
|
import java.util.Optional;
|
||||||
|
|
||||||
import org.apache.commons.io.IOUtils;
|
import org.apache.commons.io.IOUtils;
|
||||||
import org.apache.solr.client.solrj.SolrQuery;
|
import org.apache.solr.client.solrj.SolrQuery;
|
||||||
import org.apache.solr.client.solrj.response.QueryResponse;
|
import org.apache.solr.client.solrj.response.QueryResponse;
|
||||||
import org.apache.solr.common.SolrInputField;
|
|
||||||
import org.apache.solr.common.params.CommonParams;
|
import org.apache.solr.common.params.CommonParams;
|
||||||
import org.apache.spark.SparkConf;
|
import org.apache.spark.SparkConf;
|
||||||
import org.apache.spark.api.java.function.MapFunction;
|
|
||||||
import org.apache.spark.sql.Dataset;
|
import org.apache.spark.sql.Dataset;
|
||||||
import org.apache.spark.sql.Encoders;
|
import org.apache.spark.sql.Encoders;
|
||||||
import org.apache.spark.sql.SparkSession;
|
import org.apache.spark.sql.SparkSession;
|
||||||
import org.dom4j.io.SAXReader;
|
|
||||||
import org.junit.jupiter.api.AfterAll;
|
import org.junit.jupiter.api.AfterAll;
|
||||||
import org.junit.jupiter.api.BeforeAll;
|
import org.junit.jupiter.api.BeforeAll;
|
||||||
import org.junit.jupiter.api.BeforeEach;
|
import org.junit.jupiter.api.BeforeEach;
|
||||||
|
@ -106,8 +103,7 @@ public class XmlIndexingJobTest extends SolrTest {
|
||||||
|
|
||||||
long nRecord = records.count();
|
long nRecord = records.count();
|
||||||
|
|
||||||
new XmlIndexingJob(spark, inputPath, FORMAT, batchSize, XmlIndexingJob.OutputFormat.SOLR, true, null)
|
new XmlIndexingJob(spark, inputPath, FORMAT, batchSize).run(isLookupClient);
|
||||||
.run(isLookupClient);
|
|
||||||
|
|
||||||
assertEquals(0, miniCluster.getSolrClient().commit().getStatus());
|
assertEquals(0, miniCluster.getSolrClient().commit().getStatus());
|
||||||
|
|
||||||
|
@ -164,71 +160,4 @@ public class XmlIndexingJobTest extends SolrTest {
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
|
||||||
void testXmlIndexingJob_saveOnHDFS() throws Exception {
|
|
||||||
final String ID_XPATH = "//*[local-name()='header']/*[local-name()='objIdentifier']";
|
|
||||||
|
|
||||||
String inputPath = "src/test/resources/eu/dnetlib/dhp/oa/provision/xml";
|
|
||||||
// String inputPath = "/Users/claudio/workspace/data/index";
|
|
||||||
|
|
||||||
Dataset<TupleWrapper> records = spark
|
|
||||||
.read()
|
|
||||||
.schema(Encoders.bean(TupleWrapper.class).schema())
|
|
||||||
.json(inputPath)
|
|
||||||
.as(Encoders.bean(TupleWrapper.class));
|
|
||||||
|
|
||||||
records.printSchema();
|
|
||||||
|
|
||||||
long nRecord = records.count();
|
|
||||||
log.info("found {} records", nRecord);
|
|
||||||
|
|
||||||
final Dataset<String> ids = records
|
|
||||||
.map((MapFunction<TupleWrapper, String>) TupleWrapper::getXml, Encoders.STRING())
|
|
||||||
.map(
|
|
||||||
(MapFunction<String, String>) s -> new SAXReader().read(new StringReader(s)).valueOf(ID_XPATH),
|
|
||||||
Encoders.STRING());
|
|
||||||
|
|
||||||
log.info("found {} ids", ids.count());
|
|
||||||
|
|
||||||
long xmlIdUnique = ids
|
|
||||||
.distinct()
|
|
||||||
.count();
|
|
||||||
|
|
||||||
log.info("found {} unique ids", xmlIdUnique);
|
|
||||||
|
|
||||||
assertEquals(nRecord, xmlIdUnique, "IDs should be unique among input records");
|
|
||||||
|
|
||||||
final String outputPath = workingDir.resolve("outputPath").toAbsolutePath().toString();
|
|
||||||
new XmlIndexingJob(spark, inputPath, FORMAT, batchSize, XmlIndexingJob.OutputFormat.HDFS, false, outputPath)
|
|
||||||
.run(isLookupClient);
|
|
||||||
|
|
||||||
final Dataset<SerializableSolrInputDocument> solrDocs = spark
|
|
||||||
.read()
|
|
||||||
.load(outputPath)
|
|
||||||
.as(Encoders.kryo(SerializableSolrInputDocument.class));
|
|
||||||
|
|
||||||
solrDocs.foreach(doc -> {
|
|
||||||
assertNotNull(doc.get("__result"));
|
|
||||||
assertNotNull(doc.get("__json"));
|
|
||||||
});
|
|
||||||
|
|
||||||
long docIdUnique = solrDocs.map((MapFunction<SerializableSolrInputDocument, String>) doc -> {
|
|
||||||
final SolrInputField id = doc.getField("__indexrecordidentifier");
|
|
||||||
return id.getFirstValue().toString();
|
|
||||||
}, Encoders.STRING())
|
|
||||||
.distinct()
|
|
||||||
.count();
|
|
||||||
assertEquals(xmlIdUnique, docIdUnique, "IDs should be unique among the output XML records");
|
|
||||||
|
|
||||||
long jsonUnique = solrDocs
|
|
||||||
.map(
|
|
||||||
(MapFunction<SerializableSolrInputDocument, String>) je -> (String) je.getField("__json").getValue(),
|
|
||||||
Encoders.STRING())
|
|
||||||
.distinct()
|
|
||||||
.count();
|
|
||||||
|
|
||||||
assertEquals(jsonUnique, docIdUnique, "IDs should be unique among the output JSON records");
|
|
||||||
|
|
||||||
}
|
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -202,6 +202,7 @@
|
||||||
<field name="__dsversion" type="pdate" omitNorms="true" omitTermFreqAndPositions="true" indexed="true" stored="true"/>
|
<field name="__dsversion" type="pdate" omitNorms="true" omitTermFreqAndPositions="true" indexed="true" stored="true"/>
|
||||||
<field name="__indexrecordidentifier" type="string" multiValued="false" indexed="true" required="true" stored="true"/>
|
<field name="__indexrecordidentifier" type="string" multiValued="false" indexed="true" required="true" stored="true"/>
|
||||||
<field name="__result" type="string" docValues="false" multiValued="false" indexed="false" stored="true"/>
|
<field name="__result" type="string" docValues="false" multiValued="false" indexed="false" stored="true"/>
|
||||||
|
<field name="__json" type="string" docValues="false" multiValued="false" indexed="false" stored="true"/>
|
||||||
<field name="_root_" type="string" docValues="false" indexed="true" stored="false"/>
|
<field name="_root_" type="string" docValues="false" indexed="true" stored="false"/>
|
||||||
<field name="_version_" type="long" multiValued="false" indexed="true" stored="true"/>
|
<field name="_version_" type="long" multiValued="false" indexed="true" stored="true"/>
|
||||||
<field name="authorid" type="string_ci" multiValued="true" indexed="true" stored="false"/>
|
<field name="authorid" type="string_ci" multiValued="true" indexed="true" stored="false"/>
|
||||||
|
@ -223,17 +224,22 @@
|
||||||
<field name="datasourcecompatibilityid" type="string" multiValued="false" indexed="true" stored="false"/>
|
<field name="datasourcecompatibilityid" type="string" multiValued="false" indexed="true" stored="false"/>
|
||||||
<field name="datasourcecompatibilityname" type="string" multiValued="false" indexed="true" stored="false"/>
|
<field name="datasourcecompatibilityname" type="string" multiValued="false" indexed="true" stored="false"/>
|
||||||
<field name="datasourceenglishname" type="text_common" multiValued="true" indexed="true" stored="false"/>
|
<field name="datasourceenglishname" type="text_common" multiValued="true" indexed="true" stored="false"/>
|
||||||
|
<field name="datasourcejurisdiction" type="string" docValues="false" multiValued="true" indexed="true" stored="false"/>
|
||||||
<field name="datasourceodcontenttypes" type="string" multiValued="true" indexed="true" stored="false"/>
|
<field name="datasourceodcontenttypes" type="string" multiValued="true" indexed="true" stored="false"/>
|
||||||
<field name="datasourceoddescription" type="text_common" multiValued="true" indexed="true" stored="false"/>
|
<field name="datasourceoddescription" type="text_common" multiValued="true" indexed="true" stored="false"/>
|
||||||
<field name="datasourceodlanguages" type="string" multiValued="true" indexed="true" stored="false"/>
|
<field name="datasourceodlanguages" type="string" multiValued="true" indexed="true" stored="false"/>
|
||||||
<field name="datasourceodsubjects" type="string" multiValued="true" indexed="true" stored="false"/>
|
<field name="datasourceodsubjects" type="string" multiValued="true" indexed="true" stored="false"/>
|
||||||
<field name="datasourceofficialname" type="text_common" multiValued="true" indexed="true" stored="false"/>
|
<field name="datasourceofficialname" type="text_common" multiValued="true" indexed="true" stored="false"/>
|
||||||
<field name="datasourcesubject" type="ngramtext" multiValued="true" indexed="true" stored="false"/>
|
<field name="datasourcesubject" type="ngramtext" multiValued="true" indexed="true" stored="false"/>
|
||||||
|
<field name="datasourcethematic" type="string" docValues="false" multiValued="true" indexed="true" stored="false"/>
|
||||||
<field name="datasourcetypename" type="string" multiValued="false" indexed="true" stored="false"/>
|
<field name="datasourcetypename" type="string" multiValued="false" indexed="true" stored="false"/>
|
||||||
<field name="datasourcetypeuiid" type="string" multiValued="false" indexed="true" stored="false"/>
|
<field name="datasourcetypeuiid" type="string" multiValued="false" indexed="true" stored="false"/>
|
||||||
<field name="datasourcetypeuiname" type="string" multiValued="false" indexed="true" stored="false"/>
|
<field name="datasourcetypeuiname" type="string" multiValued="false" indexed="true" stored="false"/>
|
||||||
<field name="dateofcollection" type="pdate" multiValued="false" indexed="true" stored="false"/>
|
<field name="dateofcollection" type="pdate" multiValued="false" indexed="true" stored="false"/>
|
||||||
<field name="deletedbyinference" type="string" multiValued="true" indexed="true" stored="false"/>
|
<field name="deletedbyinference" type="string" multiValued="true" indexed="true" stored="false"/>
|
||||||
|
<field name="eoscdatasourcetype" type="string" docValues="false" multiValued="true" indexed="true" stored="false"/>
|
||||||
|
<field name="eoscifguidelines" type="string" docValues="false" multiValued="true" indexed="true" stored="false"/>
|
||||||
|
<field name="eosctype" type="string" docValues="false" multiValued="true" indexed="true" stored="false"/>
|
||||||
<field name="externalrefclass" type="string" multiValued="true" indexed="true" stored="false"/>
|
<field name="externalrefclass" type="string" multiValued="true" indexed="true" stored="false"/>
|
||||||
<field name="externalrefid" type="string" multiValued="true" indexed="true" stored="false"/>
|
<field name="externalrefid" type="string" multiValued="true" indexed="true" stored="false"/>
|
||||||
<field name="externalreflabel" type="text_common" multiValued="true" indexed="true" stored="false"/>
|
<field name="externalreflabel" type="text_common" multiValued="true" indexed="true" stored="false"/>
|
||||||
|
|
Loading…
Reference in New Issue