Merge branch 'provision_indexing'

This commit is contained in:
Claudio Atzori 2020-11-19 15:59:55 +01:00
commit d48f388fb2
5 changed files with 122 additions and 74 deletions

View File

@ -1,14 +1,21 @@
package eu.dnetlib.dhp.oa.provision; package eu.dnetlib.dhp.oa.provision;
import com.lucidworks.spark.util.SolrSupport; import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession;
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
import eu.dnetlib.dhp.oa.provision.model.SerializableSolrInputDocument; import java.io.IOException;
import eu.dnetlib.dhp.oa.provision.utils.ISLookupClient; import java.io.StringReader;
import eu.dnetlib.dhp.oa.provision.utils.StreamingInputDocumentFactory; import java.io.StringWriter;
import eu.dnetlib.dhp.utils.ISLookupClientFactory; import java.text.SimpleDateFormat;
import eu.dnetlib.dhp.utils.saxon.SaxonTransformerFactory; import java.util.Date;
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException; import java.util.Optional;
import javax.swing.text.html.Option;
import javax.xml.transform.Transformer;
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.commons.lang3.StringUtils;
import org.apache.hadoop.io.Text; import org.apache.hadoop.io.Text;
@ -22,23 +29,24 @@ import org.apache.spark.sql.SparkSession;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
import javax.xml.transform.Transformer; import com.lucidworks.spark.util.SolrSupport;
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 static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession; 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;
public class XmlIndexingJob { public class XmlIndexingJob {
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'"; protected static final String DATE_FORMAT = "yyyy-MM-dd'T'hh:mm:ss'Z'";
@ -49,6 +57,8 @@ public class XmlIndexingJob {
private int batchSize; private int batchSize;
private OutputFormat outputFormat;
private String outputPath; private String outputPath;
private SparkSession spark; private SparkSession spark;
@ -75,17 +85,28 @@ 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")) final String outputPath = Optional
.orElse(null); .ofNullable(parser.get("outputPath"))
.map(StringUtils::trim)
.orElse(null);
log.info("outputPath: {}", outputPath); log.info("outputPath: {}", outputPath);
final Integer batchSize = parser.getObjectMap().containsKey("batchSize") final Integer batchSize = Optional
? Integer.valueOf(parser.get("batchSize")) .ofNullable(parser.get("batchSize"))
: DEFAULT_BATCH_SIZE; .map(Integer::valueOf)
.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 SparkConf conf = new SparkConf(); final SparkConf conf = new SparkConf();
conf.registerKryoClasses(new Class[] { SerializableSolrInputDocument.class }); conf.registerKryoClasses(new Class[] {
SerializableSolrInputDocument.class
});
runWithSparkSession( runWithSparkSession(
conf, conf,
@ -94,15 +115,18 @@ 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, outputPath).run(isLookup); new XmlIndexingJob(spark, inputPath, format, batchSize, outputFormat, outputPath).run(isLookup);
}); });
} }
public XmlIndexingJob(SparkSession spark, String inputPath, String format, Integer batchSize, String outputPath) { public XmlIndexingJob(SparkSession spark, String inputPath, String format, Integer batchSize,
OutputFormat outputFormat,
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.outputPath = outputPath; this.outputPath = outputPath;
} }
@ -126,21 +150,27 @@ public class XmlIndexingJob {
final JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext()); final JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext());
JavaRDD<SolrInputDocument> docs = sc JavaRDD<SolrInputDocument> docs = sc
.sequenceFile(inputPath, Text.class, Text.class) .sequenceFile(inputPath, Text.class, Text.class)
.map(t -> t._2().toString()) .map(t -> t._2().toString())
.map(s -> toIndexRecord(SaxonTransformerFactory.newInstance(indexRecordXslt), s)) .map(s -> toIndexRecord(SaxonTransformerFactory.newInstance(indexRecordXslt), s))
.map(s -> new StreamingInputDocumentFactory(version, dsId).parseDocument(s)); .map(s -> new StreamingInputDocumentFactory(version, dsId).parseDocument(s));
if (StringUtils.isNotBlank(outputPath)) { switch (outputFormat) {
spark.createDataset( case SOLR:
docs.map(s -> new SerializableSolrInputDocument(s)).rdd(), final String collection = ProvisionConstants.getCollectionName(format);
Encoders.kryo(SerializableSolrInputDocument.class)) SolrSupport.indexDocs(zkHost, collection, batchSize, docs.rdd());
.write() break;
.mode(SaveMode.Overwrite) case HDFS:
.parquet(outputPath); spark
} else { .createDataset(
final String collection = ProvisionConstants.getCollectionName(format); docs.map(s -> new SerializableSolrInputDocument(s)).rdd(),
SolrSupport.indexDocs(zkHost, collection, batchSize, docs.rdd()); Encoders.kryo(SerializableSolrInputDocument.class))
.write()
.mode(SaveMode.Overwrite)
.parquet(outputPath);
break;
default:
throw new IllegalArgumentException("invalid outputFormat: " + outputFormat);
} }
} }

View File

@ -1,22 +1,23 @@
package eu.dnetlib.dhp.oa.provision.model;
import org.apache.solr.common.SolrInputDocument; package eu.dnetlib.dhp.oa.provision.model;
import org.apache.solr.common.SolrInputField;
import java.util.HashMap; import java.util.HashMap;
import java.util.Map; import java.util.Map;
import org.apache.solr.common.SolrInputDocument;
import org.apache.solr.common.SolrInputField;
/** /**
* Wrapper class needed to make the SolrInputDocument compatible with the Kryo serialization mechanism. * Wrapper class needed to make the SolrInputDocument compatible with the Kryo serialization mechanism.
*/ */
public class SerializableSolrInputDocument extends SolrInputDocument { public class SerializableSolrInputDocument extends SolrInputDocument {
public SerializableSolrInputDocument() { public SerializableSolrInputDocument() {
super(new HashMap<>()); super(new HashMap<>());
} }
public SerializableSolrInputDocument(Map<String, SolrInputField> fields) { public SerializableSolrInputDocument(Map<String, SolrInputField> fields) {
super(fields); super(fields);
} }
} }

View File

@ -24,7 +24,13 @@
"paramRequired": false "paramRequired": false
}, },
{ {
"paramName": "o", "paramName": "of",
"paramLongName": "outputFormat",
"paramDescription": "decides the job output format, SOLR | HDFS",
"paramRequired": false
},
{
"paramName": "op",
"paramLongName": "outputPath", "paramLongName": "outputPath",
"paramDescription": "path on hdfs activating an alternative output for the SolrInputDocuments", "paramDescription": "path on hdfs activating an alternative output for the SolrInputDocuments",
"paramRequired": false "paramRequired": false

View File

@ -42,6 +42,7 @@
<value>*:*</value> <value>*:*</value>
<description>query used in the deleted by query operation</description> <description>query used in the deleted by query operation</description>
</property> </property>
<property> <property>
<name>sparkDriverMemoryForJoining</name> <name>sparkDriverMemoryForJoining</name>
<description>memory for driver process</description> <description>memory for driver process</description>
@ -638,7 +639,8 @@
<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>--outputPath</arg><arg>${outputPath}</arg> <arg>--outputFormat</arg><arg>${outputFormat}</arg>
<arg>--outputPath</arg><arg>${workingDir}/solr_documents</arg>
</spark> </spark>
<ok to="commit_solr_collection"/> <ok to="commit_solr_collection"/>
<error to="Kill"/> <error to="Kill"/>

View File

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