forked from antonis.lempesis/dnet-hadoop
introduced configuration param outputFormat: HDFS | SOLR
This commit is contained in:
parent
5218718e8b
commit
0374d34c3e
|
@ -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);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -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);
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -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
|
||||||
|
|
|
@ -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"/>
|
||||||
|
|
|
@ -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");
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
Loading…
Reference in New Issue