/**
* Licensed to the Apache Software Foundation (ASF) under one or more contributor license
* agreements. See the NOTICE file distributed with this work for additional information regarding
* copyright ownership. The ASF licenses this file to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance with the License. You may obtain a
* copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software distributed under the License
* is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
* or implied. See the License for the specific language governing permissions and limitations under
* the License.
*/
package org.apache.gora.solr.store;
import java.io.IOException;
import java.net.MalformedURLException;
import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Properties;
import java.util.concurrent.ConcurrentHashMap;
import org.apache.avro.Schema;
import org.apache.avro.Schema.Field;
import org.apache.avro.Schema.Type;
import org.apache.avro.specific.SpecificDatumReader;
import org.apache.avro.specific.SpecificDatumWriter;
import org.apache.avro.util.Utf8;
import org.apache.gora.persistency.Persistent;
import org.apache.gora.persistency.impl.PersistentBase;
import org.apache.gora.query.PartitionQuery;
import org.apache.gora.query.Query;
import org.apache.gora.query.Result;
import org.apache.gora.query.impl.PartitionQueryImpl;
import org.apache.gora.solr.query.SolrQuery;
import org.apache.gora.solr.query.SolrResult;
import org.apache.gora.store.DataStoreFactory;
import org.apache.gora.store.impl.DataStoreBase;
import org.apache.gora.util.AvroUtils;
import org.apache.gora.util.IOUtils;
import org.apache.hadoop.util.StringUtils;
import org.apache.solr.client.solrj.SolrServer;
import org.apache.solr.client.solrj.SolrServerException;
import org.apache.solr.client.solrj.impl.CloudSolrServer;
import org.apache.solr.client.solrj.impl.ConcurrentUpdateSolrServer;
import org.apache.solr.client.solrj.impl.HttpSolrServer;
import org.apache.solr.client.solrj.impl.LBHttpSolrServer;
import org.apache.solr.client.solrj.request.CoreAdminRequest;
import org.apache.solr.client.solrj.response.CoreAdminResponse;
import org.apache.solr.client.solrj.response.QueryResponse;
import org.apache.solr.client.solrj.response.UpdateResponse;
import org.apache.solr.common.SolrDocument;
import org.apache.solr.common.SolrInputDocument;
import org.apache.solr.common.params.CommonParams;
import org.apache.solr.common.params.ModifiableSolrParams;
import org.jdom.Document;
import org.jdom.Element;
import org.jdom.input.SAXBuilder;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
public class SolrStore<K, T extends PersistentBase> extends DataStoreBase<K, T> {
private static final Logger LOG = LoggerFactory.getLogger(SolrStore.class);
/** The default file name value to be used for obtaining the Solr object field mapping's */
protected static final String DEFAULT_MAPPING_FILE = "gora-solr-mapping.xml";
/** The URL of the Solr server - defined in <code>gora.properties</code> */
protected static final String SOLR_URL_PROPERTY = "solr.url";
/** The <code>solrconfig.xml</code> file to be used - defined in <code>gora.properties</code>*/
protected static final String SOLR_CONFIG_PROPERTY = "solr.config";
/** The <code>schema.xml</code> file to be used - defined in <code>gora.properties</code>*/
protected static final String SOLR_SCHEMA_PROPERTY = "solr.schema";
/** A batch size unit (ArrayList) of SolrDocument's to be used for writing to Solr.
* Should be defined in <code>gora.properties</code>.
* A default value of 100 is used if this value is absent. This value must be of type <b>Integer</b>.
*/
protected static final String SOLR_BATCH_SIZE_PROPERTY = "solr.batch_size";
/** The solrj implementation to use. This has a default value of <i>http</i> for HttpSolrServer.
* Available options include <b>http</b>, <b>cloud</b>, <b>concurrent</b> and <b>loadbalance</b>.
* Defined in <code>gora.properties</code>
* This value must be of type <b>String</b>.
*/
protected static final String SOLR_SOLRJSERVER_IMPL = "solr.solrjserver";
/** A batch commit unit for SolrDocument's used when making (commit) calls to Solr.
* Should be defined in <code>gora.properties</code>.
* A default value of 1000 is used if this value is absent. This value must be of type <b>Integer</b>.
*/
protected static final String SOLR_COMMIT_WITHIN_PROPERTY = "solr.commit_within";
/** The maximum number of result to return when we make a call to
* {@link org.apache.gora.solr.store.SolrStore#execute(Query)}. This should be
* defined in <code>gora.properties</code>. This value must be of type <b>Integer</b>.
*/
protected static final String SOLR_RESULTS_SIZE_PROPERTY = "solr.results_size";
/** The default batch size (ArrayList) of SolrDocuments to be used in the event of an absent
* value for <code>solr.batchSize</code>.
* Set to 100 by default.
*/
protected static final int DEFAULT_BATCH_SIZE = 100;
/** The default commit size of SolrDocuments to be used in the event of an absent
* value for <code>solr.commitSize</code>.
* Set to 1000 by default.
*/
protected static final int DEFAULT_COMMIT_WITHIN = 1000;
/** The default results size of SolrDocuments to be used in the event of an absent
* value for <code>solr.resultsSize</code>.
* Set to 100 by default.
*/
protected static final int DEFAULT_RESULTS_SIZE = 100;
private SolrMapping mapping;
private String solrServerUrl, solrConfig, solrSchema, solrJServerImpl;
private SolrServer server, adminServer;
private ArrayList<SolrInputDocument> batch;
private int batchSize = DEFAULT_BATCH_SIZE;
private int commitWithin = DEFAULT_COMMIT_WITHIN;
private int resultsSize = DEFAULT_RESULTS_SIZE;
/**
* Default schema index with value "0" used when AVRO Union data types are
* stored
*/
public static int DEFAULT_UNION_SCHEMA = 0;
/*
* Create a threadlocal map for the datum readers and writers, because they
* are not thread safe, at least not before Avro 1.4.0 (See AVRO-650). When
* they are thread safe, it is possible to maintain a single reader and writer
* pair for every schema, instead of one for every thread.
*/
public static final ConcurrentHashMap<String, SpecificDatumReader<?>> readerMap = new ConcurrentHashMap<String, SpecificDatumReader<?>>();
public static final ConcurrentHashMap<String, SpecificDatumWriter<?>> writerMap = new ConcurrentHashMap<String, SpecificDatumWriter<?>>();
@Override
public void initialize(Class<K> keyClass, Class<T> persistentClass,
Properties properties) {
super.initialize(keyClass, persistentClass, properties);
try {
String mappingFile = DataStoreFactory.getMappingFile(properties, this,
DEFAULT_MAPPING_FILE);
mapping = readMapping(mappingFile);
} catch (IOException e) {
LOG.error(e.getMessage());
LOG.error(e.getStackTrace().toString());
}
solrServerUrl = DataStoreFactory.findProperty(properties, this,
SOLR_URL_PROPERTY, null);
solrConfig = DataStoreFactory.findProperty(properties, this,
SOLR_CONFIG_PROPERTY, null);
solrSchema = DataStoreFactory.findProperty(properties, this,
SOLR_SCHEMA_PROPERTY, null);
solrJServerImpl = DataStoreFactory.findProperty(properties, this,
SOLR_SOLRJSERVER_IMPL, "http");
LOG.info("Using Solr server at " + solrServerUrl);
String solrJServerType = ((solrJServerImpl == null || solrJServerImpl.equals(""))?"http":solrJServerImpl);
// HttpSolrServer - denoted by "http" in properties
if (solrJServerType.toString().toLowerCase().equals("http")) {
LOG.info("Using HttpSolrServer Solrj implementation.");
this.adminServer = new HttpSolrServer(solrServerUrl);
this.server = new HttpSolrServer( solrServerUrl + "/" + mapping.getCoreName() );
// CloudSolrServer - denoted by "cloud" in properties
} else if (solrJServerType.toString().toLowerCase().equals("cloud")) {
LOG.info("Using CloudSolrServer Solrj implementation.");
this.adminServer = new CloudSolrServer(solrServerUrl);
this.server = new CloudSolrServer( solrServerUrl + "/" + mapping.getCoreName() );
} else if (solrJServerType.toString().toLowerCase().equals("concurrent")) {
LOG.info("Using ConcurrentUpdateSolrServer Solrj implementation.");
this.adminServer = new ConcurrentUpdateSolrServer(solrServerUrl, 1000, 10);
this.server = new ConcurrentUpdateSolrServer( solrServerUrl + "/" + mapping.getCoreName(), 1000, 10);
// LBHttpSolrServer - denoted by "loadbalance" in properties
} else if (solrJServerType.toString().toLowerCase().equals("loadbalance")) {
LOG.info("Using LBHttpSolrServer Solrj implementation.");
String[] solrUrlElements = StringUtils.split(solrServerUrl);
try {
this.adminServer = new LBHttpSolrServer(solrUrlElements);
} catch (MalformedURLException e) {
e.printStackTrace();
}
try {
this.server = new LBHttpSolrServer( solrUrlElements + "/" + mapping.getCoreName() );
} catch (MalformedURLException e) {
e.printStackTrace();
}
}
if (autoCreateSchema) {
createSchema();
}
String batchSizeString = DataStoreFactory.findProperty(properties, this,
SOLR_BATCH_SIZE_PROPERTY, null);
if (batchSizeString != null) {
try {
batchSize = Integer.parseInt(batchSizeString);
} catch (NumberFormatException nfe) {
LOG.warn("Invalid batch size '" + batchSizeString + "', using default "
+ DEFAULT_BATCH_SIZE);
}
}
batch = new ArrayList<SolrInputDocument>(batchSize);
String commitWithinString = DataStoreFactory.findProperty(properties, this,
SOLR_COMMIT_WITHIN_PROPERTY, null);
if (commitWithinString != null) {
try {
commitWithin = Integer.parseInt(commitWithinString);
} catch (NumberFormatException nfe) {
LOG.warn("Invalid commit within '" + commitWithinString
+ "', using default " + DEFAULT_COMMIT_WITHIN);
}
}
String resultsSizeString = DataStoreFactory.findProperty(properties, this,
SOLR_RESULTS_SIZE_PROPERTY, null);
if (resultsSizeString != null) {
try {
resultsSize = Integer.parseInt(resultsSizeString);
} catch (NumberFormatException nfe) {
LOG.warn("Invalid results size '" + resultsSizeString
+ "', using default " + DEFAULT_RESULTS_SIZE);
}
}
}
@SuppressWarnings("unchecked")
private SolrMapping readMapping(String filename) throws IOException {
SolrMapping map = new SolrMapping();
try {
SAXBuilder builder = new SAXBuilder();
Document doc = builder.build(getClass().getClassLoader()
.getResourceAsStream(filename));
List<Element> classes = doc.getRootElement().getChildren("class");
for (Element classElement : classes) {
if (classElement.getAttributeValue("keyClass").equals(
keyClass.getCanonicalName())
&& classElement.getAttributeValue("name").equals(
persistentClass.getCanonicalName())) {
String tableName = getSchemaName(
classElement.getAttributeValue("table"), persistentClass);
map.setCoreName(tableName);
Element primaryKeyEl = classElement.getChild("primarykey");
map.setPrimaryKey(primaryKeyEl.getAttributeValue("column"));
List<Element> fields = classElement.getChildren("field");
for (Element field : fields) {
String fieldName = field.getAttributeValue("name");
String columnName = field.getAttributeValue("column");
map.addField(fieldName, columnName);
}
break;
}
LOG.warn("Check that 'keyClass' and 'name' parameters in gora-solr-mapping.xml "
+ "match with intended values. A mapping mismatch has been found therefore "
+ "no mapping has been initialized for class mapping at position "
+ classes.indexOf(classElement) + " in mapping file.");
}
} catch (Exception ex) {
throw new IOException(ex);
}
return map;
}
public SolrMapping getMapping() {
return mapping;
}
@Override
public String getSchemaName() {
return mapping.getCoreName();
}
@Override
public void createSchema() {
try {
if (!schemaExists())
CoreAdminRequest.createCore(mapping.getCoreName(),
mapping.getCoreName(), adminServer, solrConfig, solrSchema);
} catch (Exception e) {
LOG.error(e.getMessage(), e.getStackTrace().toString());
}
}
@Override
/** Default implementation deletes and recreates the schema*/
public void truncateSchema() {
try {
server.deleteByQuery("*:*");
server.commit();
} catch (Exception e) {
// ignore?
LOG.error(e.getMessage(), e.getStackTrace().toString());
}
}
@Override
public void deleteSchema() {
// XXX should this be only in truncateSchema ???
try {
server.deleteByQuery("*:*");
server.commit();
} catch (Exception e) {
// ignore?
// LOG.error(e.getMessage());
// LOG.error(e.getStackTrace().toString());
}
try {
CoreAdminRequest.unloadCore(mapping.getCoreName(), adminServer);
} catch (Exception e) {
if (e.getMessage().contains("No such core")) {
return; // it's ok, the core is not there
} else {
LOG.error(e.getMessage(), e.getStackTrace().toString());
}
}
}
@Override
public boolean schemaExists() {
boolean exists = false;
try {
CoreAdminResponse rsp = CoreAdminRequest.getStatus(mapping.getCoreName(),
adminServer);
exists = rsp.getUptime(mapping.getCoreName()) != null;
} catch (Exception e) {
LOG.error(e.getMessage(), e.getStackTrace().toString());
}
return exists;
}
private static final String toDelimitedString(String[] arr, String sep) {
if (arr == null || arr.length == 0) {
return "";
}
StringBuilder sb = new StringBuilder();
for (int i = 0; i < arr.length; i++) {
if (i > 0)
sb.append(sep);
sb.append(arr[i]);
}
return sb.toString();
}
public static String escapeQueryKey(String key) {
if (key == null) {
return null;
}
StringBuilder sb = new StringBuilder();
for (int i = 0; i < key.length(); i++) {
char c = key.charAt(i);
switch (c) {
case ':':
case '*':
sb.append("\\" + c);
break;
default:
sb.append(c);
}
}
return sb.toString();
}
@Override
public T get(K key, String[] fields) {
ModifiableSolrParams params = new ModifiableSolrParams();
params.set(CommonParams.QT, "/get");
params.set(CommonParams.FL, toDelimitedString(fields, ","));
params.set("id", key.toString());
try {
QueryResponse rsp = server.query(params);
Object o = rsp.getResponse().get("doc");
if (o == null) {
return null;
}
return newInstance((SolrDocument) o, fields);
} catch (Exception e) {
LOG.error(e.getMessage(), e.getStackTrace().toString());
}
return null;
}
public T newInstance(SolrDocument doc, String[] fields) throws IOException {
T persistent = newPersistent();
if (fields == null) {
fields = fieldMap.keySet().toArray(new String[fieldMap.size()]);
}
String pk = mapping.getPrimaryKey();
for (String f : fields) {
Field field = fieldMap.get(f);
Schema fieldSchema = field.schema();
String sf = null;
if (pk.equals(f)) {
sf = f;
} else {
sf = mapping.getSolrField(f);
}
Object sv = doc.get(sf);
if (sv == null) {
continue;
}
Object v = deserializeFieldValue(field, fieldSchema, sv, persistent);
persistent.put(field.pos(), v);
persistent.setDirty(field.pos());
}
persistent.clearDirty();
return persistent;
}
@SuppressWarnings("rawtypes")
private SpecificDatumReader getDatumReader(String schemaId, Schema fieldSchema) {
SpecificDatumReader<?> reader = (SpecificDatumReader<?>) readerMap
.get(schemaId);
if (reader == null) {
reader = new SpecificDatumReader(fieldSchema);// ignore dirty bits
SpecificDatumReader localReader = null;
if ((localReader = readerMap.putIfAbsent(schemaId, reader)) != null) {
reader = localReader;
}
}
return reader;
}
@SuppressWarnings("rawtypes")
private SpecificDatumWriter getDatumWriter(String schemaId, Schema fieldSchema) {
SpecificDatumWriter writer = (SpecificDatumWriter<?>) writerMap
.get(schemaId);
if (writer == null) {
writer = new SpecificDatumWriter(fieldSchema);// ignore dirty bits
writerMap.put(schemaId, writer);
}
return writer;
}
@SuppressWarnings("unchecked")
private Object deserializeFieldValue(Field field, Schema fieldSchema,
Object solrValue, T persistent) throws IOException {
Object fieldValue = null;
switch (fieldSchema.getType()) {
case MAP:
case ARRAY:
case RECORD:
@SuppressWarnings("rawtypes")
SpecificDatumReader reader = getDatumReader(fieldSchema.getFullName(),
fieldSchema);
fieldValue = IOUtils.deserialize((byte[]) solrValue, reader, fieldSchema,
persistent.get(field.pos()));
break;
case ENUM:
fieldValue = AvroUtils.getEnumValue(fieldSchema, (String) solrValue);
break;
case FIXED:
throw new IOException("???");
// break;
case BYTES:
fieldValue = ByteBuffer.wrap((byte[]) solrValue);
break;
case STRING:
fieldValue = new Utf8(solrValue.toString());
break;
case UNION:
if (fieldSchema.getTypes().size() == 2 && isNullable(fieldSchema)) {
// schema [type0, type1]
Type type0 = fieldSchema.getTypes().get(0).getType();
Type type1 = fieldSchema.getTypes().get(1).getType();
// Check if types are different and there's a "null", like
// ["null","type"] or ["type","null"]
if (!type0.equals(type1)) {
if (type0.equals(Schema.Type.NULL))
fieldSchema = fieldSchema.getTypes().get(1);
else
fieldSchema = fieldSchema.getTypes().get(0);
} else {
fieldSchema = fieldSchema.getTypes().get(0);
}
fieldValue = deserializeFieldValue(field, fieldSchema, solrValue,
persistent);
} else {
@SuppressWarnings("rawtypes")
SpecificDatumReader unionReader = getDatumReader(
String.valueOf(fieldSchema.hashCode()), fieldSchema);
fieldValue = IOUtils.deserialize((byte[]) solrValue, unionReader,
fieldSchema, persistent.get(field.pos()));
break;
}
break;
default:
fieldValue = solrValue;
}
return fieldValue;
}
@Override
public void put(K key, T persistent) {
Schema schema = persistent.getSchema();
if (!persistent.isDirty()) {
// nothing to do
return;
}
SolrInputDocument doc = new SolrInputDocument();
// add primary key
doc.addField(mapping.getPrimaryKey(), key);
// populate the doc
List<Field> fields = schema.getFields();
for (Field field : fields) {
String sf = mapping.getSolrField(field.name());
// Solr will append values to fields in a SolrInputDocument, even the key
// mapping won't find the primary
if (sf == null) {
continue;
}
Schema fieldSchema = field.schema();
Object v = persistent.get(field.pos());
if (v == null) {
continue;
}
v = serializeFieldValue(fieldSchema, v);
doc.addField(sf, v);
}
LOG.info("Putting DOCUMENT: " + doc);
batch.add(doc);
if (batch.size() >= batchSize) {
try {
add(batch, commitWithin);
batch.clear();
} catch (Exception e) {
LOG.error(e.getMessage(), e.getStackTrace().toString());
}
}
}
@SuppressWarnings("unchecked")
private Object serializeFieldValue(Schema fieldSchema, Object fieldValue) {
switch (fieldSchema.getType()) {
case MAP:
case ARRAY:
case RECORD:
byte[] data = null;
try {
@SuppressWarnings("rawtypes")
SpecificDatumWriter writer = getDatumWriter(fieldSchema.getFullName(),
fieldSchema);
data = IOUtils.serialize(writer, fieldSchema, fieldValue);
} catch (IOException e) {
LOG.error(e.getMessage(), e.getStackTrace().toString());
}
fieldValue = data;
break;
case BYTES:
fieldValue = ((ByteBuffer) fieldValue).array();
break;
case ENUM:
case STRING:
fieldValue = fieldValue.toString();
break;
case UNION:
// If field's schema is null and one type, we do undertake serialization.
// All other types are serialized.
if (fieldSchema.getTypes().size() == 2 && isNullable(fieldSchema)) {
int schemaPos = getUnionSchema(fieldValue, fieldSchema);
Schema unionSchema = fieldSchema.getTypes().get(schemaPos);
fieldValue = serializeFieldValue(unionSchema, fieldValue);
} else {
byte[] serilazeData = null;
try {
@SuppressWarnings("rawtypes")
SpecificDatumWriter writer = getDatumWriter(
String.valueOf(fieldSchema.hashCode()), fieldSchema);
serilazeData = IOUtils.serialize(writer, fieldSchema, fieldValue);
} catch (IOException e) {
LOG.error(e.getMessage(), e.getStackTrace().toString());
}
fieldValue = serilazeData;
}
break;
default:
// LOG.error("Unknown field type: " + fieldSchema.getType());
break;
}
return fieldValue;
}
private boolean isNullable(Schema unionSchema) {
for (Schema innerSchema : unionSchema.getTypes()) {
if (innerSchema.getType().equals(Schema.Type.NULL)) {
return true;
}
}
return false;
}
/**
* Given an object and the object schema this function obtains, from within
* the UNION schema, the position of the type used. If no data type can be
* inferred then we return a default value of position 0.
*
* @param pValue
* @param pUnionSchema
* @return the unionSchemaPosition.
*/
private int getUnionSchema(Object pValue, Schema pUnionSchema) {
int unionSchemaPos = 0;
Iterator<Schema> it = pUnionSchema.getTypes().iterator();
while (it.hasNext()) {
Type schemaType = it.next().getType();
if (pValue instanceof Utf8 && schemaType.equals(Type.STRING))
return unionSchemaPos;
else if (pValue instanceof ByteBuffer && schemaType.equals(Type.BYTES))
return unionSchemaPos;
else if (pValue instanceof Integer && schemaType.equals(Type.INT))
return unionSchemaPos;
else if (pValue instanceof Long && schemaType.equals(Type.LONG))
return unionSchemaPos;
else if (pValue instanceof Double && schemaType.equals(Type.DOUBLE))
return unionSchemaPos;
else if (pValue instanceof Float && schemaType.equals(Type.FLOAT))
return unionSchemaPos;
else if (pValue instanceof Boolean && schemaType.equals(Type.BOOLEAN))
return unionSchemaPos;
else if (pValue instanceof Map && schemaType.equals(Type.MAP))
return unionSchemaPos;
else if (pValue instanceof List && schemaType.equals(Type.ARRAY))
return unionSchemaPos;
else if (pValue instanceof Persistent && schemaType.equals(Type.RECORD))
return unionSchemaPos;
unionSchemaPos++;
}
// if we weren't able to determine which data type it is, then we return the
// default
return DEFAULT_UNION_SCHEMA;
}
@Override
public boolean delete(K key) {
String keyField = mapping.getPrimaryKey();
try {
UpdateResponse rsp = server.deleteByQuery(keyField + ":"
+ escapeQueryKey(key.toString()));
server.commit();
LOG.info(rsp.toString());
return true;
} catch (Exception e) {
LOG.error(e.getMessage(), e.getStackTrace().toString());
}
return false;
}
@Override
public long deleteByQuery(Query<K, T> query) {
String q = ((SolrQuery<K, T>) query).toSolrQuery();
try {
UpdateResponse rsp = server.deleteByQuery(q);
server.commit();
LOG.info(rsp.toString());
} catch (Exception e) {
LOG.error(e.getMessage(), e.getStackTrace().toString());
}
return 0;
}
@Override
public Result<K, T> execute(Query<K, T> query) {
try {
return new SolrResult<K, T>(this, query, server, resultsSize);
} catch (IOException e) {
LOG.error(e.getMessage(), e.getStackTrace().toString());
}
return null;
}
@Override
public Query<K, T> newQuery() {
return new SolrQuery<K, T>(this);
}
@Override
public List<PartitionQuery<K, T>> getPartitions(Query<K, T> query)
throws IOException {
// TODO: implement this using Hadoop DB support
ArrayList<PartitionQuery<K, T>> partitions = new ArrayList<PartitionQuery<K, T>>();
PartitionQueryImpl<K, T> pqi = new PartitionQueryImpl<K, T>(query);
pqi.setConf(getConf());
partitions.add(pqi);
return partitions;
}
@Override
public void flush() {
try {
if (batch.size() > 0) {
add(batch, commitWithin);
batch.clear();
}
} catch (Exception e) {
LOG.error(e.getMessage(), e.getStackTrace());
}
}
@Override
public void close() {
// In testing, the index gets closed before the commit in flush() can happen
// so an exception gets thrown
// flush();
}
private void add(ArrayList<SolrInputDocument> batch, int commitWithin)
throws SolrServerException, IOException {
if (commitWithin == 0) {
server.add(batch);
server.commit(false, true, true);
} else {
server.add(batch, commitWithin);
}
}
}