extended XmlIndexingJob to accept an optional parameter: outputPath. When present, forces the job to write its output on the specified HDFS location

pull/58/head
Claudio Atzori 3 years ago
parent 29dcff0f34
commit d9e07a242b

@ -1,38 +1,39 @@
package eu.dnetlib.dhp.oa.provision;
import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession;
import java.io.IOException;
import java.io.StringReader;
import java.io.StringWriter;
import java.text.SimpleDateFormat;
import java.util.Date;
import java.util.Optional;
import javax.xml.transform.Transformer;
import javax.xml.transform.TransformerException;
import javax.xml.transform.stream.StreamResult;
import javax.xml.transform.stream.StreamSource;
import com.lucidworks.spark.util.SolrSupport;
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
import eu.dnetlib.dhp.oa.provision.model.SerializableSolrInputDocument;
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;
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.spark.SparkConf;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.rdd.RDD;
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 com.lucidworks.spark.util.SolrSupport;
import javax.xml.transform.Transformer;
import javax.xml.transform.TransformerException;
import javax.xml.transform.stream.StreamResult;
import javax.xml.transform.stream.StreamSource;
import java.io.IOException;
import java.io.StringReader;
import java.io.StringWriter;
import java.text.SimpleDateFormat;
import java.util.Date;
import java.util.Optional;
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
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;
import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession;
public class XmlIndexingJob {
@ -48,6 +49,8 @@ public class XmlIndexingJob {
private int batchSize;
private String outputPath;
private SparkSession spark;
public static void main(String[] args) throws Exception {
@ -72,12 +75,17 @@ public class XmlIndexingJob {
final String format = parser.get("format");
log.info("format: {}", format);
final String outputPath = Optional.ofNullable(parser.get("outputPath"))
.orElse(null);
log.info("outputPath: {}", outputPath);
final Integer batchSize = parser.getObjectMap().containsKey("batchSize")
? Integer.valueOf(parser.get("batchSize"))
: DEFAULT_BATCH_SIZE;
log.info("batchSize: {}", batchSize);
final SparkConf conf = new SparkConf();
conf.registerKryoClasses(new Class[] { SerializableSolrInputDocument.class });
runWithSparkSession(
conf,
@ -86,15 +94,16 @@ public class XmlIndexingJob {
final String isLookupUrl = parser.get("isLookupUrl");
log.info("isLookupUrl: {}", isLookupUrl);
final ISLookupClient isLookup = new ISLookupClient(ISLookupClientFactory.getLookUpService(isLookupUrl));
new XmlIndexingJob(spark, inputPath, format, batchSize).run(isLookup);
new XmlIndexingJob(spark, inputPath, format, batchSize, outputPath).run(isLookup);
});
}
public XmlIndexingJob(SparkSession spark, String inputPath, String format, Integer batchSize) {
public XmlIndexingJob(SparkSession spark, String inputPath, String format, Integer batchSize, String outputPath) {
this.spark = spark;
this.inputPath = inputPath;
this.format = format;
this.batchSize = batchSize;
this.outputPath = outputPath;
}
public void run(ISLookupClient isLookup) throws ISLookUpException, TransformerException {
@ -116,15 +125,23 @@ public class XmlIndexingJob {
final JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext());
RDD<SolrInputDocument> docs = sc
.sequenceFile(inputPath, Text.class, Text.class)
.map(t -> t._2().toString())
.map(s -> toIndexRecord(SaxonTransformerFactory.newInstance(indexRecordXslt), s))
.map(s -> new StreamingInputDocumentFactory(version, dsId).parseDocument(s))
.rdd();
final String collection = ProvisionConstants.getCollectionName(format);
SolrSupport.indexDocs(zkHost, collection, batchSize, docs);
JavaRDD<SolrInputDocument> docs = sc
.sequenceFile(inputPath, Text.class, Text.class)
.map(t -> t._2().toString())
.map(s -> toIndexRecord(SaxonTransformerFactory.newInstance(indexRecordXslt), s))
.map(s -> new StreamingInputDocumentFactory(version, dsId).parseDocument(s));
if (StringUtils.isNotBlank(outputPath)) {
spark.createDataset(
docs.map(s -> new SerializableSolrInputDocument(s)).rdd(),
Encoders.kryo(SerializableSolrInputDocument.class))
.write()
.mode(SaveMode.Overwrite)
.parquet(outputPath);
} else {
final String collection = ProvisionConstants.getCollectionName(format);
SolrSupport.indexDocs(zkHost, collection, batchSize, docs.rdd());
}
}
protected static String toIndexRecord(Transformer tr, final String record) {

@ -6,6 +6,9 @@ import org.apache.solr.common.SolrInputField;
import java.util.HashMap;
import java.util.Map;
/**
* Wrapper class needed to make the SolrInputDocument compatible with the Kryo serialization mechanism.
*/
public class SerializableSolrInputDocument extends SolrInputDocument {
public SerializableSolrInputDocument() {

@ -22,5 +22,11 @@
"paramLongName": "batchSize",
"paramDescription": "size of the batch of documents sent to solr",
"paramRequired": false
},
{
"paramName": "o",
"paramLongName": "outputPath",
"paramDescription": "path on hdfs activating an alternative output for the SolrInputDocuments",
"paramRequired": false
}
]

@ -638,6 +638,7 @@
<arg>--isLookupUrl</arg><arg>${isLookupUrl}</arg>
<arg>--format</arg><arg>${format}</arg>
<arg>--batchSize</arg><arg>${batchSize}</arg>
<arg>--outputPath</arg><arg>${outputPath}</arg>
</spark>
<ok to="commit_solr_collection"/>
<error to="Kill"/>

@ -54,13 +54,14 @@ public abstract class SolrTest {
miniCluster.uploadConfigSet(configDir.toPath(), CONFIG_NAME);
// override settings in the solrconfig include
// System.setProperty("solr.tests.maxBufferedDocs", "100000");
// System.setProperty("solr.tests.maxIndexingThreads", "-1");
// System.setProperty("solr.tests.ramBufferSizeMB", "100");
System.setProperty("solr.tests.maxBufferedDocs", "100000");
System.setProperty("solr.tests.maxIndexingThreads", "-1");
System.setProperty("solr.tests.ramBufferSizeMB", "100");
// use non-test classes so RandomizedRunner isn't necessary
// System.setProperty("solr.tests.mergeScheduler", "org.apache.lucene.index.ConcurrentMergeScheduler");
// System.setProperty("solr.directoryFactory", "solr.RAMDirectoryFactory");
System.setProperty("solr.tests.mergeScheduler", "org.apache.lucene.index.ConcurrentMergeScheduler");
System.setProperty("solr.directoryFactory", "solr.RAMDirectoryFactory");
System.setProperty("solr.lock.type", "single");
log.info(new ConfigSetAdminRequest.List().process(miniCluster.getSolrClient()).toString());
log

@ -1,26 +1,33 @@
package eu.dnetlib.dhp.oa.provision;
import java.io.IOException;
import java.net.URI;
import eu.dnetlib.dhp.oa.provision.model.SerializableSolrInputDocument;
import eu.dnetlib.dhp.oa.provision.utils.ISLookupClient;
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException;
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService;
import org.apache.commons.io.IOUtils;
import org.apache.hadoop.io.Text;
import org.apache.solr.client.solrj.SolrQuery;
import org.apache.solr.client.solrj.response.QueryResponse;
import org.apache.solr.common.SolrInputField;
import org.apache.solr.common.params.CommonParams;
import org.apache.spark.SparkConf;
import org.apache.spark.api.java.JavaPairRDD;
import org.apache.spark.api.java.JavaSparkContext;
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.*;
import org.junit.jupiter.api.extension.ExtendWith;
import org.mockito.Mock;
import org.mockito.Mockito;
import org.mockito.junit.jupiter.MockitoExtension;
import eu.dnetlib.dhp.oa.provision.utils.ISLookupClient;
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException;
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService;
import java.io.IOException;
import java.io.StringReader;
import java.net.URI;
@ExtendWith(MockitoExtension.class)
public class XmlIndexingJobTest extends SolrTest {
@ -58,6 +65,7 @@ public class XmlIndexingJobTest extends SolrTest {
SparkConf conf = new SparkConf();
conf.setAppName(XmlIndexingJobTest.class.getSimpleName());
conf.registerKryoClasses(new Class[] { SerializableSolrInputDocument.class });
conf.setMaster("local[1]");
conf.set("spark.driver.host", "localhost");
@ -78,7 +86,7 @@ public class XmlIndexingJobTest extends SolrTest {
}
@Test
public void testXmlIndexingJob() throws Exception {
public void testXmlIndexingJob_onSolr() throws Exception {
String inputPath = "src/test/resources/eu/dnetlib/dhp/oa/provision/xml";
@ -87,13 +95,46 @@ public class XmlIndexingJobTest extends SolrTest {
.sequenceFile(inputPath, Text.class, Text.class)
.count();
new XmlIndexingJob(spark, inputPath, FORMAT, batchSize).run(isLookupClient);
new XmlIndexingJob(spark, inputPath, FORMAT, batchSize, null).run(isLookupClient);
Assertions.assertEquals(0, miniCluster.getSolrClient().commit().getStatus());
QueryResponse rsp = miniCluster.getSolrClient().query(new SolrQuery().add(CommonParams.Q, "*:*"));
Assertions.assertEquals(nRecord, rsp.getResults().getNumFound());
Assertions.assertEquals(nRecord, rsp.getResults().getNumFound(),
"the number of indexed records should be equal to the number of input records");
}
@Test
public void testXmlIndexingJob_saveOnHDFS() throws Exception {
final String ID_XPATH = "//header/*[local-name()='objIdentifier']";
String inputPath = "src/test/resources/eu/dnetlib/dhp/oa/provision/xml";
final JavaPairRDD<Text, Text> xmlRecords = JavaSparkContext
.fromSparkContext(spark.sparkContext())
.sequenceFile(inputPath, Text.class, Text.class);
long nRecord = xmlRecords.count();
long xmlIdUnique = xmlRecords
.map(t -> t._2().toString())
.map(s -> new SAXReader().read(new StringReader(s)).valueOf(ID_XPATH))
.distinct().count();
Assertions.assertEquals(nRecord, xmlIdUnique, "IDs should be unique among input records");
final String outputPath = workingDir.resolve("outputPath").toAbsolutePath().toString();
new XmlIndexingJob(spark, inputPath, FORMAT, batchSize, outputPath).run(isLookupClient);
final Dataset<SerializableSolrInputDocument> solrDocs = spark.read()
.load(outputPath)
.as(Encoders.kryo(SerializableSolrInputDocument.class));
long docIdUnique = solrDocs.map((MapFunction<SerializableSolrInputDocument, String>) doc -> {
final SolrInputField id = doc.getField("__indexrecordidentifier");
return id.getFirstValue().toString();
}, Encoders.STRING())
.distinct()
.count();
Assertions.assertEquals(xmlIdUnique, docIdUnique, "IDs should be unique among the output records");
}
}

Loading…
Cancel
Save