mirror of https://github.com/apache/lucene.git
SOLR-561 -- Added Replication implemented in Java as a request handler. Supports index replication as well as configuration replication and exposes detailed statistics and progress information on the Admin page. Works on all platforms.
git-svn-id: https://svn.apache.org/repos/asf/lucene/solr/trunk@706565 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
be8a3a53ff
commit
fa8ca41a71
|
@ -63,6 +63,9 @@ New Features
|
|||
8. SOLR-680: Add StatsComponent. This gets simple statists on matched numeric fields,
|
||||
including: min, max, mean, median, stddev. (koji, ryan)
|
||||
|
||||
9. SOLR-561: Added Replication implemented in Java as a request handler. Supports index replication
|
||||
as well as configuration replication and exposes detailed statistics and progress information
|
||||
on the Admin page. Works on all platforms. (Noble Paul, yonik, Akshay Ukey, shalin)
|
||||
|
||||
Optimizations
|
||||
----------------------
|
||||
|
|
|
@ -397,6 +397,24 @@
|
|||
</lst>
|
||||
</requestHandler>
|
||||
|
||||
<!--Master config-->
|
||||
<!--
|
||||
<requestHandler name="/replication" class="solr.ReplicationHandler" >
|
||||
<lst name="master">
|
||||
<str name="replicateAfter">commit</str>
|
||||
<str name="confFiles">schema.xml,stopwords.txt</str>
|
||||
</lst>
|
||||
</requestHandler>
|
||||
-->
|
||||
<!-- Slave config-->
|
||||
<!--
|
||||
<requestHandler name="/replication" class="solr.ReplicationHandler" >
|
||||
<lst name="slave">
|
||||
<str name="masterUrl">http://localhost:8983/solr/replication</str>
|
||||
<str name="pollInterval">00:00:60</str>
|
||||
</lst>
|
||||
</requestHandler>
|
||||
-->
|
||||
|
||||
<!-- DisMaxRequestHandler allows easy searching across multiple fields
|
||||
for simple user-entered phrases. It's implementation is now
|
||||
|
|
|
@ -0,0 +1,842 @@
|
|||
/**
|
||||
* 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.solr.handler;
|
||||
|
||||
import org.apache.commons.httpclient.HttpClient;
|
||||
import org.apache.lucene.index.IndexCommit;
|
||||
import org.apache.solr.common.params.CommonParams;
|
||||
import org.apache.solr.common.params.ModifiableSolrParams;
|
||||
import org.apache.solr.common.params.SolrParams;
|
||||
import org.apache.solr.common.util.FastOutputStream;
|
||||
import org.apache.solr.common.util.NamedList;
|
||||
import org.apache.solr.common.util.SimpleOrderedMap;
|
||||
import org.apache.solr.core.CloseHook;
|
||||
import org.apache.solr.core.SolrCore;
|
||||
import org.apache.solr.core.SolrEventListener;
|
||||
import org.apache.solr.request.BinaryQueryResponseWriter;
|
||||
import org.apache.solr.request.SolrQueryRequest;
|
||||
import org.apache.solr.request.SolrQueryResponse;
|
||||
import org.apache.solr.search.SolrIndexSearcher;
|
||||
import org.apache.solr.util.RefCounted;
|
||||
import org.apache.solr.util.plugin.SolrCoreAware;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import java.io.*;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.nio.channels.FileChannel;
|
||||
import java.text.NumberFormat;
|
||||
import java.util.*;
|
||||
import java.util.concurrent.locks.ReentrantLock;
|
||||
import java.util.zip.Adler32;
|
||||
import java.util.zip.Checksum;
|
||||
|
||||
/**
|
||||
* <p> A Handler which provides a REST API for replication and serves replication requests from Slaves.
|
||||
* <p/>
|
||||
* </p>
|
||||
* <p>When running on the master, it provides the following commands
|
||||
* <ol>
|
||||
* <li>Get the current replicatable index version (command=indexversion)</li>
|
||||
* <li>Get the list of files for a given index version (command=filelist&indexversion=<VERSION>)</li>
|
||||
* <li>Get full or a part (chunk) of a given index or a config file (command=filecontent&file=<FILE_NAME>)
|
||||
* You can optionally specify an offset and length to get that chunk of the file.
|
||||
* You can request a configuration file by using "cf" parameter instead of the "file" parameter.</li>
|
||||
* <li>Get status/statistics (command=details)</li>
|
||||
* </ol>
|
||||
* </p>
|
||||
* <p>When running on the slave, it provides the following commands
|
||||
* <ol>
|
||||
* <li>Perform a snap pull now (command=snappull)</li>
|
||||
* <li>Get status/statistics (command=details)</li>
|
||||
* <li>Abort a snap pull (command=abort)</li>
|
||||
* <li>Enable/Disable polling the master for new versions (command=enablepoll or command=disablepoll)</li>
|
||||
* </ol>
|
||||
* </p>
|
||||
*
|
||||
* @version $Id$
|
||||
* @since solr 1.4
|
||||
*/
|
||||
public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAware {
|
||||
private static final Logger LOG = LoggerFactory.getLogger(ReplicationHandler.class.getName());
|
||||
private SolrCore core;
|
||||
|
||||
private SnapPuller snapPuller;
|
||||
|
||||
private ReentrantLock snapPullLock = new ReentrantLock();
|
||||
|
||||
private List<String> includeConfFiles;
|
||||
|
||||
private boolean isMaster = false;
|
||||
|
||||
private boolean isSlave = false;
|
||||
|
||||
private boolean replicateOnOptimize = false;
|
||||
|
||||
private boolean replicateOnCommit = false;
|
||||
|
||||
//private String masterUrl;
|
||||
|
||||
//private String pollInterval;
|
||||
|
||||
private int numTimesReplicated = 0;
|
||||
|
||||
private final Map<String, FileInfo> confFileInfoCache = new HashMap<String, FileInfo>();
|
||||
|
||||
private Integer reserveCommitDuration = SnapPuller.readInterval("01:00:00");
|
||||
|
||||
private IndexCommit indexCommitPoint;
|
||||
|
||||
public void handleRequestBody(SolrQueryRequest req, SolrQueryResponse rsp) throws Exception {
|
||||
rsp.setHttpCaching(false);
|
||||
SolrParams solrParams = req.getParams();
|
||||
String command = solrParams.get(COMMAND);
|
||||
if (command == null) {
|
||||
rsp.add("status", "OK");
|
||||
return;
|
||||
}
|
||||
if (command.equals(CMD_INDEX_VERSION)) {
|
||||
if (indexCommitPoint != null) {
|
||||
rsp.add(CMD_INDEX_VERSION, indexCommitPoint.getVersion());
|
||||
rsp.add(GENERATION, indexCommitPoint.getGeneration());
|
||||
} else {
|
||||
// must never happen
|
||||
rsp.add(CMD_INDEX_VERSION, 0L);
|
||||
rsp.add(GENERATION, 0L);
|
||||
}
|
||||
} else if (command.equals(CMD_GET_FILE)) {
|
||||
getFileStream(solrParams, rsp);
|
||||
} else if (command.equals(CMD_GET_FILE_LIST)) {
|
||||
getFileList(solrParams, rsp);
|
||||
} else if (command.equals(CMD_SNAP_SHOOT)) {
|
||||
doSnapShoot(rsp);
|
||||
} else if (command.equals(CMD_SNAP_PULL)) {
|
||||
doSnapPull();
|
||||
} else if (command.equals(CMD_DISABLE_POLL)) {
|
||||
if (snapPuller != null)
|
||||
snapPuller.disablePoll();
|
||||
} else if (command.equals(CMD_ENABLE_POLL)) {
|
||||
if (snapPuller != null)
|
||||
snapPuller.enablePoll();
|
||||
} else if (command.equals(CMD_ABORT_SNAP_PULL)) {
|
||||
if (snapPuller != null)
|
||||
snapPuller.abortPull();
|
||||
} else if (command.equals(CMD_FILE_CHECKSUM)) {
|
||||
getFileChecksum(solrParams, rsp);
|
||||
} else if (command.equals(CMD_SHOW_COMMITS)) {
|
||||
rsp.add(CMD_SHOW_COMMITS, getCommits());
|
||||
} else if (command.equals(CMD_DETAILS)) {
|
||||
getReplicationDetails(rsp);
|
||||
}
|
||||
}
|
||||
|
||||
private List<NamedList> getCommits() {
|
||||
Map<Long, IndexCommit> commits = core.getDeletionPolicy().getCommits();
|
||||
List<NamedList> l = new ArrayList<NamedList>();
|
||||
|
||||
for (IndexCommit c : commits.values()) {
|
||||
try {
|
||||
NamedList nl = new NamedList();
|
||||
nl.add(CMD_INDEX_VERSION, c.getVersion());
|
||||
nl.add(GENERATION, c.getGeneration());
|
||||
nl.add(CMD_GET_FILE_LIST, c.getFileNames().toString());
|
||||
l.add(nl);
|
||||
} catch (IOException e) {
|
||||
LOG.warn("Exception while reading files for commit " + c, e);
|
||||
}
|
||||
}
|
||||
return l;
|
||||
}
|
||||
|
||||
private void getFileChecksum(SolrParams solrParams, SolrQueryResponse rsp) {
|
||||
Checksum checksum = new Adler32();
|
||||
File dir = new File(core.getIndexDir());
|
||||
rsp.add(CHECKSUM, getCheckSums(solrParams.getParams(FILE), dir, checksum));
|
||||
dir = new File(core.getResourceLoader().getConfigDir());
|
||||
rsp.add(CONF_CHECKSUM, getCheckSums(solrParams.getParams(CONF_FILE_SHORT), dir, checksum));
|
||||
}
|
||||
|
||||
private Map<String, Long> getCheckSums(String[] files, File dir, Checksum checksum) {
|
||||
Map<String, Long> checksumMap = new HashMap<String, Long>();
|
||||
if (files == null || files.length == 0)
|
||||
return checksumMap;
|
||||
for (String file : files) {
|
||||
File f = new File(dir, file);
|
||||
Long checkSumVal = getCheckSum(checksum, f);
|
||||
if (checkSumVal != null)
|
||||
checksumMap.put(file, checkSumVal);
|
||||
}
|
||||
return checksumMap;
|
||||
}
|
||||
|
||||
static Long getCheckSum(Checksum checksum, File f) {
|
||||
FileInputStream fis = null;
|
||||
checksum.reset();
|
||||
byte[] buffer = new byte[1024 * 1024];
|
||||
int bytesRead;
|
||||
try {
|
||||
fis = new FileInputStream(f);
|
||||
while ((bytesRead = fis.read(buffer)) >= 0)
|
||||
checksum.update(buffer, 0, bytesRead);
|
||||
return checksum.getValue();
|
||||
} catch (Exception e) {
|
||||
LOG.warn("Exception in finding checksum of " + f, e);
|
||||
} finally {
|
||||
closeNoExp(fis);
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
void doSnapPull() {
|
||||
if (!isSlave)
|
||||
return;
|
||||
if (snapPullLock.isLocked())
|
||||
return;
|
||||
snapPullLock.lock();
|
||||
try {
|
||||
snapPuller.fetchLatestIndex(core);
|
||||
} catch (Exception e) {
|
||||
LOG.error("SnapPull failed ", e);
|
||||
} finally {
|
||||
snapPullLock.unlock();
|
||||
}
|
||||
}
|
||||
|
||||
boolean isReplicating() {
|
||||
boolean b = snapPullLock.isLocked();
|
||||
return snapPullLock.isLocked();
|
||||
}
|
||||
|
||||
private void doSnapShoot(SolrQueryResponse rsp) {
|
||||
try {
|
||||
new SnapShooter(core).createSnapAsync(core.getDeletionPolicy().getLatestCommit().getFileNames());
|
||||
} catch (Exception e) {
|
||||
rsp.add("exception", e);
|
||||
}
|
||||
}
|
||||
|
||||
private void getFileStream(SolrParams solrParams, SolrQueryResponse rsp) {
|
||||
ModifiableSolrParams rawParams = new ModifiableSolrParams(solrParams);
|
||||
rawParams.set(CommonParams.WT, FILE_STREAM);
|
||||
rsp.add(FILE_STREAM, new FileStream(solrParams));
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
private void getFileList(SolrParams solrParams, SolrQueryResponse rsp) {
|
||||
String v = solrParams.get(CMD_INDEX_VERSION);
|
||||
if (v == null) {
|
||||
rsp.add("status", "no indexversion specified");
|
||||
return;
|
||||
}
|
||||
long version = Long.parseLong(v);
|
||||
IndexCommit commit = core.getDeletionPolicy().getCommitPoint(version);
|
||||
if (commit == null) {
|
||||
rsp.add("status", "invalid indexversion");
|
||||
return;
|
||||
}
|
||||
core.getDeletionPolicy().setReserveDuration(version, reserveCommitDuration);
|
||||
List<Map<String, Object>> result = new ArrayList<Map<String, Object>>();
|
||||
try {
|
||||
Collection<String> files = commit.getFileNames();
|
||||
for (String fileName : files) {
|
||||
File file = new File(core.getIndexDir(), fileName);
|
||||
Map<String, Object> fileMeta = getFileInfo(file);
|
||||
result.add(fileMeta);
|
||||
}
|
||||
} catch (IOException e) {
|
||||
rsp.add("status", "unable to get file names for given indexversion");
|
||||
rsp.add("exception", e);
|
||||
LOG.warn("Unable to get file names for indexCommit version: "
|
||||
+ version, e);
|
||||
}
|
||||
rsp.add(CMD_GET_FILE_LIST, result);
|
||||
if (includeConfFiles == null)
|
||||
return;
|
||||
LOG.debug("Adding config files to list: " + includeConfFiles);
|
||||
List<Map<String, Object>> confFiles = getConfFileCache(includeConfFiles);
|
||||
rsp.add(CONF_FILES, confFiles);
|
||||
}
|
||||
|
||||
List<Map<String, Object>> getConfFileCache(Collection<String> filenames) {
|
||||
List<Map<String, Object>> confFiles = new ArrayList<Map<String, Object>>();
|
||||
synchronized (confFileInfoCache) {
|
||||
File confDir = new File(core.getResourceLoader().getConfigDir());
|
||||
Checksum checksum = null;
|
||||
for (String cf : filenames) {
|
||||
File f = new File(confDir, cf);
|
||||
if (!f.exists() || f.isDirectory()) continue;//must not happen
|
||||
FileInfo info = confFileInfoCache.get(cf);
|
||||
if (info == null || info.lastmodified != f.lastModified() || info.size != f.length()) {
|
||||
if (checksum == null) checksum = new Adler32();
|
||||
info = new FileInfo(f.lastModified(), cf, f.length(), getCheckSum(checksum, f));
|
||||
confFileInfoCache.put(cf, info);
|
||||
}
|
||||
confFiles.add(info.getAsMap());
|
||||
}
|
||||
}
|
||||
return confFiles;
|
||||
}
|
||||
|
||||
private static class FileInfo {
|
||||
|
||||
long lastmodified;
|
||||
String name;
|
||||
long size;
|
||||
long checksum;
|
||||
|
||||
public FileInfo(long lasmodified, String name, long size, long checksum) {
|
||||
this.lastmodified = lasmodified;
|
||||
this.name = name;
|
||||
this.size = size;
|
||||
this.checksum = checksum;
|
||||
}
|
||||
|
||||
public Map<String, Object> getAsMap() {
|
||||
Map<String, Object> map = new HashMap<String, Object>();
|
||||
map.put(NAME, name);
|
||||
map.put(SIZE, size);
|
||||
map.put(LAST_MODIFIED, lastmodified);
|
||||
map.put(CHECKSUM, checksum);
|
||||
return map;
|
||||
}
|
||||
}
|
||||
|
||||
void disablePoll() {
|
||||
if (isSlave)
|
||||
snapPuller.disablePoll();
|
||||
}
|
||||
|
||||
void enablePoll() {
|
||||
if (isSlave)
|
||||
snapPuller.enablePoll();
|
||||
}
|
||||
|
||||
boolean isPollingDisabled() {
|
||||
return snapPuller.isPollingDisabled();
|
||||
}
|
||||
|
||||
int getTimesReplicatedSinceStartup() {
|
||||
return numTimesReplicated;
|
||||
}
|
||||
|
||||
void setTimesReplicatedSinceStartup() {
|
||||
numTimesReplicated++;
|
||||
}
|
||||
|
||||
long getIndexSize() {
|
||||
return computeIndexSize(new File(core.getIndexDir()));
|
||||
}
|
||||
|
||||
private long computeIndexSize(File f) {
|
||||
if (f.isFile())
|
||||
return f.length();
|
||||
File[] files = f.listFiles();
|
||||
long size = 0;
|
||||
if (files != null && files.length > 0) {
|
||||
for (File file : files) size += file.length();
|
||||
}
|
||||
return size;
|
||||
}
|
||||
|
||||
private Map<String, Object> getFileInfo(File file) {
|
||||
Map<String, Object> fileMeta = new HashMap<String, Object>();
|
||||
fileMeta.put(NAME, file.getName());
|
||||
fileMeta.put(SIZE, file.length());
|
||||
fileMeta.put(LAST_MODIFIED, file.lastModified());
|
||||
return fileMeta;
|
||||
}
|
||||
|
||||
public String getDescription() {
|
||||
return "";
|
||||
}
|
||||
|
||||
public String getSourceId() {
|
||||
return "";
|
||||
}
|
||||
|
||||
public String getSource() {
|
||||
return "";
|
||||
}
|
||||
|
||||
public String getVersion() {
|
||||
return "$Id$";
|
||||
}
|
||||
|
||||
String readableSize(long size) {
|
||||
NumberFormat formatter = NumberFormat.getNumberInstance();
|
||||
formatter.setMaximumFractionDigits(2);
|
||||
if (size / (1024 * 1024 * 1024) > 0) {
|
||||
return formatter.format(size * 1.0d / (1024 * 1024 * 1024)) + " GB";
|
||||
} else if (size / (1024 * 1024) > 0) {
|
||||
return formatter.format(size * 1.0d / (1024 * 1024)) + " MB";
|
||||
} else if (size / 1024 > 0) {
|
||||
return formatter.format(size * 1.0d / 1024) + " KB";
|
||||
} else {
|
||||
return String.valueOf(size) + " bytes";
|
||||
}
|
||||
}
|
||||
|
||||
private long[] getIndexVersion() {
|
||||
long version[] = new long[2];
|
||||
RefCounted<SolrIndexSearcher> searcher = core.getSearcher();
|
||||
try {
|
||||
version[0] = searcher.get().getReader().getIndexCommit().getVersion();
|
||||
version[1] = searcher.get().getReader().getIndexCommit().getGeneration();
|
||||
} catch (IOException e) {
|
||||
LOG.warn("Unable to get index version : ", e);
|
||||
} finally {
|
||||
searcher.decref();
|
||||
}
|
||||
return version;
|
||||
}
|
||||
|
||||
@Override
|
||||
@SuppressWarnings("unchecked")
|
||||
public NamedList getStatistics() {
|
||||
NamedList list = super.getStatistics();
|
||||
if (core != null) {
|
||||
list.add("indexSize", readableSize(getIndexSize()));
|
||||
long[] versionGen = getIndexVersion();
|
||||
list.add(CMD_INDEX_VERSION, versionGen[0]);
|
||||
list.add(GENERATION, versionGen[1]);
|
||||
|
||||
list.add("indexPath", core.getIndexDir());
|
||||
list.add("isMaster", String.valueOf(isMaster));
|
||||
|
||||
if (isSlave) {
|
||||
list.add(MASTER_URL, snapPuller.getMasterUrl());
|
||||
if (snapPuller.getPollInterval() != null) {
|
||||
list.add(SnapPuller.POLL_INTERVAL, snapPuller.getPollInterval());
|
||||
}
|
||||
list.add("isPollingDisabled", String.valueOf(isPollingDisabled()));
|
||||
list.add("isReplicating", String.valueOf(isReplicating()));
|
||||
}
|
||||
if (isMaster) {
|
||||
list.add("confFilesToReplicate", includeConfFiles.toString());
|
||||
if (replicateOnCommit)
|
||||
list.add(REPLICATE_AFTER, "commit");
|
||||
if (replicateOnOptimize)
|
||||
list.add(REPLICATE_AFTER, "optimize");
|
||||
}
|
||||
}
|
||||
return list;
|
||||
}
|
||||
|
||||
void getReplicationDetails(SolrQueryResponse resp) {
|
||||
String timeLastReplicated = "", confFilesReplicated = "", confFilesReplicatedTime = "", timesIndexReplicated = "", timesConfigReplicated = "";
|
||||
NamedList<Object> details = new SimpleOrderedMap<Object>();
|
||||
FileInputStream inFile = null;
|
||||
|
||||
details.add("indexSize", readableSize(getIndexSize()));
|
||||
details.add("indexPath", core.getIndexDir());
|
||||
details.add(CMD_SHOW_COMMITS, getCommits());
|
||||
details.add("isMaster", String.valueOf(isMaster));
|
||||
long[] versionAndGeneration = getIndexVersion();
|
||||
details.add(CMD_INDEX_VERSION, versionAndGeneration[0]);
|
||||
details.add(GENERATION, versionAndGeneration[1]);
|
||||
if (isMaster && indexCommitPoint != null) {
|
||||
details.add("replicatable" + CMD_INDEX_VERSION, indexCommitPoint.getVersion());
|
||||
details.add("replicatable" + GENERATION, indexCommitPoint.getGeneration());
|
||||
}
|
||||
|
||||
if (isSlave) {
|
||||
try {
|
||||
Properties props = new Properties();
|
||||
File f = new File(core.getDataDir(), SnapPuller.REPLICATION_PROPERTIES);
|
||||
if (f.exists()) {
|
||||
inFile = new FileInputStream(f);
|
||||
props.load(inFile);
|
||||
timeLastReplicated = props.getProperty("indexReplicatedAt");
|
||||
if (props.containsKey("timesIndexReplicated"))
|
||||
timesIndexReplicated = props.getProperty("timesIndexReplicated");
|
||||
if (props.containsKey("confFilesReplicated"))
|
||||
confFilesReplicated = props.getProperty("confFilesReplicated");
|
||||
if (props.containsKey("confFilesReplicatedAt"))
|
||||
confFilesReplicatedTime = props.getProperty("confFilesReplicatedAt");
|
||||
if (props.containsKey("timesConfigReplicated"))
|
||||
timesConfigReplicated = props.getProperty("timesConfigReplicated");
|
||||
}
|
||||
} catch (Exception e) {
|
||||
LOG.warn("Exception while reading " + SnapPuller.REPLICATION_PROPERTIES);
|
||||
} finally {
|
||||
closeNoExp(inFile);
|
||||
}
|
||||
|
||||
HttpClient client = null;
|
||||
try {
|
||||
client = new HttpClient();
|
||||
NamedList nl = snapPuller.getCommandResponse(client, CMD_DETAILS);
|
||||
details.add("masterDetails", nl.get(CMD_DETAILS));
|
||||
} catch (IOException e) {
|
||||
LOG.warn("Exception while invoking a 'details' method on master ", e);
|
||||
} finally {
|
||||
if (client != null)
|
||||
client.getHttpConnectionManager().closeIdleConnections(0);
|
||||
}
|
||||
details.add(MASTER_URL, snapPuller.getMasterUrl());
|
||||
if (snapPuller.getPollInterval() != null) {
|
||||
details.add(SnapPuller.POLL_INTERVAL, snapPuller.getPollInterval());
|
||||
}
|
||||
if (snapPuller.getNextScheduledExecTime() != null && !isPollingDisabled()) {
|
||||
Date d = new Date(snapPuller.getNextScheduledExecTime());
|
||||
details.add("nextExecutionAt", d.toString());
|
||||
} else if (isPollingDisabled()) {
|
||||
details.add("nextExecutionAt", "Polling disabled");
|
||||
} else
|
||||
details.add("nextExecutionAt", "");
|
||||
|
||||
if (timeLastReplicated != null && timeLastReplicated.length() > 0) {
|
||||
Date d = new Date(Long.valueOf(timeLastReplicated));
|
||||
details.add("indexReplicatedAt", d.toString());
|
||||
} else {
|
||||
details.add("indexReplicatedAt", "");
|
||||
}
|
||||
details.add("timesIndexReplicated", timesIndexReplicated);
|
||||
details.add("confFilesReplicated", confFilesReplicated);
|
||||
details.add("timesConfigReplicated", timesConfigReplicated);
|
||||
if (confFilesReplicatedTime != null && confFilesReplicatedTime.length() > 0) {
|
||||
Date d = new Date(Long.valueOf(confFilesReplicatedTime));
|
||||
details.add("confFilesReplicatedAt", d.toString());
|
||||
} else {
|
||||
details.add("confFilesReplicatedAt", confFilesReplicatedTime);
|
||||
}
|
||||
|
||||
try {
|
||||
long bytesToDownload = 0;
|
||||
List<String> filesToDownload = new ArrayList<String>();
|
||||
if (snapPuller.getFilesToDownload() != null) {
|
||||
for (Map<String, Object> file : snapPuller.getFilesToDownload()) {
|
||||
filesToDownload.add((String) file.get(NAME));
|
||||
bytesToDownload += (Long) file.get(SIZE);
|
||||
}
|
||||
}
|
||||
|
||||
//get list of conf files to download
|
||||
for (Map<String, Object> file : snapPuller.getConfFilesToDownload()) {
|
||||
filesToDownload.add((String) file.get(NAME));
|
||||
bytesToDownload += (Long) file.get(SIZE);
|
||||
}
|
||||
|
||||
details.add("filesToDownload", filesToDownload.toString());
|
||||
details.add("numFilesToDownload", String.valueOf(filesToDownload.size()));
|
||||
details.add("bytesToDownload", readableSize(bytesToDownload));
|
||||
|
||||
long bytesDownloaded = 0;
|
||||
List<String> filesDownloaded = new ArrayList<String>();
|
||||
for (Map<String, Object> file : snapPuller.getFilesDownloaded()) {
|
||||
filesDownloaded.add((String) file.get(NAME));
|
||||
bytesDownloaded += (Long) file.get(SIZE);
|
||||
}
|
||||
|
||||
//get list of conf files downloaded
|
||||
for (Map<String, Object> file : snapPuller.getConfFilesDownloaded()) {
|
||||
filesDownloaded.add((String) file.get(NAME));
|
||||
bytesDownloaded += (Long) file.get(SIZE);
|
||||
}
|
||||
|
||||
details.add("filesDownloaded", filesDownloaded.toString());
|
||||
details.add("numFilesDownloaded", String.valueOf(filesDownloaded.size()));
|
||||
|
||||
Map<String, Object> currentFile = snapPuller.getCurrentFile();
|
||||
String currFile = null;
|
||||
long currFileSize = 0, currFileSizeDownloaded = 0;
|
||||
float percentDownloaded = 0;
|
||||
if (currentFile != null) {
|
||||
currFile = (String) currentFile.get(NAME);
|
||||
currFileSize = (Long) currentFile.get(SIZE);
|
||||
if (currentFile.containsKey("bytesDownloaded")) {
|
||||
currFileSizeDownloaded = (Long) currentFile.get("bytesDownloaded");
|
||||
bytesDownloaded += currFileSizeDownloaded;
|
||||
if (currFileSize > 0)
|
||||
percentDownloaded = (currFileSizeDownloaded * 100) / currFileSize;
|
||||
}
|
||||
}
|
||||
|
||||
long timeElapsed = 0, estimatedTimeRemaining = 0;
|
||||
Date replicationStartTime = null;
|
||||
if (snapPuller.getReplicationStartTime() > 0) {
|
||||
replicationStartTime = new Date(snapPuller.getReplicationStartTime());
|
||||
timeElapsed = (System.currentTimeMillis() - snapPuller.getReplicationStartTime()) / 1000;
|
||||
}
|
||||
if (replicationStartTime != null) {
|
||||
details.add("replicationStartTime", replicationStartTime.toString());
|
||||
}
|
||||
details.add("timeElapsed", String.valueOf(timeElapsed) + "s");
|
||||
|
||||
if (bytesDownloaded > 0)
|
||||
estimatedTimeRemaining = ((bytesToDownload - bytesDownloaded) * timeElapsed) / bytesDownloaded;
|
||||
float totalPercent = 0;
|
||||
long downloadSpeed = 0;
|
||||
if (bytesToDownload > 0)
|
||||
totalPercent = (bytesDownloaded * 100) / bytesToDownload;
|
||||
if (timeElapsed > 0)
|
||||
downloadSpeed = (bytesDownloaded / timeElapsed);
|
||||
details.add("currentFile", currFile);
|
||||
details.add("currentFileSize", readableSize(currFileSize));
|
||||
details.add("currentFileSizeDownloaded", readableSize(currFileSizeDownloaded));
|
||||
details.add("currentFileSizePercent", String.valueOf(percentDownloaded));
|
||||
details.add("bytesDownloaded", readableSize(bytesDownloaded));
|
||||
details.add("totalPercent", String.valueOf(totalPercent));
|
||||
details.add("timeRemaining", String.valueOf(estimatedTimeRemaining) + "s");
|
||||
details.add("downloadSpeed", readableSize(downloadSpeed));
|
||||
details.add("isPollingDisabled", String.valueOf(isPollingDisabled()));
|
||||
details.add("isReplicating", String.valueOf(isReplicating()));
|
||||
} catch (Exception e) {
|
||||
LOG.error("Exception while writing details: ", e);
|
||||
}
|
||||
}
|
||||
|
||||
if (isMaster) {
|
||||
details.add(CONF_FILES, includeConfFiles.toString());
|
||||
if (replicateOnCommit)
|
||||
details.add(REPLICATE_AFTER, "commit");
|
||||
if (replicateOnOptimize)
|
||||
details.add(REPLICATE_AFTER, "optimize");
|
||||
}
|
||||
|
||||
resp.add(CMD_DETAILS, details);
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
public void inform(SolrCore core) {
|
||||
this.core = core;
|
||||
registerFileStreamResponseWriter();
|
||||
registerCloseHook();
|
||||
NamedList slave = (NamedList) initArgs.get("slave");
|
||||
if (slave != null) {
|
||||
snapPuller = new SnapPuller(slave, this, core);
|
||||
isSlave = true;
|
||||
}
|
||||
NamedList master = (NamedList) initArgs.get("master");
|
||||
if (master != null) {
|
||||
String includeFiles = (String) master.get(CONF_FILES);
|
||||
if (includeFiles != null && !includeFiles.trim().equals("")) {
|
||||
includeConfFiles = Arrays.asList(includeFiles.split(","));
|
||||
LOG.info("Replication enabled for following config files: " + includeConfFiles);
|
||||
}
|
||||
String snapshot = (String) master.get("snapshot");
|
||||
if ("optimize".equals(master.get(REPLICATE_AFTER))) {
|
||||
replicateOnOptimize = true;
|
||||
boolean snapshoot = "optimize".equals(snapshot);
|
||||
core.getUpdateHandler().registerOptimizeCallback(getEventListener(snapshoot));
|
||||
} else if ("commit".equals(master.get(REPLICATE_AFTER))) {
|
||||
replicateOnCommit = true;
|
||||
boolean snapshoot = "commit".equals(snapshot);
|
||||
core.getUpdateHandler().registerCommitCallback(getEventListener(snapshoot));
|
||||
}
|
||||
String reserve = (String) master.get(RESERVE);
|
||||
if (reserve != null && !reserve.trim().equals("")) {
|
||||
reserveCommitDuration = SnapPuller.readInterval(reserve);
|
||||
}
|
||||
LOG.info("Commits will be reserved for " + reserveCommitDuration);
|
||||
isMaster = true;
|
||||
}
|
||||
}
|
||||
|
||||
private void registerCloseHook() {
|
||||
core.addCloseHook(new CloseHook() {
|
||||
public void close(SolrCore core) {
|
||||
if (snapPuller != null) {
|
||||
snapPuller.destroy();
|
||||
}
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
private void registerFileStreamResponseWriter() {
|
||||
core.registerResponseWriter(FILE_STREAM, new BinaryQueryResponseWriter() {
|
||||
public void write(OutputStream out, SolrQueryRequest request, SolrQueryResponse resp) throws IOException {
|
||||
FileStream stream = (FileStream) resp.getValues().get(FILE_STREAM);
|
||||
stream.write(out);
|
||||
}
|
||||
|
||||
public void write(Writer writer, SolrQueryRequest request, SolrQueryResponse response) throws IOException {
|
||||
throw new RuntimeException("This is a binary writer , Cannot write to a characterstream");
|
||||
}
|
||||
|
||||
public String getContentType(SolrQueryRequest request, SolrQueryResponse response) {
|
||||
return "application/octet-stream";
|
||||
}
|
||||
|
||||
public void init(NamedList args) { /*no op*/ }
|
||||
});
|
||||
|
||||
}
|
||||
|
||||
private SolrEventListener getEventListener(final boolean snapshoot) {
|
||||
return new SolrEventListener() {
|
||||
public void init(NamedList args) {/*no op*/ }
|
||||
|
||||
public void postCommit() {
|
||||
indexCommitPoint = core.getDeletionPolicy().getLatestCommit();
|
||||
if (snapshoot) {
|
||||
try {
|
||||
SnapShooter snapShooter = new SnapShooter(core);
|
||||
snapShooter.createSnapAsync(core.getDeletionPolicy().getLatestCommit().getFileNames());
|
||||
} catch (Exception e) {
|
||||
LOG.error("Exception while snapshooting", e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public void newSearcher(SolrIndexSearcher newSearcher, SolrIndexSearcher currentSearcher) { /*no op*/}
|
||||
};
|
||||
}
|
||||
|
||||
static void closeNoExp(Closeable closeable) {
|
||||
try {
|
||||
if (closeable != null)
|
||||
closeable.close();
|
||||
} catch (Exception e) {/*no op*/ }
|
||||
}
|
||||
|
||||
private class FileStream {
|
||||
private SolrParams params;
|
||||
|
||||
private FastOutputStream fos;
|
||||
|
||||
public FileStream(SolrParams solrParams) {
|
||||
params = solrParams;
|
||||
}
|
||||
|
||||
public void write(OutputStream out) {
|
||||
fos = new FastOutputStream(out);
|
||||
String fileName = params.get(FILE);
|
||||
String cfileName = params.get(CONF_FILE_SHORT);
|
||||
String sOffset = params.get(OFFSET);
|
||||
String sLen = params.get(LEN);
|
||||
String sChecksum = params.get(CHECKSUM);
|
||||
FileInputStream inputStream = null;
|
||||
try {
|
||||
long offset = -1;
|
||||
int len = -1;
|
||||
boolean useChecksum = Boolean.parseBoolean(sChecksum);
|
||||
if (sOffset != null)
|
||||
offset = Long.parseLong(sOffset);
|
||||
if (sLen != null)
|
||||
len = Integer.parseInt(sLen);
|
||||
if (fileName == null && cfileName == null) {
|
||||
writeNothing();
|
||||
}
|
||||
|
||||
File file = null;
|
||||
if (cfileName != null) {
|
||||
file = new File(core.getResourceLoader().getConfigDir(), cfileName);
|
||||
} else {
|
||||
file = new File(core.getIndexDir(), fileName);
|
||||
}
|
||||
if (file.exists() && file.canRead()) {
|
||||
inputStream = new FileInputStream(file);
|
||||
FileChannel channel = inputStream.getChannel();
|
||||
if (offset != -1)
|
||||
channel.position(offset);
|
||||
byte[] buf = new byte[(len == -1 || len > PACKET_SZ) ? PACKET_SZ : len];
|
||||
Checksum checksum = null;
|
||||
if (useChecksum)
|
||||
checksum = new Adler32();
|
||||
ByteBuffer bb = ByteBuffer.wrap(buf);
|
||||
|
||||
while (true) {
|
||||
bb.clear();
|
||||
long bytesRead = channel.read(bb);
|
||||
if (bytesRead <= 0) {
|
||||
writeNothing();
|
||||
fos.close();
|
||||
break;
|
||||
}
|
||||
fos.writeInt((int) bytesRead);
|
||||
if (useChecksum) {
|
||||
checksum.reset();
|
||||
checksum.update(buf, 0, (int) bytesRead);
|
||||
fos.writeLong(checksum.getValue());
|
||||
}
|
||||
fos.write(buf, 0, (int) bytesRead);
|
||||
fos.flush();
|
||||
}
|
||||
} else {
|
||||
writeNothing();
|
||||
}
|
||||
} catch (IOException e) {
|
||||
LOG.warn("Exception while writing response for params: " + params, e);
|
||||
} finally {
|
||||
closeNoExp(inputStream);
|
||||
}
|
||||
}
|
||||
|
||||
private void writeNothing() throws IOException {
|
||||
fos.writeInt(0);
|
||||
fos.flush();
|
||||
}
|
||||
}
|
||||
|
||||
public static final String MASTER_URL = "masterUrl";
|
||||
|
||||
public static final String COMMAND = "command";
|
||||
|
||||
public static final String CMD_DETAILS = "details";
|
||||
|
||||
public static final String CMD_SNAP_SHOOT = "snapshoot";
|
||||
|
||||
public static final String CMD_SNAP_PULL = "snappull";
|
||||
|
||||
public static final String CMD_ABORT_SNAP_PULL = "abortsnappull";
|
||||
|
||||
public static final String CMD_GET_FILE_LIST = "filelist";
|
||||
|
||||
public static final String CMD_GET_FILE = "filecontent";
|
||||
|
||||
public static final String CMD_FILE_CHECKSUM = "filechecksum";
|
||||
|
||||
public static final String CMD_DISABLE_POLL = "disablepoll";
|
||||
|
||||
public static final String CMD_ENABLE_POLL = "enablepoll";
|
||||
|
||||
public static final String CMD_INDEX_VERSION = "indexversion";
|
||||
|
||||
public static final String CMD_SHOW_COMMITS = "commits";
|
||||
|
||||
public static final String GENERATION = "generation";
|
||||
|
||||
public static final String OFFSET = "offset";
|
||||
|
||||
public static final String LEN = "len";
|
||||
|
||||
public static final String FILE = "file";
|
||||
|
||||
public static final String NAME = "name";
|
||||
|
||||
public static final String SIZE = "size";
|
||||
|
||||
public static final String LAST_MODIFIED = "lastmodified";
|
||||
|
||||
public static final String CONF_FILE_SHORT = "cf";
|
||||
|
||||
public static final String CHECKSUM = "checksum";
|
||||
|
||||
public static final String CONF_CHECKSUM = "confchecksum";
|
||||
|
||||
public static final String CONF_FILES = "confFiles";
|
||||
|
||||
public static final String REPLICATE_AFTER = "replicateAfter";
|
||||
|
||||
public static final String FILE_STREAM = "filestream";
|
||||
|
||||
public static final int PACKET_SZ = 1024 * 1024; // 1MB
|
||||
|
||||
public static final String RESERVE = "commitReserveDuration";
|
||||
|
||||
}
|
|
@ -0,0 +1,889 @@
|
|||
/**
|
||||
* 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.solr.handler;
|
||||
|
||||
import org.apache.commons.httpclient.HttpClient;
|
||||
import org.apache.commons.httpclient.HttpStatus;
|
||||
import org.apache.commons.httpclient.MultiThreadedHttpConnectionManager;
|
||||
import org.apache.commons.httpclient.methods.PostMethod;
|
||||
import org.apache.lucene.index.IndexCommit;
|
||||
import org.apache.solr.common.SolrException;
|
||||
import org.apache.solr.common.util.FastInputStream;
|
||||
import org.apache.solr.common.util.NamedList;
|
||||
import org.apache.solr.common.util.NamedListCodec;
|
||||
import org.apache.solr.core.SolrCore;
|
||||
import static org.apache.solr.handler.ReplicationHandler.*;
|
||||
import org.apache.solr.search.SolrIndexSearcher;
|
||||
import org.apache.solr.update.CommitUpdateCommand;
|
||||
import org.apache.solr.util.RefCounted;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import java.io.*;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.nio.channels.FileChannel;
|
||||
import java.text.SimpleDateFormat;
|
||||
import java.util.*;
|
||||
import java.util.concurrent.Executors;
|
||||
import java.util.concurrent.ScheduledExecutorService;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
import java.util.regex.Matcher;
|
||||
import java.util.regex.Pattern;
|
||||
import java.util.zip.Adler32;
|
||||
import java.util.zip.Checksum;
|
||||
|
||||
/**
|
||||
* <p/> Provides functionality equivalent to the snappull script as well as a
|
||||
* timer for scheduling pulls from the master.
|
||||
* </p>
|
||||
*
|
||||
* @version $Id$
|
||||
* @since solr 1.4
|
||||
*/
|
||||
public class SnapPuller {
|
||||
private static final Logger LOG = LoggerFactory.getLogger(SnapPuller.class.getName());
|
||||
|
||||
private String masterUrl;
|
||||
|
||||
private ReplicationHandler replicationHandler;
|
||||
|
||||
private Integer pollInterval;
|
||||
|
||||
private String pollIntervalStr;
|
||||
|
||||
private ScheduledExecutorService executorService;
|
||||
|
||||
private long executorStartTime;
|
||||
|
||||
private long replicationStartTime;
|
||||
|
||||
private SolrCore solrCore;
|
||||
|
||||
private List<Map<String, Object>> filesToDownload;
|
||||
|
||||
private List<Map<String, Object>> confFilesToDownload;
|
||||
|
||||
private List<Map<String, Object>> filesDownloaded;
|
||||
|
||||
private List<Map<String, Object>> confFilesDownloaded;
|
||||
|
||||
private Map<String, Object> currentFile;
|
||||
|
||||
private FileFetcher fileFetcher;
|
||||
|
||||
private boolean stop = false;
|
||||
|
||||
/**
|
||||
* Disable the timer task for polling
|
||||
*/
|
||||
private AtomicBoolean pollDisabled = new AtomicBoolean(false);
|
||||
|
||||
public SnapPuller(NamedList initArgs, ReplicationHandler handler, SolrCore sc) {
|
||||
solrCore = sc;
|
||||
masterUrl = (String) initArgs.get(MASTER_URL);
|
||||
if (masterUrl == null)
|
||||
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
|
||||
"'masterUrl' is required for a slave");
|
||||
this.replicationHandler = handler;
|
||||
pollIntervalStr = (String) initArgs.get(POLL_INTERVAL);
|
||||
pollInterval = readInterval(pollIntervalStr);
|
||||
if (pollInterval != null && pollInterval > 0) {
|
||||
startExecutorService();
|
||||
} else {
|
||||
LOG.info(" No value set for 'pollInterval'. Timer Task not started.");
|
||||
}
|
||||
}
|
||||
|
||||
private void startExecutorService() {
|
||||
Runnable task = new Runnable() {
|
||||
public void run() {
|
||||
if (pollDisabled.get()) {
|
||||
LOG.info("Poll disabled");
|
||||
return;
|
||||
}
|
||||
try {
|
||||
executorStartTime = System.currentTimeMillis();
|
||||
replicationHandler.doSnapPull();
|
||||
} catch (Exception e) {
|
||||
LOG.error("Exception in pulling snapshot", e);
|
||||
}
|
||||
}
|
||||
};
|
||||
executorService = Executors.newSingleThreadScheduledExecutor();
|
||||
executorService.scheduleAtFixedRate(task, pollInterval, pollInterval, TimeUnit.MILLISECONDS);
|
||||
LOG.info("Poll Scheduled at an interval of " + pollInterval + "ms");
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the latest commit version and generation from the master
|
||||
*
|
||||
* @param client
|
||||
* @return
|
||||
* @throws IOException
|
||||
*/
|
||||
@SuppressWarnings("unchecked")
|
||||
NamedList getLatestVersion(HttpClient client) throws IOException {
|
||||
PostMethod post = new PostMethod(masterUrl);
|
||||
post.addParameter(COMMAND, CMD_INDEX_VERSION);
|
||||
post.addParameter("wt", "javabin");
|
||||
return getNamedListResponse(client, post);
|
||||
}
|
||||
|
||||
NamedList getCommandResponse(HttpClient client, String cmd) throws IOException {
|
||||
PostMethod post = new PostMethod(masterUrl);
|
||||
post.addParameter(COMMAND, cmd);
|
||||
post.addParameter("wt", "javabin");
|
||||
return getNamedListResponse(client, post);
|
||||
}
|
||||
|
||||
private NamedList getNamedListResponse(HttpClient client, PostMethod method)
|
||||
throws IOException {
|
||||
try {
|
||||
int status = client.executeMethod(method);
|
||||
if (status != HttpStatus.SC_OK) {
|
||||
throw new SolrException(SolrException.ErrorCode.SERVICE_UNAVAILABLE,
|
||||
"Request failed for the url " + method);
|
||||
}
|
||||
return new NamedListCodec().unmarshal(method.getResponseBodyAsStream());
|
||||
} finally {
|
||||
try {
|
||||
method.releaseConnection();
|
||||
} catch (Exception e) {
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Fetches the list of files in a given snapshot
|
||||
*
|
||||
* @param version
|
||||
* @param client
|
||||
* @return
|
||||
* @throws IOException
|
||||
*/
|
||||
void fetchFileList(long version, HttpClient client) throws IOException {
|
||||
PostMethod post = new PostMethod(masterUrl);
|
||||
post.addParameter(COMMAND, CMD_GET_FILE_LIST);
|
||||
post.addParameter(CMD_INDEX_VERSION, String.valueOf(version));
|
||||
post.addParameter("wt", "javabin");
|
||||
NamedList nl = getNamedListResponse(client, post);
|
||||
List<Map<String, Object>> f = (List<Map<String, Object>>) nl.get(CMD_GET_FILE_LIST);
|
||||
if (f != null)
|
||||
filesToDownload = Collections.synchronizedList(f);
|
||||
f = (List<Map<String, Object>>) nl.get(CONF_FILES);
|
||||
if (f != null)
|
||||
confFilesToDownload = Collections.synchronizedList(f);
|
||||
}
|
||||
|
||||
/**
|
||||
* This command downloads all the necessary files from master to install a
|
||||
* snapshot. Only changed files are downloaded.
|
||||
*
|
||||
* @param core the SolrCore
|
||||
* @return true on success, false if slave is already in sync
|
||||
* @throws IOException if an exception occurs
|
||||
*/
|
||||
@SuppressWarnings("unchecked")
|
||||
boolean fetchLatestIndex(SolrCore core) throws IOException {
|
||||
HttpClient client = null;
|
||||
replicationStartTime = System.currentTimeMillis();
|
||||
try {
|
||||
client = new HttpClient();
|
||||
client.setHttpConnectionManager(new MultiThreadedHttpConnectionManager());
|
||||
NamedList response = getLatestVersion(client);
|
||||
long latestVersion = (Long) response.get(CMD_INDEX_VERSION);
|
||||
long latestGeneration = (Long) response.get(GENERATION);
|
||||
if (latestVersion == 0L) {
|
||||
return false;
|
||||
}
|
||||
IndexCommit commit;
|
||||
RefCounted<SolrIndexSearcher> searcherRefCounted = null;
|
||||
try {
|
||||
searcherRefCounted = core.getSearcher();
|
||||
commit = searcherRefCounted.get().getReader().getIndexCommit();
|
||||
} finally {
|
||||
if (searcherRefCounted != null)
|
||||
searcherRefCounted.decref();
|
||||
}
|
||||
if (commit.getVersion() == latestVersion && commit.getGeneration() == latestGeneration) {
|
||||
LOG.info("Slave in sync with master.");
|
||||
return false;
|
||||
}
|
||||
LOG.info("Master's version: " + latestVersion + ", generation: " + latestGeneration);
|
||||
LOG.info("Slave's version: " + commit.getVersion() + ", generation: " + commit.getGeneration());
|
||||
LOG.info("Starting replication process");
|
||||
fetchFileList(latestVersion, client);
|
||||
LOG.info("Number of files in latest snapshot in master: " + filesToDownload.size());
|
||||
|
||||
filesDownloaded = Collections.synchronizedList(new ArrayList<Map<String, Object>>());
|
||||
boolean isSnapNeeded = commit.getGeneration() >= latestGeneration;
|
||||
File tmpIndexDir = createTempindexDir(core);
|
||||
if (isIndexStale())
|
||||
isSnapNeeded = true;
|
||||
boolean successfulInstall = false;
|
||||
try {
|
||||
File indexDir = new File(core.getIndexDir());
|
||||
downloadIndexFiles(isSnapNeeded, tmpIndexDir, client);
|
||||
LOG.info("Total time taken for download : " + ((System.currentTimeMillis() - replicationStartTime) / 1000) + " secs");
|
||||
Collection<Map<String, Object>> modifiedConfFiles = getModifiedConfFiles(confFilesToDownload);
|
||||
if (modifiedConfFiles != null && !modifiedConfFiles.isEmpty()) {
|
||||
downloadConfFiles(client, confFilesToDownload);
|
||||
if (isSnapNeeded) {
|
||||
modifyIndexProps(tmpIndexDir.getName());
|
||||
} else {
|
||||
successfulInstall = copyIndexFiles(tmpIndexDir, indexDir);
|
||||
}
|
||||
if (successfulInstall) {
|
||||
LOG.info("Configuration files are modified, core will be reloaded");
|
||||
logReplicationTimeAndConfFiles(modifiedConfFiles);//write to a file time of replication and conf files.
|
||||
reloadCore();
|
||||
}
|
||||
} else {
|
||||
LOG.info("Conf files are not downloaded or are in sync");
|
||||
if (isSnapNeeded) {
|
||||
modifyIndexProps(tmpIndexDir.getName());
|
||||
} else {
|
||||
successfulInstall = copyIndexFiles(tmpIndexDir, indexDir);
|
||||
}
|
||||
if (successfulInstall) {
|
||||
logReplicationTimeAndConfFiles(modifiedConfFiles);
|
||||
doCommit();
|
||||
}
|
||||
}
|
||||
replicationStartTime = 0;
|
||||
return successfulInstall;
|
||||
} catch (ReplicationHandlerException e) {
|
||||
delTree(tmpIndexDir);
|
||||
LOG.error("User aborted Replication");
|
||||
} catch (SolrException e) {
|
||||
delTree(tmpIndexDir);
|
||||
throw e;
|
||||
} catch (Exception e) {
|
||||
delTree(tmpIndexDir);
|
||||
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Snappull failed : ", e);
|
||||
}
|
||||
return successfulInstall;
|
||||
} finally {
|
||||
filesToDownload = filesDownloaded = confFilesDownloaded = confFilesToDownload = null;
|
||||
replicationStartTime = 0;
|
||||
fileFetcher = null;
|
||||
stop = false;
|
||||
client.getHttpConnectionManager().closeIdleConnections(0);
|
||||
}
|
||||
}
|
||||
|
||||
private void logReplicationTimeAndConfFiles(Collection<Map<String, Object>> modifiedConfFiles) {
|
||||
FileOutputStream outFile = null;
|
||||
FileInputStream inFile = null;
|
||||
List<String> confFiles = new ArrayList<String>();
|
||||
if (modifiedConfFiles != null && !modifiedConfFiles.isEmpty())
|
||||
for (Map<String, Object> map1 : modifiedConfFiles)
|
||||
confFiles.add((String) map1.get(NAME));
|
||||
|
||||
Properties props = new Properties();
|
||||
long replicationTime = System.currentTimeMillis();
|
||||
try {
|
||||
File f = new File(solrCore.getDataDir(), REPLICATION_PROPERTIES);
|
||||
int indexCount = 1, confFilesCount = 1;
|
||||
if (f.exists()) {
|
||||
inFile = new FileInputStream(f);
|
||||
props.load(inFile);
|
||||
if (!props.isEmpty()) {
|
||||
if (props.containsKey("timesIndexReplicated"))
|
||||
indexCount = Integer.valueOf(props.getProperty("timesIndexReplicated")) + 1;
|
||||
|
||||
props.setProperty("timesIndexReplicated", String.valueOf(indexCount));
|
||||
props.setProperty("indexReplicatedAt", String.valueOf(replicationTime));
|
||||
|
||||
if (modifiedConfFiles != null && !modifiedConfFiles.isEmpty()) {
|
||||
props.setProperty("confFilesReplicated", confFiles.toString());
|
||||
props.setProperty("confFilesReplicatedAt", String.valueOf(replicationTime));
|
||||
if (props.containsKey("timesConfigReplicated"))
|
||||
confFilesCount = Integer.valueOf(props.getProperty("timesConfigReplicated")) + 1;
|
||||
props.setProperty("timesConfigReplicated", String.valueOf(confFilesCount));
|
||||
}
|
||||
} else {
|
||||
props.setProperty("timesIndexReplicated", String.valueOf(indexCount));
|
||||
props.setProperty("indexReplicatedAt", String.valueOf(replicationTime));
|
||||
if (modifiedConfFiles != null && !modifiedConfFiles.isEmpty()) {
|
||||
props.setProperty("confFilesReplicated", confFiles.toString());
|
||||
props.setProperty("confFilesReplicatedAt", String.valueOf(replicationTime));
|
||||
props.setProperty("timesConfigReplicated", String.valueOf(confFilesCount));
|
||||
}
|
||||
}
|
||||
} else {
|
||||
props.setProperty("timesIndexReplicated", String.valueOf(indexCount));
|
||||
props.setProperty("indexReplicatedAt", String.valueOf(replicationTime));
|
||||
if (modifiedConfFiles != null && !modifiedConfFiles.isEmpty()) {
|
||||
props.setProperty("confFilesReplicated", confFiles.toString());
|
||||
props.setProperty("confFilesReplicatedAt", String.valueOf(replicationTime));
|
||||
props.setProperty("timesConfigReplicated", String.valueOf(confFilesCount));
|
||||
}
|
||||
}
|
||||
outFile = new FileOutputStream(f);
|
||||
props.store(outFile, "Replication details");
|
||||
} catch (Exception e) {
|
||||
LOG.warn("Exception while updating statistics", e);
|
||||
}
|
||||
finally {
|
||||
closeNoExp(inFile);
|
||||
closeNoExp(outFile);
|
||||
}
|
||||
}
|
||||
|
||||
private void doCommit() throws IOException {
|
||||
CommitUpdateCommand cmd = new CommitUpdateCommand(false);
|
||||
cmd.waitFlush = true;
|
||||
cmd.waitSearcher = true;
|
||||
solrCore.getUpdateHandler().commit(cmd);
|
||||
}
|
||||
|
||||
private File createTempindexDir(SolrCore core) {
|
||||
String snapName = "index."
|
||||
+ new SimpleDateFormat(SnapShooter.DATE_FMT).format(new Date());
|
||||
File snapDir = new File(core.getDataDir(), snapName);
|
||||
snapDir.mkdirs();
|
||||
return snapDir;
|
||||
}
|
||||
|
||||
private void reloadCore() {
|
||||
new Thread() {
|
||||
public void run() {
|
||||
try {
|
||||
solrCore.getCoreDescriptor().getCoreContainer().reload(solrCore.getName());
|
||||
} catch (Exception e) {
|
||||
LOG.error("Could not restart core ", e);
|
||||
}
|
||||
}
|
||||
}.start();
|
||||
}
|
||||
|
||||
private void downloadConfFiles(HttpClient client, List<Map<String, Object>> confFilesToDownload) throws Exception {
|
||||
LOG.info("Starting download of configuration files from master: " + confFilesToDownload);
|
||||
confFilesDownloaded = Collections.synchronizedList(new ArrayList<Map<String, Object>>());
|
||||
File tmpconfDir = new File(solrCore.getResourceLoader().getConfigDir(), "conf." + getDateAsStr(new Date()));
|
||||
boolean status = tmpconfDir.mkdirs();
|
||||
if (!status) {
|
||||
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
|
||||
"Failed to create temporary config folder: " + tmpconfDir.getName());
|
||||
}
|
||||
for (Map<String, Object> file : confFilesToDownload) {
|
||||
fileFetcher = new FileFetcher(tmpconfDir, file, (String) file.get(NAME), client, true);
|
||||
currentFile = file;
|
||||
fileFetcher.fetchFile();
|
||||
confFilesDownloaded.add(new HashMap<String, Object>(file));
|
||||
}
|
||||
copyTmpConfFiles2Conf(tmpconfDir);
|
||||
}
|
||||
|
||||
private void downloadIndexFiles(boolean snapNeeded, File snapDir,
|
||||
HttpClient client) throws Exception {
|
||||
for (Map<String, Object> file : filesToDownload) {
|
||||
File localIndexFile = new File(solrCore.getIndexDir(), (String) file.get(NAME));
|
||||
if (!localIndexFile.exists() || snapNeeded) {
|
||||
fileFetcher = new FileFetcher(snapDir, file, (String) file.get(NAME),
|
||||
client, false);
|
||||
currentFile = file;
|
||||
fileFetcher.fetchFile();
|
||||
filesDownloaded.add(new HashMap<String, Object>(file));
|
||||
} else {
|
||||
LOG.info("Skipping download for " + localIndexFile);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private boolean isIndexStale() {
|
||||
for (Map<String, Object> file : filesToDownload) {
|
||||
File localIndexFile = new File(solrCore.getIndexDir(), (String) file
|
||||
.get(NAME));
|
||||
if (localIndexFile.exists()
|
||||
&& localIndexFile.length() != (Long) file.get(SIZE)) {
|
||||
// file exists and size is different, therefore we must assume
|
||||
// corrupted index
|
||||
return true;
|
||||
}
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
private boolean copyAFile(File snapDir, File indexDir, String fname, List<String> copiedfiles) {
|
||||
File indexFileInSnap = new File(snapDir, fname);
|
||||
File indexFileInIndex = new File(indexDir, fname);
|
||||
boolean success = indexFileInSnap.renameTo(indexFileInIndex);
|
||||
if (!success) {
|
||||
LOG.error("Unable to move index file from: " + indexFileInSnap
|
||||
+ " to: " + indexFileInIndex);
|
||||
for (String f : copiedfiles) {
|
||||
File indexFile = new File(indexDir, f);
|
||||
if (indexFile.exists())
|
||||
indexFile.delete();
|
||||
}
|
||||
delTree(snapDir);
|
||||
return false;
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
private boolean copyIndexFiles(File snapDir, File indexDir) {
|
||||
String segmentsFile = null;
|
||||
List<String> copiedfiles = new ArrayList<String>();
|
||||
for (Map<String, Object> f : filesDownloaded) {
|
||||
String fname = (String) f.get(NAME);
|
||||
if (fname.startsWith("segments_")) {
|
||||
//The segments file must be copied in the end
|
||||
//Otherwise , if the copy fails index ends up corrupted
|
||||
segmentsFile = fname;
|
||||
continue;
|
||||
}
|
||||
if (!copyAFile(snapDir, indexDir, fname, copiedfiles)) return false;
|
||||
copiedfiles.add(fname);
|
||||
}
|
||||
if (segmentsFile != null) {
|
||||
if (!copyAFile(snapDir, indexDir, segmentsFile, copiedfiles)) return false;
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
private void copyTmpConfFiles2Conf(File tmpconfDir) throws IOException {
|
||||
File confDir = new File(solrCore.getResourceLoader().getConfigDir());
|
||||
try {
|
||||
for (File file : tmpconfDir.listFiles()) {
|
||||
File oldFile = new File(confDir, file.getName());
|
||||
if (oldFile.exists()) {
|
||||
File backupFile = new File(confDir, oldFile.getName() + "." + getDateAsStr(new Date(oldFile.lastModified())));
|
||||
boolean status = oldFile.renameTo(backupFile);
|
||||
if (!status) {
|
||||
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
|
||||
"Unable to rename: " + oldFile + " to: " + backupFile);
|
||||
}
|
||||
}
|
||||
boolean status = file.renameTo(oldFile);
|
||||
if (!status) {
|
||||
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
|
||||
"Unable to rename: " + file + " to: " + oldFile);
|
||||
}
|
||||
}
|
||||
} finally {
|
||||
delTree(tmpconfDir);
|
||||
}
|
||||
}
|
||||
|
||||
private String getDateAsStr(Date d) {
|
||||
return new SimpleDateFormat(SnapShooter.DATE_FMT).format(d);
|
||||
}
|
||||
|
||||
private void modifyIndexProps(String snap) {
|
||||
LOG.info("New index installed. Updating index properties...");
|
||||
File idxprops = new File(solrCore.getDataDir() + "index.properties");
|
||||
Properties p = new Properties();
|
||||
if (idxprops.exists()) {
|
||||
InputStream is = null;
|
||||
try {
|
||||
is = new FileInputStream(idxprops);
|
||||
p.load(is);
|
||||
} catch (Exception e) {
|
||||
LOG.error("Unable to load index.properties");
|
||||
} finally {
|
||||
closeNoExp(is);
|
||||
}
|
||||
}
|
||||
p.put("index", snap);
|
||||
FileOutputStream os = null;
|
||||
try {
|
||||
os = new FileOutputStream(idxprops);
|
||||
p.store(os, "index properties");
|
||||
} catch (Exception e) {
|
||||
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
|
||||
"Unable to write index.properties", e);
|
||||
} finally {
|
||||
closeNoExp(os);
|
||||
}
|
||||
}
|
||||
|
||||
private Collection<Map<String, Object>> getModifiedConfFiles(List<Map<String, Object>> confFilesToDownload) {
|
||||
if (confFilesToDownload == null || confFilesToDownload.isEmpty())
|
||||
return Collections.EMPTY_LIST;
|
||||
Map<String, Map<String, Object>> nameVsFile = new HashMap<String, Map<String, Object>>();
|
||||
for (Map<String, Object> map : confFilesToDownload) {
|
||||
nameVsFile.put((String) map.get(NAME), map);
|
||||
}
|
||||
List<Map<String, Object>> localFilesInfo = replicationHandler.getConfFileCache(nameVsFile.keySet());
|
||||
for (Map<String, Object> fileInfo : localFilesInfo) {
|
||||
String name = (String) fileInfo.get(NAME);
|
||||
Map<String, Object> m = nameVsFile.get(name);
|
||||
if (m == null) continue;
|
||||
if (m.get(CHECKSUM).equals(fileInfo.get(CHECKSUM))) {
|
||||
nameVsFile.remove(name);
|
||||
}
|
||||
}
|
||||
return nameVsFile.isEmpty() ? Collections.EMPTY_LIST : nameVsFile.values();
|
||||
}
|
||||
|
||||
static boolean delTree(File dir) {
|
||||
if (dir == null || !dir.exists())
|
||||
return false;
|
||||
File contents[] = dir.listFiles();
|
||||
if (contents != null) {
|
||||
for (File file : contents) {
|
||||
if (file.isDirectory()) {
|
||||
boolean success = delTree(file);
|
||||
if (!success)
|
||||
return false;
|
||||
} else {
|
||||
boolean success = file.delete();
|
||||
if (!success)
|
||||
return false;
|
||||
}
|
||||
}
|
||||
}
|
||||
return dir.delete();
|
||||
}
|
||||
|
||||
void disablePoll() {
|
||||
pollDisabled.set(true);
|
||||
LOG.info("inside disable poll, value of pollDisabled = " + pollDisabled);
|
||||
}
|
||||
|
||||
/**
|
||||
* Enable polling
|
||||
*/
|
||||
void enablePoll() {
|
||||
pollDisabled.set(false);
|
||||
LOG.info("inside enable poll, value of pollDisabled = " + pollDisabled);
|
||||
}
|
||||
|
||||
/**
|
||||
* Stops the ongoing pull
|
||||
*/
|
||||
void abortPull() {
|
||||
stop = true;
|
||||
}
|
||||
|
||||
long getReplicationStartTime() {
|
||||
return replicationStartTime;
|
||||
}
|
||||
|
||||
List<Map<String, Object>> getConfFilesToDownload() {
|
||||
//make a copy first because it can be null later
|
||||
List<Map<String, Object>> tmp = confFilesToDownload;
|
||||
return tmp == null ? Collections.EMPTY_LIST : new ArrayList<Map<String, Object>>(tmp);
|
||||
}
|
||||
|
||||
List<Map<String, Object>> getConfFilesDownloaded() {
|
||||
//make a copy first because it can be null later
|
||||
List<Map<String, Object>> tmp = confFilesDownloaded;
|
||||
return tmp == null ? Collections.EMPTY_LIST : new ArrayList<Map<String, Object>>(tmp);
|
||||
}
|
||||
|
||||
List<Map<String, Object>> getFilesToDownload() {
|
||||
//make a copy first because it can be null later
|
||||
List<Map<String, Object>> tmp = filesToDownload;
|
||||
return tmp == null ? Collections.EMPTY_LIST : new ArrayList<Map<String, Object>>(tmp);
|
||||
}
|
||||
|
||||
List<Map<String, Object>> getFilesDownloaded() {
|
||||
List<Map<String, Object>> tmp = filesDownloaded;
|
||||
return tmp == null ? Collections.EMPTY_LIST : new ArrayList<Map<String, Object>>(tmp);
|
||||
}
|
||||
|
||||
Map<String, Object> getCurrentFile() {
|
||||
Map<String, Object> tmp = currentFile;
|
||||
FileFetcher tmpFileFetcher = fileFetcher;
|
||||
if (tmp == null)
|
||||
return null;
|
||||
tmp = new HashMap<String, Object>(tmp);
|
||||
if (tmpFileFetcher != null)
|
||||
tmp.put("bytesDownloaded", tmpFileFetcher.bytesDownloaded);
|
||||
return tmp;
|
||||
}
|
||||
|
||||
boolean isPollingDisabled() {
|
||||
return pollDisabled.get();
|
||||
}
|
||||
|
||||
Long getNextScheduledExecTime() {
|
||||
Long nextTime = null;
|
||||
if (executorStartTime > 0)
|
||||
nextTime = executorStartTime + pollInterval;
|
||||
return nextTime;
|
||||
}
|
||||
|
||||
private class ReplicationHandlerException extends InterruptedException {
|
||||
|
||||
public ReplicationHandlerException(String message) {
|
||||
super(message);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
private class FileFetcher {
|
||||
boolean includeChecksum = true;
|
||||
|
||||
File snapDir;
|
||||
|
||||
String fileName;
|
||||
|
||||
String saveAs;
|
||||
|
||||
long size, lastmodified;
|
||||
|
||||
long bytesDownloaded = 0;
|
||||
|
||||
HttpClient client;
|
||||
|
||||
FileChannel fileChannel;
|
||||
|
||||
byte[] buf = new byte[1024 * 1024];
|
||||
|
||||
Checksum checksum;
|
||||
|
||||
File file;
|
||||
|
||||
int errorCount = 0;
|
||||
|
||||
private boolean isConf;
|
||||
|
||||
private PostMethod post;
|
||||
|
||||
private boolean aborted = false;
|
||||
|
||||
FileFetcher(File dir, Map<String, Object> fileDetails, String saveAs,
|
||||
HttpClient client, boolean isConf) throws FileNotFoundException {
|
||||
this.snapDir = dir;
|
||||
this.fileName = (String) fileDetails.get(NAME);
|
||||
this.size = (Long) fileDetails.get(SIZE);
|
||||
this.client = client;
|
||||
this.isConf = isConf;
|
||||
this.saveAs = saveAs;
|
||||
|
||||
this.file = new File(snapDir, saveAs);
|
||||
this.fileChannel = new FileOutputStream(file).getChannel();
|
||||
if (includeChecksum)
|
||||
checksum = new Adler32();
|
||||
}
|
||||
|
||||
void fetchFile() throws Exception {
|
||||
try {
|
||||
while (true) {
|
||||
final FastInputStream is = getStream();
|
||||
int result;
|
||||
try {
|
||||
result = fetchPackets(is);
|
||||
if (result == 0 || result == NO_CONTENT) {
|
||||
if (file.exists())
|
||||
file.setLastModified(lastmodified);
|
||||
return;
|
||||
}
|
||||
} finally {
|
||||
//closing Inputstream and HTTP connection takes a long time,
|
||||
// so replication status shows as 'replicating' even though it is aborted.
|
||||
new Thread() {
|
||||
public void run() {
|
||||
closeNoExp(is);
|
||||
try {
|
||||
if (post != null)
|
||||
post.releaseConnection();
|
||||
} catch (Exception e) {
|
||||
}
|
||||
}
|
||||
}.start();
|
||||
}
|
||||
}
|
||||
} finally {
|
||||
cleanup();
|
||||
}
|
||||
}
|
||||
|
||||
private int fetchPackets(FastInputStream fis) throws Exception {
|
||||
byte[] intbytes = new byte[4];
|
||||
byte[] longbytes = new byte[8];
|
||||
try {
|
||||
while (true) {
|
||||
if (stop) {
|
||||
stop = false;
|
||||
aborted = true;
|
||||
throw new ReplicationHandlerException("User aborted replication");
|
||||
}
|
||||
long checkSumServer = -1;
|
||||
fis.readFully(intbytes);
|
||||
int packetSize = readInt(intbytes);
|
||||
if (packetSize <= 0) {
|
||||
LOG.warn("No content recieved for file: " + currentFile);
|
||||
return NO_CONTENT;
|
||||
}
|
||||
if (buf.length < packetSize)
|
||||
buf = new byte[packetSize];
|
||||
if (checksum != null) {
|
||||
fis.readFully(longbytes);
|
||||
checkSumServer = readLong(longbytes);
|
||||
}
|
||||
fis.readFully(buf, 0, packetSize);
|
||||
if (includeChecksum) {
|
||||
checksum.reset();
|
||||
checksum.update(buf, 0, packetSize);
|
||||
long checkSumClient = checksum.getValue();
|
||||
if (checkSumClient != checkSumServer) {
|
||||
LOG.error("Checksum not matched between client and server for: " + currentFile);
|
||||
return 1;
|
||||
}
|
||||
}
|
||||
fileChannel.write(ByteBuffer.wrap(buf, 0, packetSize));
|
||||
bytesDownloaded += packetSize;
|
||||
if (bytesDownloaded >= size)
|
||||
return 0;
|
||||
errorCount = 0;
|
||||
}
|
||||
} catch (ReplicationHandlerException e) {
|
||||
throw e;
|
||||
} catch (Exception e) {
|
||||
LOG.warn("Error in fetching packets ", e);
|
||||
errorCount++;
|
||||
if (errorCount > MAX_RETRIES) {
|
||||
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
|
||||
"Snappull failed for file:" + fileName, e);
|
||||
}
|
||||
return ERR;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* The webcontainer flushes the data only after it fills the buffer size.
|
||||
* So, all data has to be read as readFully() other wise it fails. So read
|
||||
* everything as bytes and then extract int out of it
|
||||
*
|
||||
* @param b
|
||||
* @return
|
||||
*/
|
||||
private int readInt(byte[] b) {
|
||||
return (((b[0] & 0xff) << 24) | ((b[1] & 0xff) << 16)
|
||||
| ((b[2] & 0xff) << 8) | (b[3] & 0xff));
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* Same as above but to read long
|
||||
*
|
||||
* @param b
|
||||
* @return
|
||||
*/
|
||||
private long readLong(byte[] b) {
|
||||
return (((long) (b[0] & 0xff)) << 56) | (((long) (b[1] & 0xff)) << 48)
|
||||
| (((long) (b[2] & 0xff)) << 40) | (((long) (b[3] & 0xff)) << 32)
|
||||
| (((long) (b[4] & 0xff)) << 24) | ((b[5] & 0xff) << 16)
|
||||
| ((b[6] & 0xff) << 8) | ((b[7] & 0xff));
|
||||
|
||||
}
|
||||
|
||||
private void cleanup() {
|
||||
try {
|
||||
fileChannel.close();
|
||||
} catch (Exception e) {/* noop */
|
||||
}
|
||||
if (bytesDownloaded != size) {
|
||||
try {
|
||||
file.delete();
|
||||
} catch (Exception e) {
|
||||
LOG.error("Error deleting file in cleanup" + e.getMessage());
|
||||
}
|
||||
if (!aborted)
|
||||
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
|
||||
"Unable to download " + fileName + " completely. Downloaded "
|
||||
+ bytesDownloaded + "!=" + size);
|
||||
}
|
||||
}
|
||||
|
||||
FastInputStream getStream() throws IOException {
|
||||
post = new PostMethod(masterUrl);
|
||||
post.addParameter(COMMAND, CMD_GET_FILE);
|
||||
if (isConf) {
|
||||
post.addParameter(CONF_FILE_SHORT, fileName);
|
||||
} else {
|
||||
post.addParameter(FILE, fileName);
|
||||
}
|
||||
if (this.includeChecksum)
|
||||
post.addParameter(CHECKSUM, "true");
|
||||
post.addParameter("wt", FILE_STREAM);
|
||||
if (bytesDownloaded > 0) {
|
||||
post.addParameter(OFFSET, "" + bytesDownloaded);
|
||||
}
|
||||
client.executeMethod(post);
|
||||
InputStream is = post.getResponseBodyAsStream();
|
||||
return new FastInputStream(is);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
static Integer readInterval(String interval) {
|
||||
Pattern pattern = Pattern.compile(INTERVAL_PATTERN);
|
||||
if (interval == null)
|
||||
return null;
|
||||
int result = 0;
|
||||
if (interval != null) {
|
||||
Matcher m = pattern.matcher(interval.trim());
|
||||
if (m.find()) {
|
||||
String hr = m.group(1);
|
||||
String min = m.group(2);
|
||||
String sec = m.group(3);
|
||||
result = 0;
|
||||
try {
|
||||
if (sec != null && sec.length() > 0)
|
||||
result += Integer.parseInt(sec);
|
||||
if (min != null && min.length() > 0)
|
||||
result += (60 * Integer.parseInt(min));
|
||||
if (hr != null && hr.length() > 0)
|
||||
result += (60 * 60 * Integer.parseInt(hr));
|
||||
result *= 1000;
|
||||
} catch (NumberFormatException e) {
|
||||
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
|
||||
INTERVAL_ERR_MSG);
|
||||
}
|
||||
} else {
|
||||
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
|
||||
INTERVAL_ERR_MSG);
|
||||
}
|
||||
|
||||
}
|
||||
return result;
|
||||
}
|
||||
|
||||
public void destroy() {
|
||||
if (executorService != null) executorService.shutdown();
|
||||
}
|
||||
|
||||
String getMasterUrl() {
|
||||
return masterUrl;
|
||||
}
|
||||
|
||||
String getPollInterval() {
|
||||
return pollIntervalStr;
|
||||
}
|
||||
|
||||
private static final int MAX_RETRIES = 5;
|
||||
|
||||
private static final int NO_CONTENT = 1;
|
||||
|
||||
private static final int ERR = 2;
|
||||
|
||||
public static final String REPLICATION_PROPERTIES = "replication.properties";
|
||||
|
||||
public static final String POLL_INTERVAL = "pollInterval";
|
||||
|
||||
public static final String INTERVAL_ERR_MSG = "The " + POLL_INTERVAL + " must be in this format 'HH:mm:ss'";
|
||||
|
||||
private static final String INTERVAL_PATTERN = "(\\d*?):(\\d*?):(\\d*)";
|
||||
}
|
|
@ -0,0 +1,100 @@
|
|||
/**
|
||||
* 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.solr.handler;
|
||||
|
||||
import org.apache.solr.common.SolrException;
|
||||
import org.apache.solr.core.SolrCore;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.FileInputStream;
|
||||
import java.io.FileOutputStream;
|
||||
import java.io.IOException;
|
||||
import java.text.SimpleDateFormat;
|
||||
import java.util.Collection;
|
||||
import java.util.Date;
|
||||
import java.util.WeakHashMap;
|
||||
|
||||
/**
|
||||
* <p/>
|
||||
* Provides functionality equivalent to the snapshooter script
|
||||
* </p>
|
||||
*
|
||||
* @version $Id$
|
||||
* @since solr 1.4
|
||||
*/
|
||||
public class SnapShooter {
|
||||
private String snapDir = null;
|
||||
private SolrCore solrCore;
|
||||
|
||||
public SnapShooter(SolrCore core) {
|
||||
solrCore = core;
|
||||
}
|
||||
|
||||
void createSnapAsync(final Collection<String> files) {
|
||||
new Thread() {
|
||||
public void run() {
|
||||
createSnapshot(files);
|
||||
}
|
||||
}.start();
|
||||
}
|
||||
|
||||
void createSnapshot(Collection<String> files) {
|
||||
File lockFile = null;
|
||||
File snapShotDir = null;
|
||||
String directoryName = null;
|
||||
try {
|
||||
lockFile = new File(snapDir, directoryName + ".lock");
|
||||
if (lockFile.exists()) {
|
||||
return;
|
||||
}
|
||||
SimpleDateFormat fmt = new SimpleDateFormat(DATE_FMT);
|
||||
directoryName = "snapshot." + fmt.format(new Date());
|
||||
snapShotDir = new File(snapDir, directoryName);
|
||||
lockFile.createNewFile();
|
||||
snapShotDir.mkdir();
|
||||
for (String indexFile : files) {
|
||||
copyFile2Dir(new File(solrCore.getIndexDir(), indexFile), snapShotDir);
|
||||
}
|
||||
} catch (Exception e) {
|
||||
SnapPuller.delTree(snapShotDir);
|
||||
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
|
||||
} finally {
|
||||
if (lockFile != null) {
|
||||
lockFile.delete();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
static void copyFile2Dir(File file, File toDir) throws IOException {
|
||||
FileInputStream fis = null;
|
||||
FileOutputStream fos = null;
|
||||
try {
|
||||
fis = new FileInputStream(file);
|
||||
File destFile = new File(toDir, file.getName());
|
||||
fos = new FileOutputStream(destFile);
|
||||
fis.getChannel().transferTo(0, fis.available(), fos.getChannel());
|
||||
destFile.setLastModified(file.lastModified());
|
||||
} finally {
|
||||
ReplicationHandler.closeNoExp(fis);
|
||||
ReplicationHandler.closeNoExp(fos);
|
||||
}
|
||||
}
|
||||
|
||||
public static final String SNAP_DIR = "snapDir";
|
||||
public static final String DATE_FMT = "yyyyMMddhhmmss";
|
||||
private static WeakHashMap<SolrCore, SnapShooter> SNAP_DIRS = new WeakHashMap<SolrCore, SnapShooter>();
|
||||
}
|
|
@ -0,0 +1,259 @@
|
|||
/**
|
||||
* 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.solr.handler;
|
||||
|
||||
import junit.framework.TestCase;
|
||||
import org.apache.solr.TestDistributedSearch;
|
||||
import org.apache.solr.client.solrj.SolrServer;
|
||||
import org.apache.solr.client.solrj.SolrServerException;
|
||||
import org.apache.solr.client.solrj.embedded.JettySolrRunner;
|
||||
import org.apache.solr.client.solrj.impl.CommonsHttpSolrServer;
|
||||
import org.apache.solr.client.solrj.response.QueryResponse;
|
||||
import org.apache.solr.common.SolrDocument;
|
||||
import org.apache.solr.common.SolrDocumentList;
|
||||
import org.apache.solr.common.SolrInputDocument;
|
||||
import org.apache.solr.common.params.ModifiableSolrParams;
|
||||
import org.apache.solr.common.util.NamedList;
|
||||
import org.apache.solr.common.util.SimpleOrderedMap;
|
||||
import org.apache.solr.util.AbstractSolrTestCase;
|
||||
|
||||
import java.io.*;
|
||||
|
||||
/**
|
||||
* Test for ReplicationHandler
|
||||
*
|
||||
* @version $Id$
|
||||
* @since 1.4
|
||||
*/
|
||||
public class TestReplicationHandler extends TestCase {
|
||||
|
||||
JettySolrRunner masterJetty, slaveJetty;
|
||||
SolrServer masterClient, slaveClient;
|
||||
SolrInstance master = null, slave = null;
|
||||
|
||||
String context = "/solr";
|
||||
|
||||
public void setUp() throws Exception {
|
||||
master = new SolrInstance("master", 1);
|
||||
slave = new SolrInstance("slave", 0);
|
||||
master.setUp();
|
||||
slave.setUp();
|
||||
|
||||
masterJetty = createJetty(master, 9999);
|
||||
masterClient = createNewSolrServer(masterJetty.getLocalPort());
|
||||
|
||||
slaveJetty = createJetty(slave, 0);
|
||||
slaveClient = createNewSolrServer(slaveJetty.getLocalPort());
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public void tearDown() throws Exception {
|
||||
destroyServers();
|
||||
master.tearDown();
|
||||
slave.tearDown();
|
||||
}
|
||||
|
||||
private void destroyServers() throws Exception {
|
||||
masterJetty.stop();
|
||||
slaveJetty.stop();
|
||||
}
|
||||
|
||||
private JettySolrRunner createJetty(SolrInstance instance, int port) throws Exception {
|
||||
System.setProperty("solr.solr.home", instance.getHomeDir());
|
||||
System.setProperty("solr.data.dir", instance.getDataDir());
|
||||
|
||||
JettySolrRunner jetty = new JettySolrRunner("/solr", port);
|
||||
|
||||
jetty.start();
|
||||
return jetty;
|
||||
}
|
||||
|
||||
protected SolrServer createNewSolrServer(int port) {
|
||||
try {
|
||||
// setup the server...
|
||||
String url = "http://localhost:" + port + context;
|
||||
CommonsHttpSolrServer s = new CommonsHttpSolrServer(url);
|
||||
s.setDefaultMaxConnectionsPerHost(100);
|
||||
s.setMaxTotalConnections(100);
|
||||
return s;
|
||||
}
|
||||
catch (Exception ex) {
|
||||
throw new RuntimeException(ex);
|
||||
}
|
||||
}
|
||||
|
||||
int index(SolrServer s, Object... fields) throws Exception {
|
||||
SolrInputDocument doc = new SolrInputDocument();
|
||||
for (int i = 0; i < fields.length; i += 2) {
|
||||
doc.addField((String) (fields[i]), fields[i + 1]);
|
||||
}
|
||||
return s.add(doc).getStatus();
|
||||
}
|
||||
|
||||
NamedList query(String query, SolrServer s) throws SolrServerException {
|
||||
NamedList res = new SimpleOrderedMap();
|
||||
ModifiableSolrParams params = new ModifiableSolrParams();
|
||||
|
||||
params.add("q", query);
|
||||
|
||||
QueryResponse qres = s.query(params);
|
||||
|
||||
res = qres.getResponse();
|
||||
|
||||
return res;
|
||||
}
|
||||
|
||||
public void testIndexAndConfigReplication() throws Exception {
|
||||
|
||||
//add 500 docs to master
|
||||
for (int i = 0; i < 500; i++)
|
||||
index(masterClient, "id", i, "name", "name = " + String.valueOf(i));
|
||||
|
||||
masterClient.commit();
|
||||
|
||||
NamedList masterQueryRsp = query("*:*", masterClient);
|
||||
SolrDocumentList masterQueryResult = (SolrDocumentList) masterQueryRsp.get("response");
|
||||
assertEquals(500, masterQueryResult.getNumFound());
|
||||
|
||||
//sleep for pollinterval time, 4s for letting slave to pull data.
|
||||
Thread.sleep(4000);
|
||||
//get docs from slave and check equal to master
|
||||
NamedList slaveQueryRsp = query("*:*", slaveClient);
|
||||
SolrDocumentList slaveQueryResult = (SolrDocumentList) slaveQueryRsp.get("response");
|
||||
assertEquals(500, slaveQueryResult.getNumFound());
|
||||
|
||||
//compare results
|
||||
String cmp = TestDistributedSearch.compare(masterQueryResult, slaveQueryResult, 0, null);
|
||||
assertEquals(null, cmp);
|
||||
|
||||
//start config files replication test
|
||||
masterClient.deleteByQuery("*:*");
|
||||
masterClient.commit();
|
||||
|
||||
copyFile(new File("." + System.getProperty("file.separator") +
|
||||
"solr" + System.getProperty("file.separator") +
|
||||
"conf" + System.getProperty("file.separator") + "schema-replication2.xml"),
|
||||
new File(master.getConfDir(), "schema.xml"));
|
||||
|
||||
masterJetty.stop();
|
||||
|
||||
masterJetty = createJetty(master, 9999);
|
||||
masterClient = createNewSolrServer(masterJetty.getLocalPort());
|
||||
|
||||
//add a doc with new field and commit on master to trigger snappull from slave.
|
||||
index(masterClient, "id", "2000", "name", "name = " + String.valueOf(2000), "newname", "newname = " + String.valueOf(2000));
|
||||
masterClient.commit();
|
||||
|
||||
//sleep for 4s for replication to happen.
|
||||
Thread.sleep(4000);
|
||||
|
||||
slaveQueryRsp = query("*:*", slaveClient);
|
||||
SolrDocument d = ((SolrDocumentList) slaveQueryRsp.get("response")).get(0);
|
||||
assertEquals("newname = 2000", (String) d.getFieldValue("newname"));
|
||||
|
||||
}
|
||||
|
||||
void copyFile(File src, File dst) throws IOException {
|
||||
InputStream in = new FileInputStream(src);
|
||||
OutputStream out = new FileOutputStream(dst);
|
||||
|
||||
byte[] buf = new byte[1024];
|
||||
int len;
|
||||
while ((len = in.read(buf)) > 0)
|
||||
out.write(buf, 0, len);
|
||||
in.close();
|
||||
out.close();
|
||||
}
|
||||
|
||||
private class SolrInstance extends AbstractSolrTestCase {
|
||||
|
||||
String name;
|
||||
int type;
|
||||
File homeDir;
|
||||
File confDir;
|
||||
|
||||
public SolrInstance(String name, int type) {
|
||||
this.name = name;
|
||||
this.type = type;
|
||||
}
|
||||
|
||||
public String getHomeDir() {
|
||||
return homeDir.toString() + System.getProperty("file.separator");
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getSchemaFile() {
|
||||
return "." + System.getProperty("file.separator") + "solr" + System.getProperty("file.separator") + "conf" + System.getProperty("file.separator") + "schema-replication1.xml";
|
||||
}
|
||||
|
||||
public String getConfDir() {
|
||||
return confDir.toString() + System.getProperty("file.separator");
|
||||
}
|
||||
|
||||
public String getDataDir() {
|
||||
return dataDir.toString() + System.getProperty("file.separator");
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getSolrConfigFile() {
|
||||
String fname = "";
|
||||
if (type == 1)
|
||||
fname = "." + System.getProperty("file.separator") + "solr" + System.getProperty("file.separator") + "conf" + System.getProperty("file.separator") + "solrconfig-master.xml";
|
||||
if (type == 0)
|
||||
fname = "." + System.getProperty("file.separator") + "solr" + System.getProperty("file.separator") + "conf" + System.getProperty("file.separator") + "solrconfig-slave.xml";
|
||||
return fname;
|
||||
}
|
||||
|
||||
public void setUp() throws Exception {
|
||||
System.setProperty("solr.test.sys.prop1", "propone");
|
||||
System.setProperty("solr.test.sys.prop2", "proptwo");
|
||||
|
||||
String home = System.getProperty("java.io.tmpdir")
|
||||
+ System.getProperty("file.separator")
|
||||
+ getClass().getName() + "-" + System.currentTimeMillis() + System.getProperty("file.separator");
|
||||
|
||||
if (type == 1) {
|
||||
homeDir = new File(home + "master" + System.getProperty("file.separator"));
|
||||
dataDir = new File(home + "master" + System.getProperty("file.separator") + "data" + System.getProperty("file.separator"));
|
||||
confDir = new File(home + "master" + System.getProperty("file.separator") + "conf" + System.getProperty("file.separator"));
|
||||
}
|
||||
if (type == 0) {
|
||||
homeDir = new File(home + "slave" + System.getProperty("file.separator"));
|
||||
dataDir = new File(home + "slave" + System.getProperty("file.separator") + "data" + System.getProperty("file.separator"));
|
||||
confDir = new File(home + "slave" + System.getProperty("file.separator") + "conf" + System.getProperty("file.separator"));
|
||||
}
|
||||
|
||||
homeDir.mkdirs();
|
||||
dataDir.mkdirs();
|
||||
confDir.mkdirs();
|
||||
|
||||
File f = new File(confDir, "solrconfig.xml");
|
||||
copyFile(new File(getSolrConfigFile()), f);
|
||||
f = new File(confDir, "schema.xml");
|
||||
copyFile(new File(getSchemaFile()), f);
|
||||
}
|
||||
|
||||
public void tearDown() throws Exception {
|
||||
super.tearDown();
|
||||
AbstractSolrTestCase.recurseDelete(homeDir);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
|
|
@ -0,0 +1,462 @@
|
|||
<?xml version="1.0" ?>
|
||||
<!--
|
||||
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.
|
||||
-->
|
||||
|
||||
<!-- The Solr schema file. This file should be named "schema.xml" and
|
||||
should be located where the classloader for the Solr webapp can find it.
|
||||
|
||||
This schema is used for testing, and as such has everything and the
|
||||
kitchen sink thrown in. See example/solr/conf/schema.xml for a
|
||||
more concise example.
|
||||
|
||||
$Id$
|
||||
$Source$
|
||||
$Name$
|
||||
-->
|
||||
|
||||
<schema name="test" version="1.0">
|
||||
<types>
|
||||
|
||||
<!-- field type definitions... note that the "name" attribute is
|
||||
just a label to be used by field definitions. The "class"
|
||||
attribute and any other attributes determine the real type and
|
||||
behavior of the fieldtype.
|
||||
-->
|
||||
|
||||
<!-- numeric field types that store and index the text
|
||||
value verbatim (and hence don't sort correctly or support range queries.)
|
||||
These are provided more for backward compatability, allowing one
|
||||
to create a schema that matches an existing lucene index.
|
||||
-->
|
||||
<fieldType name="integer" class="solr.IntField"/>
|
||||
<fieldType name="long" class="solr.LongField"/>
|
||||
<fieldtype name="float" class="solr.FloatField"/>
|
||||
<fieldType name="double" class="solr.DoubleField"/>
|
||||
|
||||
<!-- numeric field types that manipulate the value into
|
||||
a string value that isn't human readable in it's internal form,
|
||||
but sorts correctly and supports range queries.
|
||||
|
||||
If sortMissingLast="true" then a sort on this field will cause documents
|
||||
without the field to come after documents with the field,
|
||||
regardless of the requested sort order.
|
||||
If sortMissingFirst="true" then a sort on this field will cause documents
|
||||
without the field to come before documents with the field,
|
||||
regardless of the requested sort order.
|
||||
If sortMissingLast="false" and sortMissingFirst="false" (the default),
|
||||
then default lucene sorting will be used which places docs without the field
|
||||
first in an ascending sort and last in a descending sort.
|
||||
-->
|
||||
<fieldtype name="sint" class="solr.SortableIntField" sortMissingLast="true"/>
|
||||
<fieldtype name="slong" class="solr.SortableLongField" sortMissingLast="true"/>
|
||||
<fieldtype name="sfloat" class="solr.SortableFloatField" sortMissingLast="true"/>
|
||||
<fieldtype name="sdouble" class="solr.SortableDoubleField" sortMissingLast="true"/>
|
||||
|
||||
<!-- bcd versions of sortable numeric type may provide smaller
|
||||
storage space and support very large numbers.
|
||||
-->
|
||||
<fieldtype name="bcdint" class="solr.BCDIntField" sortMissingLast="true"/>
|
||||
<fieldtype name="bcdlong" class="solr.BCDLongField" sortMissingLast="true"/>
|
||||
<fieldtype name="bcdstr" class="solr.BCDStrField" sortMissingLast="true"/>
|
||||
|
||||
<!-- Field type demonstrating an Analyzer failure -->
|
||||
<fieldtype name="failtype1" class="solr.TextField">
|
||||
<analyzer type="index">
|
||||
<tokenizer class="solr.WhitespaceTokenizerFactory"/>
|
||||
<filter class="solr.WordDelimiterFilterFactory" generateWordParts="1" generateNumberParts="0" catenateWords="0" catenateNumbers="0" catenateAll="0"/>
|
||||
<filter class="solr.LowerCaseFilterFactory"/>
|
||||
</analyzer>
|
||||
</fieldtype>
|
||||
|
||||
<!-- Demonstrating ignoreCaseChange -->
|
||||
<fieldtype name="wdf_nocase" class="solr.TextField">
|
||||
<analyzer>
|
||||
<tokenizer class="solr.WhitespaceTokenizerFactory"/>
|
||||
<filter class="solr.WordDelimiterFilterFactory" generateWordParts="1" generateNumberParts="0" catenateWords="0" catenateNumbers="0" catenateAll="0" splitOnCaseChange="0" preserveOriginal="0"/>
|
||||
<filter class="solr.LowerCaseFilterFactory"/>
|
||||
</analyzer>
|
||||
</fieldtype>
|
||||
|
||||
<fieldtype name="wdf_preserve" class="solr.TextField">
|
||||
<analyzer>
|
||||
<tokenizer class="solr.WhitespaceTokenizerFactory"/>
|
||||
<filter class="solr.WordDelimiterFilterFactory" generateWordParts="0" generateNumberParts="1" catenateWords="0" catenateNumbers="0" catenateAll="0" splitOnCaseChange="0" preserveOriginal="1"/>
|
||||
<filter class="solr.LowerCaseFilterFactory"/>
|
||||
</analyzer>
|
||||
</fieldtype>
|
||||
|
||||
|
||||
<!-- HighlitText optimizes storage for (long) columns which will be highlit -->
|
||||
<fieldtype name="highlittext" class="solr.TextField" compressThreshold="345" />
|
||||
|
||||
<fieldtype name="boolean" class="solr.BoolField" sortMissingLast="true"/>
|
||||
<fieldtype name="string" class="solr.StrField" sortMissingLast="true"/>
|
||||
|
||||
<!-- format for date is 1995-12-31T23:59:59.999Z and only the fractional
|
||||
seconds part (.999) is optional.
|
||||
-->
|
||||
<fieldtype name="date" class="solr.DateField" sortMissingLast="true"/>
|
||||
|
||||
<!-- solr.TextField allows the specification of custom
|
||||
text analyzers specified as a tokenizer and a list
|
||||
of token filters.
|
||||
-->
|
||||
<fieldtype name="text" class="solr.TextField">
|
||||
<analyzer>
|
||||
<tokenizer class="solr.StandardTokenizerFactory"/>
|
||||
<filter class="solr.StandardFilterFactory"/>
|
||||
<filter class="solr.LowerCaseFilterFactory"/>
|
||||
<filter class="solr.StopFilterFactory"/>
|
||||
<!-- lucene PorterStemFilterFactory deprecated
|
||||
<filter class="solr.PorterStemFilterFactory"/>
|
||||
-->
|
||||
<filter class="solr.EnglishPorterFilterFactory"/>
|
||||
</analyzer>
|
||||
</fieldtype>
|
||||
|
||||
|
||||
<fieldtype name="nametext" class="solr.TextField">
|
||||
<analyzer class="org.apache.lucene.analysis.WhitespaceAnalyzer"/>
|
||||
</fieldtype>
|
||||
|
||||
<fieldtype name="teststop" class="solr.TextField">
|
||||
<analyzer>
|
||||
<tokenizer class="solr.LowerCaseTokenizerFactory"/>
|
||||
<filter class="solr.StandardFilterFactory"/>
|
||||
<!--filter class="solr.StopFilterFactory" words="stopwords.txt"/-->
|
||||
</analyzer>
|
||||
</fieldtype>
|
||||
|
||||
<!-- fieldtypes in this section isolate tokenizers and tokenfilters for testing -->
|
||||
<fieldtype name="lowertok" class="solr.TextField">
|
||||
<analyzer><tokenizer class="solr.LowerCaseTokenizerFactory"/></analyzer>
|
||||
</fieldtype>
|
||||
<fieldtype name="keywordtok" class="solr.TextField">
|
||||
<analyzer><tokenizer class="solr.KeywordTokenizerFactory"/></analyzer>
|
||||
</fieldtype>
|
||||
<fieldtype name="standardtok" class="solr.TextField">
|
||||
<analyzer><tokenizer class="solr.StandardTokenizerFactory"/></analyzer>
|
||||
</fieldtype>
|
||||
<fieldtype name="lettertok" class="solr.TextField">
|
||||
<analyzer><tokenizer class="solr.LetterTokenizerFactory"/></analyzer>
|
||||
</fieldtype>
|
||||
<fieldtype name="whitetok" class="solr.TextField">
|
||||
<analyzer><tokenizer class="solr.WhitespaceTokenizerFactory"/></analyzer>
|
||||
</fieldtype>
|
||||
<fieldtype name="HTMLstandardtok" class="solr.TextField">
|
||||
<analyzer><tokenizer class="solr.HTMLStripStandardTokenizerFactory"/></analyzer>
|
||||
</fieldtype>
|
||||
<fieldtype name="HTMLwhitetok" class="solr.TextField">
|
||||
<analyzer><tokenizer class="solr.HTMLStripWhitespaceTokenizerFactory"/></analyzer>
|
||||
</fieldtype>
|
||||
<fieldtype name="standardtokfilt" class="solr.TextField">
|
||||
<analyzer>
|
||||
<tokenizer class="solr.StandardTokenizerFactory"/>
|
||||
<filter class="solr.StandardFilterFactory"/>
|
||||
</analyzer>
|
||||
</fieldtype>
|
||||
<fieldtype name="standardfilt" class="solr.TextField">
|
||||
<analyzer>
|
||||
<tokenizer class="solr.WhitespaceTokenizerFactory"/>
|
||||
<filter class="solr.StandardFilterFactory"/>
|
||||
</analyzer>
|
||||
</fieldtype>
|
||||
<fieldtype name="lowerfilt" class="solr.TextField">
|
||||
<analyzer>
|
||||
<tokenizer class="solr.WhitespaceTokenizerFactory"/>
|
||||
<filter class="solr.LowerCaseFilterFactory"/>
|
||||
</analyzer>
|
||||
</fieldtype>
|
||||
<fieldtype name="patternreplacefilt" class="solr.TextField">
|
||||
<analyzer type="index">
|
||||
<tokenizer class="solr.KeywordTokenizerFactory"/>
|
||||
<filter class="solr.PatternReplaceFilterFactory"
|
||||
pattern="([^a-zA-Z])" replacement="_" replace="all"
|
||||
/>
|
||||
</analyzer>
|
||||
<analyzer type="query">
|
||||
<tokenizer class="solr.KeywordTokenizerFactory"/>
|
||||
</analyzer>
|
||||
</fieldtype>
|
||||
<fieldtype name="porterfilt" class="solr.TextField">
|
||||
<analyzer>
|
||||
<tokenizer class="solr.WhitespaceTokenizerFactory"/>
|
||||
<filter class="solr.PorterStemFilterFactory"/>
|
||||
</analyzer>
|
||||
</fieldtype>
|
||||
<!-- fieldtype name="snowballfilt" class="solr.TextField">
|
||||
<analyzer>
|
||||
<tokenizer class="solr.WhitespaceTokenizerFactory"/>
|
||||
<filter class="solr.SnowballPorterFilterFactory"/>
|
||||
</analyzer>
|
||||
</fieldtype -->
|
||||
<fieldtype name="engporterfilt" class="solr.TextField">
|
||||
<analyzer>
|
||||
<tokenizer class="solr.WhitespaceTokenizerFactory"/>
|
||||
<filter class="solr.EnglishPorterFilterFactory"/>
|
||||
</analyzer>
|
||||
</fieldtype>
|
||||
<fieldtype name="custengporterfilt" class="solr.TextField">
|
||||
<analyzer>
|
||||
<tokenizer class="solr.WhitespaceTokenizerFactory"/>
|
||||
<!--filter class="solr.EnglishPorterFilterFactory" protected="protwords.txt"/-->
|
||||
</analyzer>
|
||||
</fieldtype>
|
||||
<fieldtype name="stopfilt" class="solr.TextField">
|
||||
<analyzer>
|
||||
<tokenizer class="solr.WhitespaceTokenizerFactory"/>
|
||||
<filter class="solr.StopFilterFactory" ignoreCase="true"/>
|
||||
</analyzer>
|
||||
</fieldtype>
|
||||
<fieldtype name="custstopfilt" class="solr.TextField">
|
||||
<analyzer>
|
||||
<tokenizer class="solr.WhitespaceTokenizerFactory"/>
|
||||
<!--filter class="solr.StopFilterFactory" words="stopwords.txt"/-->
|
||||
</analyzer>
|
||||
</fieldtype>
|
||||
<fieldtype name="lengthfilt" class="solr.TextField">
|
||||
<analyzer>
|
||||
<tokenizer class="solr.WhitespaceTokenizerFactory"/>
|
||||
<filter class="solr.LengthFilterFactory" min="2" max="5"/>
|
||||
</analyzer>
|
||||
</fieldtype>
|
||||
|
||||
<fieldtype name="subword" class="solr.TextField" multiValued="true" positionIncrementGap="100">
|
||||
<analyzer type="index">
|
||||
<tokenizer class="solr.WhitespaceTokenizerFactory"/>
|
||||
<filter class="solr.WordDelimiterFilterFactory" generateWordParts="1" generateNumberParts="1" catenateWords="1" catenateNumbers="1" catenateAll="0"/>
|
||||
<filter class="solr.LowerCaseFilterFactory"/>
|
||||
<filter class="solr.StopFilterFactory"/>
|
||||
<filter class="solr.EnglishPorterFilterFactory"/>
|
||||
</analyzer>
|
||||
<analyzer type="query">
|
||||
<tokenizer class="solr.WhitespaceTokenizerFactory"/>
|
||||
<filter class="solr.WordDelimiterFilterFactory" generateWordParts="1" generateNumberParts="1" catenateWords="0" catenateNumbers="0" catenateAll="0"/>
|
||||
<filter class="solr.LowerCaseFilterFactory"/>
|
||||
<filter class="solr.StopFilterFactory"/>
|
||||
<filter class="solr.EnglishPorterFilterFactory"/>
|
||||
</analyzer>
|
||||
</fieldtype>
|
||||
|
||||
<!-- more flexible in matching skus, but more chance of a false match -->
|
||||
<fieldtype name="skutype1" class="solr.TextField">
|
||||
<analyzer type="index">
|
||||
<tokenizer class="solr.WhitespaceTokenizerFactory"/>
|
||||
<filter class="solr.WordDelimiterFilterFactory" generateWordParts="1" generateNumberParts="1" catenateWords="1" catenateNumbers="1" catenateAll="0"/>
|
||||
<filter class="solr.LowerCaseFilterFactory"/>
|
||||
</analyzer>
|
||||
<analyzer type="query">
|
||||
<tokenizer class="solr.WhitespaceTokenizerFactory"/>
|
||||
<filter class="solr.WordDelimiterFilterFactory" generateWordParts="0" generateNumberParts="0" catenateWords="1" catenateNumbers="1" catenateAll="0"/>
|
||||
<filter class="solr.LowerCaseFilterFactory"/>
|
||||
</analyzer>
|
||||
</fieldtype>
|
||||
|
||||
<!-- less flexible in matching skus, but less chance of a false match -->
|
||||
<fieldtype name="skutype2" class="solr.TextField">
|
||||
<analyzer type="index">
|
||||
<tokenizer class="solr.WhitespaceTokenizerFactory"/>
|
||||
<filter class="solr.WordDelimiterFilterFactory" generateWordParts="0" generateNumberParts="0" catenateWords="1" catenateNumbers="1" catenateAll="0"/>
|
||||
<filter class="solr.LowerCaseFilterFactory"/>
|
||||
</analyzer>
|
||||
<analyzer type="query">
|
||||
<tokenizer class="solr.WhitespaceTokenizerFactory"/>
|
||||
<filter class="solr.WordDelimiterFilterFactory" generateWordParts="0" generateNumberParts="0" catenateWords="1" catenateNumbers="1" catenateAll="0"/>
|
||||
<filter class="solr.LowerCaseFilterFactory"/>
|
||||
</analyzer>
|
||||
</fieldtype>
|
||||
|
||||
<!-- less flexible in matching skus, but less chance of a false match -->
|
||||
<fieldtype name="syn" class="solr.TextField">
|
||||
<analyzer>
|
||||
<tokenizer class="solr.WhitespaceTokenizerFactory"/>
|
||||
<!--filter name="syn" class="solr.SynonymFilterFactory" synonyms="synonyms.txt"/-->
|
||||
</analyzer>
|
||||
</fieldtype>
|
||||
|
||||
<!-- Demonstrates How RemoveDuplicatesTokenFilter makes stemmed
|
||||
synonyms "better"
|
||||
-->
|
||||
<fieldtype name="dedup" class="solr.TextField">
|
||||
<analyzer>
|
||||
<tokenizer class="solr.WhitespaceTokenizerFactory"/>
|
||||
<!--filter class="solr.SynonymFilterFactory"
|
||||
synonyms="synonyms.txt" expand="true" /-->
|
||||
<filter class="solr.EnglishPorterFilterFactory"/>
|
||||
<filter class="solr.RemoveDuplicatesTokenFilterFactory" />
|
||||
</analyzer>
|
||||
</fieldtype>
|
||||
|
||||
<fieldtype name="unstored" class="solr.StrField" indexed="true" stored="false"/>
|
||||
|
||||
|
||||
<fieldtype name="textgap" class="solr.TextField" multiValued="true" positionIncrementGap="100">
|
||||
<analyzer>
|
||||
<tokenizer class="solr.WhitespaceTokenizerFactory"/>
|
||||
<filter class="solr.LowerCaseFilterFactory"/>
|
||||
</analyzer>
|
||||
</fieldtype>
|
||||
|
||||
</types>
|
||||
|
||||
|
||||
<fields>
|
||||
<field name="id" type="integer" indexed="true" stored="true" multiValued="false" required="false"/>
|
||||
<field name="name" type="nametext" indexed="true" stored="true"/>
|
||||
<field name="text" type="text" indexed="true" stored="false"/>
|
||||
<field name="subject" type="text" indexed="true" stored="true"/>
|
||||
<field name="title" type="nametext" indexed="true" stored="true"/>
|
||||
<field name="weight" type="float" indexed="true" stored="true"/>
|
||||
<field name="bday" type="date" indexed="true" stored="true"/>
|
||||
|
||||
<field name="title_stemmed" type="text" indexed="true" stored="false"/>
|
||||
<field name="title_lettertok" type="lettertok" indexed="true" stored="false"/>
|
||||
|
||||
<field name="syn" type="syn" indexed="true" stored="true"/>
|
||||
|
||||
<!-- to test property inheritance and overriding -->
|
||||
<field name="shouldbeunstored" type="unstored" />
|
||||
<field name="shouldbestored" type="unstored" stored="true"/>
|
||||
<field name="shouldbeunindexed" type="unstored" indexed="false" stored="true"/>
|
||||
|
||||
|
||||
<!-- test different combinations of indexed and stored -->
|
||||
<field name="bind" type="boolean" indexed="true" stored="false"/>
|
||||
<field name="bsto" type="boolean" indexed="false" stored="true"/>
|
||||
<field name="bindsto" type="boolean" indexed="true" stored="true"/>
|
||||
<field name="isto" type="integer" indexed="false" stored="true"/>
|
||||
<field name="iind" type="integer" indexed="true" stored="false"/>
|
||||
<field name="ssto" type="string" indexed="false" stored="true"/>
|
||||
<field name="sind" type="string" indexed="true" stored="false"/>
|
||||
<field name="sindsto" type="string" indexed="true" stored="true"/>
|
||||
|
||||
<!-- test combinations of term vector settings -->
|
||||
<field name="test_basictv" type="text" termVectors="true"/>
|
||||
<field name="test_notv" type="text" termVectors="false"/>
|
||||
<field name="test_postv" type="text" termVectors="true" termPositions="true"/>
|
||||
<field name="test_offtv" type="text" termVectors="true" termOffsets="true"/>
|
||||
<field name="test_posofftv" type="text" termVectors="true"
|
||||
termPositions="true" termOffsets="true"/>
|
||||
|
||||
<!-- test highlit field settings -->
|
||||
<field name="test_hlt" type="highlittext" indexed="true" compressed="true"/>
|
||||
<field name="test_hlt_off" type="highlittext" indexed="true" compressed="false"/>
|
||||
|
||||
<!-- fields to test individual tokenizers and tokenfilters -->
|
||||
<field name="teststop" type="teststop" indexed="true" stored="true"/>
|
||||
<field name="lowertok" type="lowertok" indexed="true" stored="true"/>
|
||||
<field name="keywordtok" type="keywordtok" indexed="true" stored="true"/>
|
||||
<field name="standardtok" type="standardtok" indexed="true" stored="true"/>
|
||||
<field name="HTMLstandardtok" type="HTMLstandardtok" indexed="true" stored="true"/>
|
||||
<field name="lettertok" type="lettertok" indexed="true" stored="true"/>
|
||||
<field name="whitetok" type="whitetok" indexed="true" stored="true"/>
|
||||
<field name="HTMLwhitetok" type="HTMLwhitetok" indexed="true" stored="true"/>
|
||||
<field name="standardtokfilt" type="standardtokfilt" indexed="true" stored="true"/>
|
||||
<field name="standardfilt" type="standardfilt" indexed="true" stored="true"/>
|
||||
<field name="lowerfilt" type="lowerfilt" indexed="true" stored="true"/>
|
||||
<field name="patternreplacefilt" type="patternreplacefilt" indexed="true" stored="true"/>
|
||||
<field name="porterfilt" type="porterfilt" indexed="true" stored="true"/>
|
||||
<field name="engporterfilt" type="engporterfilt" indexed="true" stored="true"/>
|
||||
<field name="custengporterfilt" type="custengporterfilt" indexed="true" stored="true"/>
|
||||
<field name="stopfilt" type="stopfilt" indexed="true" stored="true"/>
|
||||
<field name="custstopfilt" type="custstopfilt" indexed="true" stored="true"/>
|
||||
<field name="lengthfilt" type="lengthfilt" indexed="true" stored="true"/>
|
||||
<field name="dedup" type="dedup" indexed="true" stored="true"/>
|
||||
<field name="wdf_nocase" type="wdf_nocase" indexed="true" stored="true"/>
|
||||
<field name="wdf_preserve" type="wdf_preserve" indexed="true" stored="true"/>
|
||||
|
||||
<field name="numberpartfail" type="failtype1" indexed="true" stored="true"/>
|
||||
|
||||
<field name="nullfirst" type="string" indexed="true" stored="true" sortMissingFirst="true"/>
|
||||
|
||||
<field name="subword" type="subword" indexed="true" stored="true"/>
|
||||
<field name="sku1" type="skutype1" indexed="true" stored="true"/>
|
||||
<field name="sku2" type="skutype2" indexed="true" stored="true"/>
|
||||
|
||||
<field name="textgap" type="textgap" indexed="true" stored="true"/>
|
||||
|
||||
<field name="timestamp" type="date" indexed="true" stored="true" default="NOW" multiValued="false"/>
|
||||
<field name="multiDefault" type="string" indexed="true" stored="true" default="muLti-Default" multiValued="true"/>
|
||||
<field name="intDefault" type="sint" indexed="true" stored="true" default="42" multiValued="false"/>
|
||||
|
||||
|
||||
|
||||
<!-- Dynamic field definitions. If a field name is not found, dynamicFields
|
||||
will be used if the name matches any of the patterns.
|
||||
RESTRICTION: the glob-like pattern in the name attribute must have
|
||||
a "*" only at the start or the end.
|
||||
EXAMPLE: name="*_i" will match any field ending in _i (like myid_i, z_i)
|
||||
Longer patterns will be matched first. if equal size patterns
|
||||
both match, the first appearing in the schema will be used.
|
||||
-->
|
||||
<dynamicField name="*_i" type="sint" indexed="true" stored="true"/>
|
||||
<dynamicField name="*_s" type="string" indexed="true" stored="true"/>
|
||||
<dynamicField name="*_s1" type="string" indexed="true" stored="true" multiValued="false"/>
|
||||
<dynamicField name="*_l" type="slong" indexed="true" stored="true"/>
|
||||
<dynamicField name="*_t" type="text" indexed="true" stored="true"/>
|
||||
<dynamicField name="*_b" type="boolean" indexed="true" stored="true"/>
|
||||
<dynamicField name="*_f" type="sfloat" indexed="true" stored="true"/>
|
||||
<dynamicField name="*_d" type="sdouble" indexed="true" stored="true"/>
|
||||
<dynamicField name="*_dt" type="date" indexed="true" stored="true"/>
|
||||
<dynamicField name="*_bcd" type="bcdstr" indexed="true" stored="true"/>
|
||||
|
||||
<dynamicField name="*_sI" type="string" indexed="true" stored="false"/>
|
||||
<dynamicField name="*_sS" type="string" indexed="false" stored="true"/>
|
||||
<dynamicField name="t_*" type="text" indexed="true" stored="true"/>
|
||||
<dynamicField name="tv_*" type="text" indexed="true" stored="true"
|
||||
termVectors="true" termPositions="true" termOffsets="true"/>
|
||||
|
||||
<!-- special fields for dynamic copyField test -->
|
||||
<dynamicField name="dynamic_*" type="string" indexed="true" stored="true"/>
|
||||
<dynamicField name="*_dynamic" type="string" indexed="true" stored="true"/>
|
||||
|
||||
<!-- for testing to ensure that longer patterns are matched first -->
|
||||
<dynamicField name="*aa" type="string" indexed="true" stored="true"/>
|
||||
<dynamicField name="*aaa" type="integer" indexed="false" stored="true"/>
|
||||
|
||||
<!-- ignored becuase not stored or indexed -->
|
||||
<dynamicField name="*_ignored" type="text" indexed="false" stored="false"/>
|
||||
|
||||
</fields>
|
||||
|
||||
<defaultSearchField>text</defaultSearchField>
|
||||
<uniqueKey>id</uniqueKey>
|
||||
|
||||
<!-- copyField commands copy one field to another at the time a document
|
||||
is added to the index. It's used either to index the same field different
|
||||
ways, or to add multiple fields to the same field for easier/faster searching.
|
||||
-->
|
||||
<copyField source="title" dest="title_stemmed"/>
|
||||
<copyField source="title" dest="title_lettertok"/>
|
||||
|
||||
<copyField source="title" dest="text"/>
|
||||
<copyField source="subject" dest="text"/>
|
||||
|
||||
<copyField source="*_t" dest="text"/>
|
||||
|
||||
<!-- dynamic destination -->
|
||||
<copyField source="*_dynamic" dest="dynamic_*"/>
|
||||
|
||||
<!-- Similarity is the scoring routine for each document vs a query.
|
||||
A custom similarity may be specified here, but the default is fine
|
||||
for most applications.
|
||||
-->
|
||||
<similarity class="org.apache.solr.schema.CustomSimilarityFactory">
|
||||
<str name="echo">is there an echo?</str>
|
||||
</similarity>
|
||||
|
||||
</schema>
|
|
@ -0,0 +1,463 @@
|
|||
<?xml version="1.0" ?>
|
||||
<!--
|
||||
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.
|
||||
-->
|
||||
|
||||
<!-- The Solr schema file. This file should be named "schema.xml" and
|
||||
should be located where the classloader for the Solr webapp can find it.
|
||||
|
||||
This schema is used for testing, and as such has everything and the
|
||||
kitchen sink thrown in. See example/solr/conf/schema.xml for a
|
||||
more concise example.
|
||||
|
||||
$Id$
|
||||
$Source$
|
||||
$Name$
|
||||
-->
|
||||
|
||||
<schema name="test" version="1.0">
|
||||
<types>
|
||||
|
||||
<!-- field type definitions... note that the "name" attribute is
|
||||
just a label to be used by field definitions. The "class"
|
||||
attribute and any other attributes determine the real type and
|
||||
behavior of the fieldtype.
|
||||
-->
|
||||
|
||||
<!-- numeric field types that store and index the text
|
||||
value verbatim (and hence don't sort correctly or support range queries.)
|
||||
These are provided more for backward compatability, allowing one
|
||||
to create a schema that matches an existing lucene index.
|
||||
-->
|
||||
<fieldType name="integer" class="solr.IntField"/>
|
||||
<fieldType name="long" class="solr.LongField"/>
|
||||
<fieldtype name="float" class="solr.FloatField"/>
|
||||
<fieldType name="double" class="solr.DoubleField"/>
|
||||
|
||||
<!-- numeric field types that manipulate the value into
|
||||
a string value that isn't human readable in it's internal form,
|
||||
but sorts correctly and supports range queries.
|
||||
|
||||
If sortMissingLast="true" then a sort on this field will cause documents
|
||||
without the field to come after documents with the field,
|
||||
regardless of the requested sort order.
|
||||
If sortMissingFirst="true" then a sort on this field will cause documents
|
||||
without the field to come before documents with the field,
|
||||
regardless of the requested sort order.
|
||||
If sortMissingLast="false" and sortMissingFirst="false" (the default),
|
||||
then default lucene sorting will be used which places docs without the field
|
||||
first in an ascending sort and last in a descending sort.
|
||||
-->
|
||||
<fieldtype name="sint" class="solr.SortableIntField" sortMissingLast="true"/>
|
||||
<fieldtype name="slong" class="solr.SortableLongField" sortMissingLast="true"/>
|
||||
<fieldtype name="sfloat" class="solr.SortableFloatField" sortMissingLast="true"/>
|
||||
<fieldtype name="sdouble" class="solr.SortableDoubleField" sortMissingLast="true"/>
|
||||
|
||||
<!-- bcd versions of sortable numeric type may provide smaller
|
||||
storage space and support very large numbers.
|
||||
-->
|
||||
<fieldtype name="bcdint" class="solr.BCDIntField" sortMissingLast="true"/>
|
||||
<fieldtype name="bcdlong" class="solr.BCDLongField" sortMissingLast="true"/>
|
||||
<fieldtype name="bcdstr" class="solr.BCDStrField" sortMissingLast="true"/>
|
||||
|
||||
<!-- Field type demonstrating an Analyzer failure -->
|
||||
<fieldtype name="failtype1" class="solr.TextField">
|
||||
<analyzer type="index">
|
||||
<tokenizer class="solr.WhitespaceTokenizerFactory"/>
|
||||
<filter class="solr.WordDelimiterFilterFactory" generateWordParts="1" generateNumberParts="0" catenateWords="0" catenateNumbers="0" catenateAll="0"/>
|
||||
<filter class="solr.LowerCaseFilterFactory"/>
|
||||
</analyzer>
|
||||
</fieldtype>
|
||||
|
||||
<!-- Demonstrating ignoreCaseChange -->
|
||||
<fieldtype name="wdf_nocase" class="solr.TextField">
|
||||
<analyzer>
|
||||
<tokenizer class="solr.WhitespaceTokenizerFactory"/>
|
||||
<filter class="solr.WordDelimiterFilterFactory" generateWordParts="1" generateNumberParts="0" catenateWords="0" catenateNumbers="0" catenateAll="0" splitOnCaseChange="0" preserveOriginal="0"/>
|
||||
<filter class="solr.LowerCaseFilterFactory"/>
|
||||
</analyzer>
|
||||
</fieldtype>
|
||||
|
||||
<fieldtype name="wdf_preserve" class="solr.TextField">
|
||||
<analyzer>
|
||||
<tokenizer class="solr.WhitespaceTokenizerFactory"/>
|
||||
<filter class="solr.WordDelimiterFilterFactory" generateWordParts="0" generateNumberParts="1" catenateWords="0" catenateNumbers="0" catenateAll="0" splitOnCaseChange="0" preserveOriginal="1"/>
|
||||
<filter class="solr.LowerCaseFilterFactory"/>
|
||||
</analyzer>
|
||||
</fieldtype>
|
||||
|
||||
|
||||
<!-- HighlitText optimizes storage for (long) columns which will be highlit -->
|
||||
<fieldtype name="highlittext" class="solr.TextField" compressThreshold="345" />
|
||||
|
||||
<fieldtype name="boolean" class="solr.BoolField" sortMissingLast="true"/>
|
||||
<fieldtype name="string" class="solr.StrField" sortMissingLast="true"/>
|
||||
|
||||
<!-- format for date is 1995-12-31T23:59:59.999Z and only the fractional
|
||||
seconds part (.999) is optional.
|
||||
-->
|
||||
<fieldtype name="date" class="solr.DateField" sortMissingLast="true"/>
|
||||
|
||||
<!-- solr.TextField allows the specification of custom
|
||||
text analyzers specified as a tokenizer and a list
|
||||
of token filters.
|
||||
-->
|
||||
<fieldtype name="text" class="solr.TextField">
|
||||
<analyzer>
|
||||
<tokenizer class="solr.StandardTokenizerFactory"/>
|
||||
<filter class="solr.StandardFilterFactory"/>
|
||||
<filter class="solr.LowerCaseFilterFactory"/>
|
||||
<filter class="solr.StopFilterFactory"/>
|
||||
<!-- lucene PorterStemFilterFactory deprecated
|
||||
<filter class="solr.PorterStemFilterFactory"/>
|
||||
-->
|
||||
<filter class="solr.EnglishPorterFilterFactory"/>
|
||||
</analyzer>
|
||||
</fieldtype>
|
||||
|
||||
|
||||
<fieldtype name="nametext" class="solr.TextField">
|
||||
<analyzer class="org.apache.lucene.analysis.WhitespaceAnalyzer"/>
|
||||
</fieldtype>
|
||||
|
||||
<fieldtype name="teststop" class="solr.TextField">
|
||||
<analyzer>
|
||||
<tokenizer class="solr.LowerCaseTokenizerFactory"/>
|
||||
<filter class="solr.StandardFilterFactory"/>
|
||||
<!--filter class="solr.StopFilterFactory" words="stopwords.txt"/-->
|
||||
</analyzer>
|
||||
</fieldtype>
|
||||
|
||||
<!-- fieldtypes in this section isolate tokenizers and tokenfilters for testing -->
|
||||
<fieldtype name="lowertok" class="solr.TextField">
|
||||
<analyzer><tokenizer class="solr.LowerCaseTokenizerFactory"/></analyzer>
|
||||
</fieldtype>
|
||||
<fieldtype name="keywordtok" class="solr.TextField">
|
||||
<analyzer><tokenizer class="solr.KeywordTokenizerFactory"/></analyzer>
|
||||
</fieldtype>
|
||||
<fieldtype name="standardtok" class="solr.TextField">
|
||||
<analyzer><tokenizer class="solr.StandardTokenizerFactory"/></analyzer>
|
||||
</fieldtype>
|
||||
<fieldtype name="lettertok" class="solr.TextField">
|
||||
<analyzer><tokenizer class="solr.LetterTokenizerFactory"/></analyzer>
|
||||
</fieldtype>
|
||||
<fieldtype name="whitetok" class="solr.TextField">
|
||||
<analyzer><tokenizer class="solr.WhitespaceTokenizerFactory"/></analyzer>
|
||||
</fieldtype>
|
||||
<fieldtype name="HTMLstandardtok" class="solr.TextField">
|
||||
<analyzer><tokenizer class="solr.HTMLStripStandardTokenizerFactory"/></analyzer>
|
||||
</fieldtype>
|
||||
<fieldtype name="HTMLwhitetok" class="solr.TextField">
|
||||
<analyzer><tokenizer class="solr.HTMLStripWhitespaceTokenizerFactory"/></analyzer>
|
||||
</fieldtype>
|
||||
<fieldtype name="standardtokfilt" class="solr.TextField">
|
||||
<analyzer>
|
||||
<tokenizer class="solr.StandardTokenizerFactory"/>
|
||||
<filter class="solr.StandardFilterFactory"/>
|
||||
</analyzer>
|
||||
</fieldtype>
|
||||
<fieldtype name="standardfilt" class="solr.TextField">
|
||||
<analyzer>
|
||||
<tokenizer class="solr.WhitespaceTokenizerFactory"/>
|
||||
<filter class="solr.StandardFilterFactory"/>
|
||||
</analyzer>
|
||||
</fieldtype>
|
||||
<fieldtype name="lowerfilt" class="solr.TextField">
|
||||
<analyzer>
|
||||
<tokenizer class="solr.WhitespaceTokenizerFactory"/>
|
||||
<filter class="solr.LowerCaseFilterFactory"/>
|
||||
</analyzer>
|
||||
</fieldtype>
|
||||
<fieldtype name="patternreplacefilt" class="solr.TextField">
|
||||
<analyzer type="index">
|
||||
<tokenizer class="solr.KeywordTokenizerFactory"/>
|
||||
<filter class="solr.PatternReplaceFilterFactory"
|
||||
pattern="([^a-zA-Z])" replacement="_" replace="all"
|
||||
/>
|
||||
</analyzer>
|
||||
<analyzer type="query">
|
||||
<tokenizer class="solr.KeywordTokenizerFactory"/>
|
||||
</analyzer>
|
||||
</fieldtype>
|
||||
<fieldtype name="porterfilt" class="solr.TextField">
|
||||
<analyzer>
|
||||
<tokenizer class="solr.WhitespaceTokenizerFactory"/>
|
||||
<filter class="solr.PorterStemFilterFactory"/>
|
||||
</analyzer>
|
||||
</fieldtype>
|
||||
<!-- fieldtype name="snowballfilt" class="solr.TextField">
|
||||
<analyzer>
|
||||
<tokenizer class="solr.WhitespaceTokenizerFactory"/>
|
||||
<filter class="solr.SnowballPorterFilterFactory"/>
|
||||
</analyzer>
|
||||
</fieldtype -->
|
||||
<fieldtype name="engporterfilt" class="solr.TextField">
|
||||
<analyzer>
|
||||
<tokenizer class="solr.WhitespaceTokenizerFactory"/>
|
||||
<filter class="solr.EnglishPorterFilterFactory"/>
|
||||
</analyzer>
|
||||
</fieldtype>
|
||||
<fieldtype name="custengporterfilt" class="solr.TextField">
|
||||
<analyzer>
|
||||
<tokenizer class="solr.WhitespaceTokenizerFactory"/>
|
||||
<!--filter class="solr.EnglishPorterFilterFactory" protected="protwords.txt"/-->
|
||||
</analyzer>
|
||||
</fieldtype>
|
||||
<fieldtype name="stopfilt" class="solr.TextField">
|
||||
<analyzer>
|
||||
<tokenizer class="solr.WhitespaceTokenizerFactory"/>
|
||||
<filter class="solr.StopFilterFactory" ignoreCase="true"/>
|
||||
</analyzer>
|
||||
</fieldtype>
|
||||
<fieldtype name="custstopfilt" class="solr.TextField">
|
||||
<analyzer>
|
||||
<tokenizer class="solr.WhitespaceTokenizerFactory"/>
|
||||
<!--filter class="solr.StopFilterFactory" words="stopwords.txt"/-->
|
||||
</analyzer>
|
||||
</fieldtype>
|
||||
<fieldtype name="lengthfilt" class="solr.TextField">
|
||||
<analyzer>
|
||||
<tokenizer class="solr.WhitespaceTokenizerFactory"/>
|
||||
<filter class="solr.LengthFilterFactory" min="2" max="5"/>
|
||||
</analyzer>
|
||||
</fieldtype>
|
||||
|
||||
<fieldtype name="subword" class="solr.TextField" multiValued="true" positionIncrementGap="100">
|
||||
<analyzer type="index">
|
||||
<tokenizer class="solr.WhitespaceTokenizerFactory"/>
|
||||
<filter class="solr.WordDelimiterFilterFactory" generateWordParts="1" generateNumberParts="1" catenateWords="1" catenateNumbers="1" catenateAll="0"/>
|
||||
<filter class="solr.LowerCaseFilterFactory"/>
|
||||
<filter class="solr.StopFilterFactory"/>
|
||||
<filter class="solr.EnglishPorterFilterFactory"/>
|
||||
</analyzer>
|
||||
<analyzer type="query">
|
||||
<tokenizer class="solr.WhitespaceTokenizerFactory"/>
|
||||
<filter class="solr.WordDelimiterFilterFactory" generateWordParts="1" generateNumberParts="1" catenateWords="0" catenateNumbers="0" catenateAll="0"/>
|
||||
<filter class="solr.LowerCaseFilterFactory"/>
|
||||
<filter class="solr.StopFilterFactory"/>
|
||||
<filter class="solr.EnglishPorterFilterFactory"/>
|
||||
</analyzer>
|
||||
</fieldtype>
|
||||
|
||||
<!-- more flexible in matching skus, but more chance of a false match -->
|
||||
<fieldtype name="skutype1" class="solr.TextField">
|
||||
<analyzer type="index">
|
||||
<tokenizer class="solr.WhitespaceTokenizerFactory"/>
|
||||
<filter class="solr.WordDelimiterFilterFactory" generateWordParts="1" generateNumberParts="1" catenateWords="1" catenateNumbers="1" catenateAll="0"/>
|
||||
<filter class="solr.LowerCaseFilterFactory"/>
|
||||
</analyzer>
|
||||
<analyzer type="query">
|
||||
<tokenizer class="solr.WhitespaceTokenizerFactory"/>
|
||||
<filter class="solr.WordDelimiterFilterFactory" generateWordParts="0" generateNumberParts="0" catenateWords="1" catenateNumbers="1" catenateAll="0"/>
|
||||
<filter class="solr.LowerCaseFilterFactory"/>
|
||||
</analyzer>
|
||||
</fieldtype>
|
||||
|
||||
<!-- less flexible in matching skus, but less chance of a false match -->
|
||||
<fieldtype name="skutype2" class="solr.TextField">
|
||||
<analyzer type="index">
|
||||
<tokenizer class="solr.WhitespaceTokenizerFactory"/>
|
||||
<filter class="solr.WordDelimiterFilterFactory" generateWordParts="0" generateNumberParts="0" catenateWords="1" catenateNumbers="1" catenateAll="0"/>
|
||||
<filter class="solr.LowerCaseFilterFactory"/>
|
||||
</analyzer>
|
||||
<analyzer type="query">
|
||||
<tokenizer class="solr.WhitespaceTokenizerFactory"/>
|
||||
<filter class="solr.WordDelimiterFilterFactory" generateWordParts="0" generateNumberParts="0" catenateWords="1" catenateNumbers="1" catenateAll="0"/>
|
||||
<filter class="solr.LowerCaseFilterFactory"/>
|
||||
</analyzer>
|
||||
</fieldtype>
|
||||
|
||||
<!-- less flexible in matching skus, but less chance of a false match -->
|
||||
<fieldtype name="syn" class="solr.TextField">
|
||||
<analyzer>
|
||||
<tokenizer class="solr.WhitespaceTokenizerFactory"/>
|
||||
<!--filter name="syn" class="solr.SynonymFilterFactory" synonyms="synonyms.txt"/-->
|
||||
</analyzer>
|
||||
</fieldtype>
|
||||
|
||||
<!-- Demonstrates How RemoveDuplicatesTokenFilter makes stemmed
|
||||
synonyms "better"
|
||||
-->
|
||||
<fieldtype name="dedup" class="solr.TextField">
|
||||
<analyzer>
|
||||
<tokenizer class="solr.WhitespaceTokenizerFactory"/>
|
||||
<!--filter class="solr.SynonymFilterFactory"
|
||||
synonyms="synonyms.txt" expand="true" /-->
|
||||
<filter class="solr.EnglishPorterFilterFactory"/>
|
||||
<filter class="solr.RemoveDuplicatesTokenFilterFactory" />
|
||||
</analyzer>
|
||||
</fieldtype>
|
||||
|
||||
<fieldtype name="unstored" class="solr.StrField" indexed="true" stored="false"/>
|
||||
|
||||
|
||||
<fieldtype name="textgap" class="solr.TextField" multiValued="true" positionIncrementGap="100">
|
||||
<analyzer>
|
||||
<tokenizer class="solr.WhitespaceTokenizerFactory"/>
|
||||
<filter class="solr.LowerCaseFilterFactory"/>
|
||||
</analyzer>
|
||||
</fieldtype>
|
||||
|
||||
</types>
|
||||
|
||||
|
||||
<fields>
|
||||
<field name="id" type="integer" indexed="true" stored="true" multiValued="false" required="false"/>
|
||||
<field name="name" type="nametext" indexed="true" stored="true"/>
|
||||
<field name="text" type="text" indexed="true" stored="false"/>
|
||||
<field name="subject" type="text" indexed="true" stored="true"/>
|
||||
<field name="title" type="nametext" indexed="true" stored="true"/>
|
||||
<field name="weight" type="float" indexed="true" stored="true"/>
|
||||
<field name="bday" type="date" indexed="true" stored="true"/>
|
||||
|
||||
<field name="title_stemmed" type="text" indexed="true" stored="false"/>
|
||||
<field name="title_lettertok" type="lettertok" indexed="true" stored="false"/>
|
||||
|
||||
<field name="syn" type="syn" indexed="true" stored="true"/>
|
||||
|
||||
<!-- to test property inheritance and overriding -->
|
||||
<field name="shouldbeunstored" type="unstored" />
|
||||
<field name="shouldbestored" type="unstored" stored="true"/>
|
||||
<field name="shouldbeunindexed" type="unstored" indexed="false" stored="true"/>
|
||||
|
||||
|
||||
<!-- test different combinations of indexed and stored -->
|
||||
<field name="bind" type="boolean" indexed="true" stored="false"/>
|
||||
<field name="bsto" type="boolean" indexed="false" stored="true"/>
|
||||
<field name="bindsto" type="boolean" indexed="true" stored="true"/>
|
||||
<field name="isto" type="integer" indexed="false" stored="true"/>
|
||||
<field name="iind" type="integer" indexed="true" stored="false"/>
|
||||
<field name="ssto" type="string" indexed="false" stored="true"/>
|
||||
<field name="sind" type="string" indexed="true" stored="false"/>
|
||||
<field name="sindsto" type="string" indexed="true" stored="true"/>
|
||||
|
||||
<!-- test combinations of term vector settings -->
|
||||
<field name="test_basictv" type="text" termVectors="true"/>
|
||||
<field name="test_notv" type="text" termVectors="false"/>
|
||||
<field name="test_postv" type="text" termVectors="true" termPositions="true"/>
|
||||
<field name="test_offtv" type="text" termVectors="true" termOffsets="true"/>
|
||||
<field name="test_posofftv" type="text" termVectors="true"
|
||||
termPositions="true" termOffsets="true"/>
|
||||
|
||||
<!-- test highlit field settings -->
|
||||
<field name="test_hlt" type="highlittext" indexed="true" compressed="true"/>
|
||||
<field name="test_hlt_off" type="highlittext" indexed="true" compressed="false"/>
|
||||
|
||||
<!-- fields to test individual tokenizers and tokenfilters -->
|
||||
<field name="teststop" type="teststop" indexed="true" stored="true"/>
|
||||
<field name="lowertok" type="lowertok" indexed="true" stored="true"/>
|
||||
<field name="keywordtok" type="keywordtok" indexed="true" stored="true"/>
|
||||
<field name="standardtok" type="standardtok" indexed="true" stored="true"/>
|
||||
<field name="HTMLstandardtok" type="HTMLstandardtok" indexed="true" stored="true"/>
|
||||
<field name="lettertok" type="lettertok" indexed="true" stored="true"/>
|
||||
<field name="whitetok" type="whitetok" indexed="true" stored="true"/>
|
||||
<field name="HTMLwhitetok" type="HTMLwhitetok" indexed="true" stored="true"/>
|
||||
<field name="standardtokfilt" type="standardtokfilt" indexed="true" stored="true"/>
|
||||
<field name="standardfilt" type="standardfilt" indexed="true" stored="true"/>
|
||||
<field name="lowerfilt" type="lowerfilt" indexed="true" stored="true"/>
|
||||
<field name="patternreplacefilt" type="patternreplacefilt" indexed="true" stored="true"/>
|
||||
<field name="porterfilt" type="porterfilt" indexed="true" stored="true"/>
|
||||
<field name="engporterfilt" type="engporterfilt" indexed="true" stored="true"/>
|
||||
<field name="custengporterfilt" type="custengporterfilt" indexed="true" stored="true"/>
|
||||
<field name="stopfilt" type="stopfilt" indexed="true" stored="true"/>
|
||||
<field name="custstopfilt" type="custstopfilt" indexed="true" stored="true"/>
|
||||
<field name="lengthfilt" type="lengthfilt" indexed="true" stored="true"/>
|
||||
<field name="dedup" type="dedup" indexed="true" stored="true"/>
|
||||
<field name="wdf_nocase" type="wdf_nocase" indexed="true" stored="true"/>
|
||||
<field name="wdf_preserve" type="wdf_preserve" indexed="true" stored="true"/>
|
||||
|
||||
<field name="numberpartfail" type="failtype1" indexed="true" stored="true"/>
|
||||
|
||||
<field name="nullfirst" type="string" indexed="true" stored="true" sortMissingFirst="true"/>
|
||||
|
||||
<field name="subword" type="subword" indexed="true" stored="true"/>
|
||||
<field name="sku1" type="skutype1" indexed="true" stored="true"/>
|
||||
<field name="sku2" type="skutype2" indexed="true" stored="true"/>
|
||||
|
||||
<field name="textgap" type="textgap" indexed="true" stored="true"/>
|
||||
|
||||
<field name="timestamp" type="date" indexed="true" stored="true" default="NOW" multiValued="false"/>
|
||||
<field name="multiDefault" type="string" indexed="true" stored="true" default="muLti-Default" multiValued="true"/>
|
||||
<field name="intDefault" type="sint" indexed="true" stored="true" default="42" multiValued="false"/>
|
||||
|
||||
<!--adding new field newname in replication test.-->
|
||||
<field name="newname" type="nametext" indexed="true" stored="true"/>
|
||||
|
||||
<!-- Dynamic field definitions. If a field name is not found, dynamicFields
|
||||
will be used if the name matches any of the patterns.
|
||||
RESTRICTION: the glob-like pattern in the name attribute must have
|
||||
a "*" only at the start or the end.
|
||||
EXAMPLE: name="*_i" will match any field ending in _i (like myid_i, z_i)
|
||||
Longer patterns will be matched first. if equal size patterns
|
||||
both match, the first appearing in the schema will be used.
|
||||
-->
|
||||
<dynamicField name="*_i" type="sint" indexed="true" stored="true"/>
|
||||
<dynamicField name="*_s" type="string" indexed="true" stored="true"/>
|
||||
<dynamicField name="*_s1" type="string" indexed="true" stored="true" multiValued="false"/>
|
||||
<dynamicField name="*_l" type="slong" indexed="true" stored="true"/>
|
||||
<dynamicField name="*_t" type="text" indexed="true" stored="true"/>
|
||||
<dynamicField name="*_b" type="boolean" indexed="true" stored="true"/>
|
||||
<dynamicField name="*_f" type="sfloat" indexed="true" stored="true"/>
|
||||
<dynamicField name="*_d" type="sdouble" indexed="true" stored="true"/>
|
||||
<dynamicField name="*_dt" type="date" indexed="true" stored="true"/>
|
||||
<dynamicField name="*_bcd" type="bcdstr" indexed="true" stored="true"/>
|
||||
|
||||
<dynamicField name="*_sI" type="string" indexed="true" stored="false"/>
|
||||
<dynamicField name="*_sS" type="string" indexed="false" stored="true"/>
|
||||
<dynamicField name="t_*" type="text" indexed="true" stored="true"/>
|
||||
<dynamicField name="tv_*" type="text" indexed="true" stored="true"
|
||||
termVectors="true" termPositions="true" termOffsets="true"/>
|
||||
|
||||
<!-- special fields for dynamic copyField test -->
|
||||
<dynamicField name="dynamic_*" type="string" indexed="true" stored="true"/>
|
||||
<dynamicField name="*_dynamic" type="string" indexed="true" stored="true"/>
|
||||
|
||||
<!-- for testing to ensure that longer patterns are matched first -->
|
||||
<dynamicField name="*aa" type="string" indexed="true" stored="true"/>
|
||||
<dynamicField name="*aaa" type="integer" indexed="false" stored="true"/>
|
||||
|
||||
<!-- ignored becuase not stored or indexed -->
|
||||
<dynamicField name="*_ignored" type="text" indexed="false" stored="false"/>
|
||||
|
||||
</fields>
|
||||
|
||||
<defaultSearchField>text</defaultSearchField>
|
||||
<uniqueKey>id</uniqueKey>
|
||||
|
||||
<!-- copyField commands copy one field to another at the time a document
|
||||
is added to the index. It's used either to index the same field different
|
||||
ways, or to add multiple fields to the same field for easier/faster searching.
|
||||
-->
|
||||
<copyField source="title" dest="title_stemmed"/>
|
||||
<copyField source="title" dest="title_lettertok"/>
|
||||
|
||||
<copyField source="title" dest="text"/>
|
||||
<copyField source="subject" dest="text"/>
|
||||
|
||||
<copyField source="*_t" dest="text"/>
|
||||
|
||||
<!-- dynamic destination -->
|
||||
<copyField source="*_dynamic" dest="dynamic_*"/>
|
||||
|
||||
<!-- Similarity is the scoring routine for each document vs a query.
|
||||
A custom similarity may be specified here, but the default is fine
|
||||
for most applications.
|
||||
-->
|
||||
<similarity class="org.apache.solr.schema.CustomSimilarityFactory">
|
||||
<str name="echo">is there an echo?</str>
|
||||
</similarity>
|
||||
|
||||
</schema>
|
|
@ -0,0 +1,425 @@
|
|||
<?xml version="1.0" ?>
|
||||
|
||||
<!--
|
||||
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.
|
||||
-->
|
||||
|
||||
<!-- $Id$
|
||||
$Source$
|
||||
$Name$
|
||||
-->
|
||||
|
||||
<config>
|
||||
|
||||
<jmx />
|
||||
|
||||
<!-- Used to specify an alternate directory to hold all index data.
|
||||
It defaults to "index" if not present, and should probably
|
||||
not be changed if replication is in use. -->
|
||||
<dataDir>${solr.data.dir:./solr/data}</dataDir>
|
||||
|
||||
<indexDefaults>
|
||||
<!-- Values here affect all index writers and act as a default
|
||||
unless overridden. -->
|
||||
<!-- Values here affect all index writers and act as a default unless overridden. -->
|
||||
<useCompoundFile>false</useCompoundFile>
|
||||
<mergeFactor>10</mergeFactor>
|
||||
<!-- If both ramBufferSizeMB and maxBufferedDocs is set, then Lucene will flush based on whichever limit is hit first.
|
||||
-->
|
||||
<!--<maxBufferedDocs>1000</maxBufferedDocs>-->
|
||||
<!-- Tell Lucene when to flush documents to disk.
|
||||
Giving Lucene more memory for indexing means faster indexing at the cost of more RAM
|
||||
|
||||
If both ramBufferSizeMB and maxBufferedDocs is set, then Lucene will flush based on whichever limit is hit first.
|
||||
|
||||
-->
|
||||
<ramBufferSizeMB>32</ramBufferSizeMB>
|
||||
<maxMergeDocs>2147483647</maxMergeDocs>
|
||||
<maxFieldLength>10000</maxFieldLength>
|
||||
<writeLockTimeout>1000</writeLockTimeout>
|
||||
<commitLockTimeout>10000</commitLockTimeout>
|
||||
|
||||
<!--
|
||||
Expert: Turn on Lucene's auto commit capability.
|
||||
|
||||
NOTE: Despite the name, this value does not have any relation to Solr's autoCommit functionality
|
||||
|
||||
-->
|
||||
<luceneAutoCommit>false</luceneAutoCommit>
|
||||
|
||||
<!--
|
||||
Expert:
|
||||
The Merge Policy in Lucene controls how merging is handled by Lucene. The default in 2.3 is the LogByteSizeMergePolicy, previous
|
||||
versions used LogDocMergePolicy.
|
||||
|
||||
LogByteSizeMergePolicy chooses segments to merge based on their size. The Lucene 2.2 default, LogDocMergePolicy chose when
|
||||
to merge based on number of documents
|
||||
|
||||
Other implementations of MergePolicy must have a no-argument constructor
|
||||
-->
|
||||
<mergePolicy>org.apache.lucene.index.LogByteSizeMergePolicy</mergePolicy>
|
||||
|
||||
<!--
|
||||
Expert:
|
||||
The Merge Scheduler in Lucene controls how merges are performed. The ConcurrentMergeScheduler (Lucene 2.3 default)
|
||||
can perform merges in the background using separate threads. The SerialMergeScheduler (Lucene 2.2 default) does not.
|
||||
-->
|
||||
<mergeScheduler>org.apache.lucene.index.ConcurrentMergeScheduler</mergeScheduler>
|
||||
<!-- these are global... can't currently override per index -->
|
||||
<writeLockTimeout>1000</writeLockTimeout>
|
||||
<commitLockTimeout>10000</commitLockTimeout>
|
||||
|
||||
<lockType>single</lockType>
|
||||
</indexDefaults>
|
||||
|
||||
<mainIndex>
|
||||
<!-- lucene options specific to the main on-disk lucene index -->
|
||||
<useCompoundFile>false</useCompoundFile>
|
||||
<mergeFactor>10</mergeFactor>
|
||||
<ramBufferSizeMB>32</ramBufferSizeMB>
|
||||
<maxMergeDocs>2147483647</maxMergeDocs>
|
||||
<maxFieldLength>10000</maxFieldLength>
|
||||
|
||||
<unlockOnStartup>true</unlockOnStartup>
|
||||
</mainIndex>
|
||||
|
||||
<updateHandler class="solr.DirectUpdateHandler2">
|
||||
|
||||
<!-- autocommit pending docs if certain criteria are met
|
||||
<autoCommit>
|
||||
<maxDocs>10000</maxDocs>
|
||||
<maxTime>3600000</maxTime>
|
||||
</autoCommit>
|
||||
-->
|
||||
<!-- represents a lower bound on the frequency that commits may
|
||||
occur (in seconds). NOTE: not yet implemented
|
||||
|
||||
<commitIntervalLowerBound>0</commitIntervalLowerBound>
|
||||
-->
|
||||
|
||||
<!-- The RunExecutableListener executes an external command.
|
||||
exe - the name of the executable to run
|
||||
dir - dir to use as the current working directory. default="."
|
||||
wait - the calling thread waits until the executable returns. default="true"
|
||||
args - the arguments to pass to the program. default=nothing
|
||||
env - environment variables to set. default=nothing
|
||||
-->
|
||||
<!-- A postCommit event is fired after every commit
|
||||
<listener event="postCommit" class="solr.RunExecutableListener">
|
||||
<str name="exe">/var/opt/resin3/__PORT__/scripts/solr/snapshooter</str>
|
||||
<str name="dir">/var/opt/resin3/__PORT__</str>
|
||||
<bool name="wait">true</bool>
|
||||
<arr name="args"> <str>arg1</str> <str>arg2</str> </arr>
|
||||
<arr name="env"> <str>MYVAR=val1</str> </arr>
|
||||
</listener>
|
||||
-->
|
||||
|
||||
|
||||
</updateHandler>
|
||||
|
||||
|
||||
<query>
|
||||
<!-- Maximum number of clauses in a boolean query... can affect
|
||||
range or wildcard queries that expand to big boolean
|
||||
queries. An exception is thrown if exceeded.
|
||||
-->
|
||||
<maxBooleanClauses>1024</maxBooleanClauses>
|
||||
|
||||
|
||||
<!-- Cache specification for Filters or DocSets - unordered set of *all* documents
|
||||
that match a particular query.
|
||||
-->
|
||||
<filterCache
|
||||
class="solr.search.LRUCache"
|
||||
size="512"
|
||||
initialSize="512"
|
||||
autowarmCount="256"/>
|
||||
|
||||
<queryResultCache
|
||||
class="solr.search.LRUCache"
|
||||
size="512"
|
||||
initialSize="512"
|
||||
autowarmCount="1024"/>
|
||||
|
||||
<documentCache
|
||||
class="solr.search.LRUCache"
|
||||
size="512"
|
||||
initialSize="512"
|
||||
autowarmCount="0"/>
|
||||
|
||||
<!-- If true, stored fields that are not requested will be loaded lazily.
|
||||
-->
|
||||
<enableLazyFieldLoading>true</enableLazyFieldLoading>
|
||||
|
||||
<!--
|
||||
|
||||
<cache name="myUserCache"
|
||||
class="solr.search.LRUCache"
|
||||
size="4096"
|
||||
initialSize="1024"
|
||||
autowarmCount="1024"
|
||||
regenerator="MyRegenerator"
|
||||
/>
|
||||
-->
|
||||
|
||||
|
||||
<useFilterForSortedQuery>true</useFilterForSortedQuery>
|
||||
|
||||
<queryResultWindowSize>10</queryResultWindowSize>
|
||||
|
||||
<!-- set maxSize artificially low to exercise both types of sets -->
|
||||
<HashDocSet maxSize="3" loadFactor="0.75"/>
|
||||
|
||||
|
||||
<!-- boolToFilterOptimizer converts boolean clauses with zero boost
|
||||
into cached filters if the number of docs selected by the clause exceeds
|
||||
the threshold (represented as a fraction of the total index)
|
||||
-->
|
||||
<boolTofilterOptimizer enabled="false" cacheSize="32" threshold=".05"/>
|
||||
|
||||
|
||||
<!-- a newSearcher event is fired whenever a new searcher is being prepared
|
||||
and there is a current searcher handling requests (aka registered). -->
|
||||
<!-- QuerySenderListener takes an array of NamedList and executes a
|
||||
local query request for each NamedList in sequence. -->
|
||||
<!--
|
||||
<listener event="newSearcher" class="solr.QuerySenderListener">
|
||||
<arr name="queries">
|
||||
<lst> <str name="q">solr</str> <str name="start">0</str> <str name="rows">10</str> </lst>
|
||||
<lst> <str name="q">rocks</str> <str name="start">0</str> <str name="rows">10</str> </lst>
|
||||
</arr>
|
||||
</listener>
|
||||
-->
|
||||
|
||||
<!-- a firstSearcher event is fired whenever a new searcher is being
|
||||
prepared but there is no current registered searcher to handle
|
||||
requests or to gain prewarming data from. -->
|
||||
<!--
|
||||
<listener event="firstSearcher" class="solr.QuerySenderListener">
|
||||
<arr name="queries">
|
||||
<lst> <str name="q">fast_warm</str> <str name="start">0</str> <str name="rows">10</str> </lst>
|
||||
</arr>
|
||||
</listener>
|
||||
-->
|
||||
|
||||
|
||||
</query>
|
||||
|
||||
|
||||
<!-- An alternate set representation that uses an integer hash to store filters (sets of docids).
|
||||
If the set cardinality <= maxSize elements, then HashDocSet will be used instead of the bitset
|
||||
based HashBitset. -->
|
||||
|
||||
<!-- requestHandler plugins... incoming queries will be dispatched to the
|
||||
correct handler based on the qt (query type) param matching the
|
||||
name of registered handlers.
|
||||
The "standard" request handler is the default and will be used if qt
|
||||
is not specified in the request.
|
||||
-->
|
||||
<requestHandler name="standard" class="solr.StandardRequestHandler">
|
||||
<bool name="httpCaching">true</bool>
|
||||
</requestHandler>
|
||||
<requestHandler name="dismaxOldStyleDefaults"
|
||||
class="solr.DisMaxRequestHandler" >
|
||||
<!-- for historic reasons, DisMaxRequestHandler will use all of
|
||||
it's init params as "defaults" if there is no "defaults" list
|
||||
specified
|
||||
-->
|
||||
<float name="tie">0.01</float>
|
||||
<str name="qf">
|
||||
text^0.5 features_t^1.0 subject^1.4 title_stemmed^2.0
|
||||
</str>
|
||||
<str name="pf">
|
||||
text^0.2 features_t^1.1 subject^1.4 title_stemmed^2.0 title^1.5
|
||||
</str>
|
||||
<str name="bf">
|
||||
ord(weight)^0.5 recip(rord(iind),1,1000,1000)^0.3
|
||||
</str>
|
||||
<str name="mm">
|
||||
3<-1 5<-2 6<90%
|
||||
</str>
|
||||
<int name="ps">100</int>
|
||||
</requestHandler>
|
||||
|
||||
<requestHandler name="/replication" class="solr.ReplicationHandler" >
|
||||
<lst name="master">
|
||||
<str name="replicateAfter">commit</str>
|
||||
<str name="confFiles">schema.xml,stopwords.txt</str>
|
||||
</lst>
|
||||
</requestHandler>
|
||||
|
||||
<requestHandler name="dismax" class="solr.DisMaxRequestHandler" >
|
||||
<lst name="defaults">
|
||||
<str name="q.alt">*:*</str>
|
||||
<float name="tie">0.01</float>
|
||||
<str name="qf">
|
||||
text^0.5 features_t^1.0 subject^1.4 title_stemmed^2.0
|
||||
</str>
|
||||
<str name="pf">
|
||||
text^0.2 features_t^1.1 subject^1.4 title_stemmed^2.0 title^1.5
|
||||
</str>
|
||||
<str name="bf">
|
||||
ord(weight)^0.5 recip(rord(iind),1,1000,1000)^0.3
|
||||
</str>
|
||||
<str name="mm">
|
||||
3<-1 5<-2 6<90%
|
||||
</str>
|
||||
<int name="ps">100</int>
|
||||
</lst>
|
||||
</requestHandler>
|
||||
<requestHandler name="old" class="solr.tst.OldRequestHandler" >
|
||||
<int name="myparam">1000</int>
|
||||
<float name="ratio">1.4142135</float>
|
||||
<arr name="myarr"><int>1</int><int>2</int></arr>
|
||||
<str>foo</str>
|
||||
</requestHandler>
|
||||
<requestHandler name="oldagain" class="solr.tst.OldRequestHandler" >
|
||||
<lst name="lst1"> <str name="op">sqrt</str> <int name="val">2</int> </lst>
|
||||
<lst name="lst2"> <str name="op">log</str> <float name="val">10</float> </lst>
|
||||
</requestHandler>
|
||||
|
||||
<requestHandler name="test" class="solr.tst.TestRequestHandler" />
|
||||
|
||||
<!-- test query parameter defaults -->
|
||||
<requestHandler name="defaults" class="solr.StandardRequestHandler">
|
||||
<lst name="defaults">
|
||||
<int name="rows">4</int>
|
||||
<bool name="hl">true</bool>
|
||||
<str name="hl.fl">text,name,subject,title,whitetok</str>
|
||||
</lst>
|
||||
</requestHandler>
|
||||
|
||||
<!-- test query parameter defaults -->
|
||||
<requestHandler name="lazy" class="solr.StandardRequestHandler" startup="lazy">
|
||||
<lst name="defaults">
|
||||
<int name="rows">4</int>
|
||||
<bool name="hl">true</bool>
|
||||
<str name="hl.fl">text,name,subject,title,whitetok</str>
|
||||
</lst>
|
||||
</requestHandler>
|
||||
|
||||
<requestHandler name="/update" class="solr.XmlUpdateRequestHandler" />
|
||||
<requestHandler name="/update/csv" class="solr.CSVRequestHandler" startup="lazy">
|
||||
<bool name="httpCaching">false</bool>
|
||||
</requestHandler>
|
||||
|
||||
<!-- test elevation -->
|
||||
<!--searchComponent name="elevate" class="org.apache.solr.handler.component.QueryElevationComponent" >
|
||||
<str name="queryFieldType">string</str>
|
||||
<str name="config-file">elevate.xml</str>
|
||||
</searchComponent-->
|
||||
|
||||
<requestHandler name="/elevate" class="org.apache.solr.handler.component.SearchHandler">
|
||||
<lst name="defaults">
|
||||
<str name="echoParams">explicit</str>
|
||||
</lst>
|
||||
<arr name="last-components">
|
||||
<!--str>elevate</str-->
|
||||
</arr>
|
||||
</requestHandler>
|
||||
|
||||
<searchComponent name="spellcheck" class="org.apache.solr.handler.component.SpellCheckComponent">
|
||||
<str name="queryAnalyzerFieldType">lowerfilt</str>
|
||||
|
||||
<lst name="spellchecker">
|
||||
<str name="name">default</str>
|
||||
<str name="field">lowerfilt</str>
|
||||
<str name="spellcheckIndexDir">spellchecker1</str>
|
||||
<str name="buildOnCommit">true</str>
|
||||
</lst>
|
||||
<!-- Example of using different distance measure -->
|
||||
<lst name="spellchecker">
|
||||
<str name="name">jarowinkler</str>
|
||||
<str name="field">lowerfilt</str>
|
||||
<!-- Use a different Distance Measure -->
|
||||
<str name="distanceMeasure">org.apache.lucene.search.spell.JaroWinklerDistance</str>
|
||||
<str name="spellcheckIndexDir">spellchecker2</str>
|
||||
|
||||
</lst>
|
||||
<lst name="spellchecker">
|
||||
<str name="classname">solr.FileBasedSpellChecker</str>
|
||||
<str name="name">external</str>
|
||||
<str name="sourceLocation">spellings.txt</str>
|
||||
<str name="characterEncoding">UTF-8</str>
|
||||
<str name="spellcheckIndexDir">spellchecker3</str>
|
||||
</lst>
|
||||
</searchComponent>
|
||||
<!--
|
||||
The SpellingQueryConverter to convert raw (CommonParams.Q) queries into tokens. Uses a simple regular expression
|
||||
to strip off field markup, boosts, ranges, etc. but it is not guaranteed to match an exact parse from the query parser.
|
||||
-->
|
||||
<queryConverter name="queryConverter" class="org.apache.solr.spelling.SpellingQueryConverter"/>
|
||||
|
||||
<requestHandler name="spellCheckCompRH" class="org.apache.solr.handler.component.SearchHandler">
|
||||
<lst name="defaults">
|
||||
<!-- omp = Only More Popular -->
|
||||
<str name="spellcheck.onlyMorePopular">false</str>
|
||||
<!-- exr = Extended Results -->
|
||||
<str name="spellcheck.extendedResults">false</str>
|
||||
<!-- The number of suggestions to return -->
|
||||
<str name="spellcheck.count">1</str>
|
||||
</lst>
|
||||
<arr name="last-components">
|
||||
<str>spellcheck</str>
|
||||
</arr>
|
||||
</requestHandler>
|
||||
|
||||
<highlighting>
|
||||
<!-- Configure the standard fragmenter -->
|
||||
<fragmenter name="gap" class="org.apache.solr.highlight.GapFragmenter" default="true">
|
||||
<lst name="defaults">
|
||||
<int name="hl.fragsize">100</int>
|
||||
</lst>
|
||||
</fragmenter>
|
||||
|
||||
<fragmenter name="regex" class="org.apache.solr.highlight.RegexFragmenter">
|
||||
<lst name="defaults">
|
||||
<int name="hl.fragsize">70</int>
|
||||
</lst>
|
||||
</fragmenter>
|
||||
|
||||
<!-- Configure the standard formatter -->
|
||||
<formatter name="html" class="org.apache.solr.highlight.HtmlFormatter" default="true">
|
||||
<lst name="defaults">
|
||||
<str name="hl.simple.pre"><![CDATA[<em>]]></str>
|
||||
<str name="hl.simple.post"><![CDATA[</em>]]></str>
|
||||
</lst>
|
||||
</formatter>
|
||||
</highlighting>
|
||||
|
||||
|
||||
<!-- enable streaming for testing... -->
|
||||
<requestDispatcher handleSelect="true" >
|
||||
<requestParsers enableRemoteStreaming="true" multipartUploadLimitInKB="2048" />
|
||||
<httpCaching lastModifiedFrom="openTime" etagSeed="Solr" never304="false">
|
||||
<cacheControl>max-age=30, public</cacheControl>
|
||||
</httpCaching>
|
||||
</requestDispatcher>
|
||||
|
||||
<admin>
|
||||
<defaultQuery>solr</defaultQuery>
|
||||
<gettableFiles>solrconfig.xml scheam.xml admin-extra.html</gettableFiles>
|
||||
</admin>
|
||||
|
||||
<!-- test getting system property -->
|
||||
<propTest attr1="${solr.test.sys.prop1}-$${literal}"
|
||||
attr2="${non.existent.sys.prop:default-from-config}">prefix-${solr.test.sys.prop2}-suffix</propTest>
|
||||
|
||||
<queryParser name="foo" class="FooQParserPlugin"/>
|
||||
|
||||
|
||||
|
||||
</config>
|
|
@ -0,0 +1,425 @@
|
|||
<?xml version="1.0" ?>
|
||||
|
||||
<!--
|
||||
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.
|
||||
-->
|
||||
|
||||
<!-- $Id$
|
||||
$Source$
|
||||
$Name$
|
||||
-->
|
||||
|
||||
<config>
|
||||
|
||||
<jmx />
|
||||
|
||||
<!-- Used to specify an alternate directory to hold all index data.
|
||||
It defaults to "index" if not present, and should probably
|
||||
not be changed if replication is in use. -->
|
||||
<dataDir>${solr.data.dir:./solr/data}</dataDir>
|
||||
|
||||
<indexDefaults>
|
||||
<!-- Values here affect all index writers and act as a default
|
||||
unless overridden. -->
|
||||
<!-- Values here affect all index writers and act as a default unless overridden. -->
|
||||
<useCompoundFile>false</useCompoundFile>
|
||||
<mergeFactor>10</mergeFactor>
|
||||
<!-- If both ramBufferSizeMB and maxBufferedDocs is set, then Lucene will flush based on whichever limit is hit first.
|
||||
-->
|
||||
<!--<maxBufferedDocs>1000</maxBufferedDocs>-->
|
||||
<!-- Tell Lucene when to flush documents to disk.
|
||||
Giving Lucene more memory for indexing means faster indexing at the cost of more RAM
|
||||
|
||||
If both ramBufferSizeMB and maxBufferedDocs is set, then Lucene will flush based on whichever limit is hit first.
|
||||
|
||||
-->
|
||||
<ramBufferSizeMB>32</ramBufferSizeMB>
|
||||
<maxMergeDocs>2147483647</maxMergeDocs>
|
||||
<maxFieldLength>10000</maxFieldLength>
|
||||
<writeLockTimeout>1000</writeLockTimeout>
|
||||
<commitLockTimeout>10000</commitLockTimeout>
|
||||
|
||||
<!--
|
||||
Expert: Turn on Lucene's auto commit capability.
|
||||
|
||||
NOTE: Despite the name, this value does not have any relation to Solr's autoCommit functionality
|
||||
|
||||
-->
|
||||
<luceneAutoCommit>false</luceneAutoCommit>
|
||||
|
||||
<!--
|
||||
Expert:
|
||||
The Merge Policy in Lucene controls how merging is handled by Lucene. The default in 2.3 is the LogByteSizeMergePolicy, previous
|
||||
versions used LogDocMergePolicy.
|
||||
|
||||
LogByteSizeMergePolicy chooses segments to merge based on their size. The Lucene 2.2 default, LogDocMergePolicy chose when
|
||||
to merge based on number of documents
|
||||
|
||||
Other implementations of MergePolicy must have a no-argument constructor
|
||||
-->
|
||||
<mergePolicy>org.apache.lucene.index.LogByteSizeMergePolicy</mergePolicy>
|
||||
|
||||
<!--
|
||||
Expert:
|
||||
The Merge Scheduler in Lucene controls how merges are performed. The ConcurrentMergeScheduler (Lucene 2.3 default)
|
||||
can perform merges in the background using separate threads. The SerialMergeScheduler (Lucene 2.2 default) does not.
|
||||
-->
|
||||
<mergeScheduler>org.apache.lucene.index.ConcurrentMergeScheduler</mergeScheduler>
|
||||
<!-- these are global... can't currently override per index -->
|
||||
<writeLockTimeout>1000</writeLockTimeout>
|
||||
<commitLockTimeout>10000</commitLockTimeout>
|
||||
|
||||
<lockType>single</lockType>
|
||||
</indexDefaults>
|
||||
|
||||
<mainIndex>
|
||||
<!-- lucene options specific to the main on-disk lucene index -->
|
||||
<useCompoundFile>false</useCompoundFile>
|
||||
<mergeFactor>10</mergeFactor>
|
||||
<ramBufferSizeMB>32</ramBufferSizeMB>
|
||||
<maxMergeDocs>2147483647</maxMergeDocs>
|
||||
<maxFieldLength>10000</maxFieldLength>
|
||||
|
||||
<unlockOnStartup>true</unlockOnStartup>
|
||||
</mainIndex>
|
||||
|
||||
<updateHandler class="solr.DirectUpdateHandler2">
|
||||
|
||||
<!-- autocommit pending docs if certain criteria are met
|
||||
<autoCommit>
|
||||
<maxDocs>10000</maxDocs>
|
||||
<maxTime>3600000</maxTime>
|
||||
</autoCommit>
|
||||
-->
|
||||
<!-- represents a lower bound on the frequency that commits may
|
||||
occur (in seconds). NOTE: not yet implemented
|
||||
|
||||
<commitIntervalLowerBound>0</commitIntervalLowerBound>
|
||||
-->
|
||||
|
||||
<!-- The RunExecutableListener executes an external command.
|
||||
exe - the name of the executable to run
|
||||
dir - dir to use as the current working directory. default="."
|
||||
wait - the calling thread waits until the executable returns. default="true"
|
||||
args - the arguments to pass to the program. default=nothing
|
||||
env - environment variables to set. default=nothing
|
||||
-->
|
||||
<!-- A postCommit event is fired after every commit
|
||||
<listener event="postCommit" class="solr.RunExecutableListener">
|
||||
<str name="exe">/var/opt/resin3/__PORT__/scripts/solr/snapshooter</str>
|
||||
<str name="dir">/var/opt/resin3/__PORT__</str>
|
||||
<bool name="wait">true</bool>
|
||||
<arr name="args"> <str>arg1</str> <str>arg2</str> </arr>
|
||||
<arr name="env"> <str>MYVAR=val1</str> </arr>
|
||||
</listener>
|
||||
-->
|
||||
|
||||
|
||||
</updateHandler>
|
||||
|
||||
|
||||
<query>
|
||||
<!-- Maximum number of clauses in a boolean query... can affect
|
||||
range or wildcard queries that expand to big boolean
|
||||
queries. An exception is thrown if exceeded.
|
||||
-->
|
||||
<maxBooleanClauses>1024</maxBooleanClauses>
|
||||
|
||||
|
||||
<!-- Cache specification for Filters or DocSets - unordered set of *all* documents
|
||||
that match a particular query.
|
||||
-->
|
||||
<filterCache
|
||||
class="solr.search.LRUCache"
|
||||
size="512"
|
||||
initialSize="512"
|
||||
autowarmCount="256"/>
|
||||
|
||||
<queryResultCache
|
||||
class="solr.search.LRUCache"
|
||||
size="512"
|
||||
initialSize="512"
|
||||
autowarmCount="1024"/>
|
||||
|
||||
<documentCache
|
||||
class="solr.search.LRUCache"
|
||||
size="512"
|
||||
initialSize="512"
|
||||
autowarmCount="0"/>
|
||||
|
||||
<!-- If true, stored fields that are not requested will be loaded lazily.
|
||||
-->
|
||||
<enableLazyFieldLoading>true</enableLazyFieldLoading>
|
||||
|
||||
<!--
|
||||
|
||||
<cache name="myUserCache"
|
||||
class="solr.search.LRUCache"
|
||||
size="4096"
|
||||
initialSize="1024"
|
||||
autowarmCount="1024"
|
||||
regenerator="MyRegenerator"
|
||||
/>
|
||||
-->
|
||||
|
||||
|
||||
<useFilterForSortedQuery>true</useFilterForSortedQuery>
|
||||
|
||||
<queryResultWindowSize>10</queryResultWindowSize>
|
||||
|
||||
<!-- set maxSize artificially low to exercise both types of sets -->
|
||||
<HashDocSet maxSize="3" loadFactor="0.75"/>
|
||||
|
||||
|
||||
<!-- boolToFilterOptimizer converts boolean clauses with zero boost
|
||||
into cached filters if the number of docs selected by the clause exceeds
|
||||
the threshold (represented as a fraction of the total index)
|
||||
-->
|
||||
<boolTofilterOptimizer enabled="false" cacheSize="32" threshold=".05"/>
|
||||
|
||||
|
||||
<!-- a newSearcher event is fired whenever a new searcher is being prepared
|
||||
and there is a current searcher handling requests (aka registered). -->
|
||||
<!-- QuerySenderListener takes an array of NamedList and executes a
|
||||
local query request for each NamedList in sequence. -->
|
||||
<!--
|
||||
<listener event="newSearcher" class="solr.QuerySenderListener">
|
||||
<arr name="queries">
|
||||
<lst> <str name="q">solr</str> <str name="start">0</str> <str name="rows">10</str> </lst>
|
||||
<lst> <str name="q">rocks</str> <str name="start">0</str> <str name="rows">10</str> </lst>
|
||||
</arr>
|
||||
</listener>
|
||||
-->
|
||||
|
||||
<!-- a firstSearcher event is fired whenever a new searcher is being
|
||||
prepared but there is no current registered searcher to handle
|
||||
requests or to gain prewarming data from. -->
|
||||
<!--
|
||||
<listener event="firstSearcher" class="solr.QuerySenderListener">
|
||||
<arr name="queries">
|
||||
<lst> <str name="q">fast_warm</str> <str name="start">0</str> <str name="rows">10</str> </lst>
|
||||
</arr>
|
||||
</listener>
|
||||
-->
|
||||
|
||||
|
||||
</query>
|
||||
|
||||
|
||||
<!-- An alternate set representation that uses an integer hash to store filters (sets of docids).
|
||||
If the set cardinality <= maxSize elements, then HashDocSet will be used instead of the bitset
|
||||
based HashBitset. -->
|
||||
|
||||
<!-- requestHandler plugins... incoming queries will be dispatched to the
|
||||
correct handler based on the qt (query type) param matching the
|
||||
name of registered handlers.
|
||||
The "standard" request handler is the default and will be used if qt
|
||||
is not specified in the request.
|
||||
-->
|
||||
<requestHandler name="standard" class="solr.StandardRequestHandler">
|
||||
<bool name="httpCaching">true</bool>
|
||||
</requestHandler>
|
||||
<requestHandler name="dismaxOldStyleDefaults"
|
||||
class="solr.DisMaxRequestHandler" >
|
||||
<!-- for historic reasons, DisMaxRequestHandler will use all of
|
||||
it's init params as "defaults" if there is no "defaults" list
|
||||
specified
|
||||
-->
|
||||
<float name="tie">0.01</float>
|
||||
<str name="qf">
|
||||
text^0.5 features_t^1.0 subject^1.4 title_stemmed^2.0
|
||||
</str>
|
||||
<str name="pf">
|
||||
text^0.2 features_t^1.1 subject^1.4 title_stemmed^2.0 title^1.5
|
||||
</str>
|
||||
<str name="bf">
|
||||
ord(weight)^0.5 recip(rord(iind),1,1000,1000)^0.3
|
||||
</str>
|
||||
<str name="mm">
|
||||
3<-1 5<-2 6<90%
|
||||
</str>
|
||||
<int name="ps">100</int>
|
||||
</requestHandler>
|
||||
<requestHandler name="dismax" class="solr.DisMaxRequestHandler" >
|
||||
<lst name="defaults">
|
||||
<str name="q.alt">*:*</str>
|
||||
<float name="tie">0.01</float>
|
||||
<str name="qf">
|
||||
text^0.5 features_t^1.0 subject^1.4 title_stemmed^2.0
|
||||
</str>
|
||||
<str name="pf">
|
||||
text^0.2 features_t^1.1 subject^1.4 title_stemmed^2.0 title^1.5
|
||||
</str>
|
||||
<str name="bf">
|
||||
ord(weight)^0.5 recip(rord(iind),1,1000,1000)^0.3
|
||||
</str>
|
||||
<str name="mm">
|
||||
3<-1 5<-2 6<90%
|
||||
</str>
|
||||
<int name="ps">100</int>
|
||||
</lst>
|
||||
</requestHandler>
|
||||
<requestHandler name="old" class="solr.tst.OldRequestHandler" >
|
||||
<int name="myparam">1000</int>
|
||||
<float name="ratio">1.4142135</float>
|
||||
<arr name="myarr"><int>1</int><int>2</int></arr>
|
||||
<str>foo</str>
|
||||
</requestHandler>
|
||||
<requestHandler name="oldagain" class="solr.tst.OldRequestHandler" >
|
||||
<lst name="lst1"> <str name="op">sqrt</str> <int name="val">2</int> </lst>
|
||||
<lst name="lst2"> <str name="op">log</str> <float name="val">10</float> </lst>
|
||||
</requestHandler>
|
||||
|
||||
<requestHandler name="test" class="solr.tst.TestRequestHandler" />
|
||||
|
||||
<!-- test query parameter defaults -->
|
||||
<requestHandler name="defaults" class="solr.StandardRequestHandler">
|
||||
<lst name="defaults">
|
||||
<int name="rows">4</int>
|
||||
<bool name="hl">true</bool>
|
||||
<str name="hl.fl">text,name,subject,title,whitetok</str>
|
||||
</lst>
|
||||
</requestHandler>
|
||||
|
||||
<!-- test query parameter defaults -->
|
||||
<requestHandler name="lazy" class="solr.StandardRequestHandler" startup="lazy">
|
||||
<lst name="defaults">
|
||||
<int name="rows">4</int>
|
||||
<bool name="hl">true</bool>
|
||||
<str name="hl.fl">text,name,subject,title,whitetok</str>
|
||||
</lst>
|
||||
</requestHandler>
|
||||
|
||||
<requestHandler name="/update" class="solr.XmlUpdateRequestHandler" />
|
||||
<requestHandler name="/update/csv" class="solr.CSVRequestHandler" startup="lazy">
|
||||
<bool name="httpCaching">false</bool>
|
||||
</requestHandler>
|
||||
|
||||
<!-- test elevation -->
|
||||
<!--searchComponent name="elevate" class="org.apache.solr.handler.component.QueryElevationComponent" >
|
||||
<str name="queryFieldType">string</str>
|
||||
<str name="config-file">elevate.xml</str>
|
||||
</searchComponent-->
|
||||
|
||||
<requestHandler name="/elevate" class="org.apache.solr.handler.component.SearchHandler">
|
||||
<lst name="defaults">
|
||||
<str name="echoParams">explicit</str>
|
||||
</lst>
|
||||
<arr name="last-components">
|
||||
<!--str>elevate</str-->
|
||||
</arr>
|
||||
</requestHandler>
|
||||
|
||||
<searchComponent name="spellcheck" class="org.apache.solr.handler.component.SpellCheckComponent">
|
||||
<str name="queryAnalyzerFieldType">lowerfilt</str>
|
||||
|
||||
<lst name="spellchecker">
|
||||
<str name="name">default</str>
|
||||
<str name="field">lowerfilt</str>
|
||||
<str name="spellcheckIndexDir">spellchecker1</str>
|
||||
<str name="buildOnCommit">true</str>
|
||||
</lst>
|
||||
<!-- Example of using different distance measure -->
|
||||
<lst name="spellchecker">
|
||||
<str name="name">jarowinkler</str>
|
||||
<str name="field">lowerfilt</str>
|
||||
<!-- Use a different Distance Measure -->
|
||||
<str name="distanceMeasure">org.apache.lucene.search.spell.JaroWinklerDistance</str>
|
||||
<str name="spellcheckIndexDir">spellchecker2</str>
|
||||
|
||||
</lst>
|
||||
<lst name="spellchecker">
|
||||
<str name="classname">solr.FileBasedSpellChecker</str>
|
||||
<str name="name">external</str>
|
||||
<str name="sourceLocation">spellings.txt</str>
|
||||
<str name="characterEncoding">UTF-8</str>
|
||||
<str name="spellcheckIndexDir">spellchecker3</str>
|
||||
</lst>
|
||||
</searchComponent>
|
||||
<!--
|
||||
The SpellingQueryConverter to convert raw (CommonParams.Q) queries into tokens. Uses a simple regular expression
|
||||
to strip off field markup, boosts, ranges, etc. but it is not guaranteed to match an exact parse from the query parser.
|
||||
-->
|
||||
<queryConverter name="queryConverter" class="org.apache.solr.spelling.SpellingQueryConverter"/>
|
||||
|
||||
<requestHandler name="/replication" class="solr.ReplicationHandler" >
|
||||
<lst name="slave">
|
||||
<str name="masterUrl">http://localhost:9999/solr/replication</str>
|
||||
<str name="pollInterval">00:00:01</str>
|
||||
</lst>
|
||||
</requestHandler>
|
||||
|
||||
|
||||
<requestHandler name="spellCheckCompRH" class="org.apache.solr.handler.component.SearchHandler">
|
||||
<lst name="defaults">
|
||||
<!-- omp = Only More Popular -->
|
||||
<str name="spellcheck.onlyMorePopular">false</str>
|
||||
<!-- exr = Extended Results -->
|
||||
<str name="spellcheck.extendedResults">false</str>
|
||||
<!-- The number of suggestions to return -->
|
||||
<str name="spellcheck.count">1</str>
|
||||
</lst>
|
||||
<arr name="last-components">
|
||||
<str>spellcheck</str>
|
||||
</arr>
|
||||
</requestHandler>
|
||||
|
||||
<highlighting>
|
||||
<!-- Configure the standard fragmenter -->
|
||||
<fragmenter name="gap" class="org.apache.solr.highlight.GapFragmenter" default="true">
|
||||
<lst name="defaults">
|
||||
<int name="hl.fragsize">100</int>
|
||||
</lst>
|
||||
</fragmenter>
|
||||
|
||||
<fragmenter name="regex" class="org.apache.solr.highlight.RegexFragmenter">
|
||||
<lst name="defaults">
|
||||
<int name="hl.fragsize">70</int>
|
||||
</lst>
|
||||
</fragmenter>
|
||||
|
||||
<!-- Configure the standard formatter -->
|
||||
<formatter name="html" class="org.apache.solr.highlight.HtmlFormatter" default="true">
|
||||
<lst name="defaults">
|
||||
<str name="hl.simple.pre"><![CDATA[<em>]]></str>
|
||||
<str name="hl.simple.post"><![CDATA[</em>]]></str>
|
||||
</lst>
|
||||
</formatter>
|
||||
</highlighting>
|
||||
|
||||
|
||||
<!-- enable streaming for testing... -->
|
||||
<requestDispatcher handleSelect="true" >
|
||||
<requestParsers enableRemoteStreaming="true" multipartUploadLimitInKB="2048" />
|
||||
<httpCaching lastModifiedFrom="openTime" etagSeed="Solr" never304="false">
|
||||
<cacheControl>max-age=30, public</cacheControl>
|
||||
</httpCaching>
|
||||
</requestDispatcher>
|
||||
|
||||
<admin>
|
||||
<defaultQuery>solr</defaultQuery>
|
||||
<gettableFiles>solrconfig.xml scheam.xml admin-extra.html</gettableFiles>
|
||||
</admin>
|
||||
|
||||
<!-- test getting system property -->
|
||||
<propTest attr1="${solr.test.sys.prop1}-$${literal}"
|
||||
attr2="${non.existent.sys.prop:default-from-config}">prefix-${solr.test.sys.prop2}-suffix</propTest>
|
||||
|
||||
<queryParser name="foo" class="FooQParserPlugin"/>
|
||||
|
||||
|
||||
|
||||
</config>
|
|
@ -23,11 +23,12 @@
|
|||
<%@ page import="java.util.Date" %>
|
||||
<%@ page import="java.util.List" %>
|
||||
<%@ page import="java.util.Collection" %>
|
||||
<%@ page import="org.apache.solr.request.SolrRequestHandler"%>
|
||||
|
||||
<%-- jsp:include page="header.jsp"/ --%>
|
||||
<%-- do a verbatim include so we can use the local vars --%>
|
||||
<%@include file="header.jsp" %>
|
||||
|
||||
<%SolrRequestHandler replicationhandler = core.getRequestHandler("/replication");%>
|
||||
<br clear="all">
|
||||
<table>
|
||||
|
||||
|
@ -43,7 +44,7 @@
|
|||
[<a href="file/?file=<%=core.getConfigResource()%>">Config</a>]
|
||||
<% } %>
|
||||
[<a href="analysis.jsp?highlight=on">Analysis</a>]
|
||||
[<a href="schema.jsp">Schema Browser</a>]
|
||||
[<a href="schema.jsp">Schema Browser</a>] <%if(replicationhandler != null ){%>[<a href="replication">Replication</a>]<%}%>
|
||||
<br>
|
||||
[<a href="stats.jsp">Statistics</a>]
|
||||
[<a href="registry.jsp">Info</a>]
|
||||
|
|
|
@ -0,0 +1,81 @@
|
|||
<%@ page contentType="text/html; charset=utf-8" pageEncoding="UTF-8"%>
|
||||
<%--
|
||||
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.
|
||||
--%>
|
||||
<!-- $Id$ -->
|
||||
<%@ page import="org.apache.solr.common.util.NamedList,
|
||||
org.apache.solr.common.util.SimpleOrderedMap,
|
||||
org.apache.solr.request.LocalSolrQueryRequest,
|
||||
org.apache.solr.request.SolrQueryResponse,
|
||||
org.apache.solr.request.SolrRequestHandler"%>
|
||||
|
||||
<html>
|
||||
<head>
|
||||
|
||||
<%
|
||||
request.setCharacterEncoding("UTF-8");
|
||||
%>
|
||||
|
||||
<%@include file="../_info.jsp" %>
|
||||
|
||||
<script>
|
||||
var host_name="<%= hostname %>"
|
||||
</script>
|
||||
|
||||
<meta http-equiv="Content-Type" content="text/html; charset=UTF-8">
|
||||
<link rel="stylesheet" type="text/css" href="../solr-admin.css">
|
||||
<link rel="icon" href="../favicon.ico" type="image/ico" />
|
||||
<link rel="shortcut icon" href="../favicon.ico" type="image/ico" />
|
||||
<title>Solr replication admin page</title>
|
||||
<script type="text/javascript" src="../jquery-1.2.3.min.js"></script>
|
||||
|
||||
<%!
|
||||
public NamedList executeCommand(String command, SolrCore core, SolrRequestHandler rh){
|
||||
NamedList namedlist = new SimpleOrderedMap();
|
||||
namedlist.add("command", command);
|
||||
LocalSolrQueryRequest solrqreq = new LocalSolrQueryRequest(core, namedlist);
|
||||
SolrQueryResponse rsp = new SolrQueryResponse();
|
||||
core.execute(rh, solrqreq, rsp);
|
||||
namedlist = rsp.getValues();
|
||||
return namedlist;
|
||||
}
|
||||
%>
|
||||
|
||||
<%
|
||||
|
||||
final SolrRequestHandler rh = core.getRequestHandler("/replication");
|
||||
NamedList namedlist = executeCommand("details",core,rh);
|
||||
NamedList detailsMap = (NamedList)namedlist.get("details");
|
||||
|
||||
if("false".equals((String)detailsMap.get("isMaster"))){
|
||||
%>
|
||||
<meta http-equiv="refresh" content="2"/>
|
||||
<%}%>
|
||||
|
||||
</head>
|
||||
|
||||
<body>
|
||||
<a href=".."><img border="0" align="right" height="61" width="142" src="../solr-head.gif" alt="Solr"></a>
|
||||
<h1>Solr replication (<%= collectionName %>)
|
||||
<%
|
||||
if("true".equals((String)detailsMap.get("isMaster")))
|
||||
out.println(" Master");
|
||||
else
|
||||
out.println(" Slave");
|
||||
%></h1>
|
||||
|
||||
<%= hostname %>:<%= port %><br/>
|
||||
cwd=<%= cwd %> SolrHome=<%= solrHome %>
|
|
@ -0,0 +1,379 @@
|
|||
<%@ page contentType="text/html; charset=utf-8" pageEncoding="UTF-8" %>
|
||||
<%--
|
||||
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.
|
||||
--%>
|
||||
<%@ page import="java.util.Collection" %>
|
||||
<%@ page import="java.util.Date" %>
|
||||
|
||||
|
||||
<%-- do a verbatim include so we can use the local vars --%>
|
||||
<%@include file="header.jsp"
|
||||
%>
|
||||
|
||||
<br clear="all">
|
||||
<table>
|
||||
|
||||
<%
|
||||
|
||||
final SolrCore solrcore = core;
|
||||
|
||||
%>
|
||||
<%
|
||||
if ("false".equals(detailsMap.get("isMaster")))
|
||||
if (detailsMap != null) {%>
|
||||
<tr>
|
||||
<td>
|
||||
<strong>Master</strong>
|
||||
</td>
|
||||
<td>
|
||||
<%
|
||||
out.println((String) detailsMap.get("masterUrl"));
|
||||
%>
|
||||
</td>
|
||||
</tr>
|
||||
|
||||
<tr>
|
||||
<%
|
||||
NamedList nl = (NamedList) detailsMap.get("masterDetails");
|
||||
if (nl != null) {
|
||||
long masterVersion = (Long) nl.get("indexversion");
|
||||
long masterGeneration = (Long) nl.get("generation");
|
||||
long replicatableMasterVer = 0, replicatableMasterGen = 0;
|
||||
if (nl.get("replicatableindexversion") != null)
|
||||
replicatableMasterVer = (Long) nl.get("replicatableindexversion");
|
||||
if (nl.get("replicatablegeneration") != null)
|
||||
replicatableMasterGen = (Long) nl.get("replicatablegeneration");
|
||||
%>
|
||||
<td>
|
||||
</td>
|
||||
<td>Latest Index Version:<%=masterVersion%>, Generation: <%=masterGeneration%>
|
||||
</td>
|
||||
</tr>
|
||||
|
||||
<tr>
|
||||
<td></td>
|
||||
<td>Replicatable Index Version:<%=replicatableMasterVer%>, Generation: <%=replicatableMasterGen%>
|
||||
</td>
|
||||
</tr>
|
||||
<%}%>
|
||||
|
||||
<tr>
|
||||
<td>
|
||||
<strong>Poll Interval</strong>
|
||||
</td>
|
||||
<td>
|
||||
<%
|
||||
out.println((String) detailsMap.get("pollInterval"));
|
||||
%>
|
||||
</td>
|
||||
</tr>
|
||||
<%}%>
|
||||
|
||||
<tr>
|
||||
<td>
|
||||
<strong>Local Index</strong>
|
||||
</td>
|
||||
<td>
|
||||
<%
|
||||
if (detailsMap != null)
|
||||
out.println("Index Version: " + detailsMap.get("indexversion") + ", Generation: " + detailsMap.get("generation"));
|
||||
%>
|
||||
</td>
|
||||
</tr>
|
||||
|
||||
<tr>
|
||||
<td>
|
||||
</td>
|
||||
<td>
|
||||
<% if (null != core.getIndexDir()) {
|
||||
File dir = new File(core.getIndexDir());
|
||||
out.println("Location: " + dir.getCanonicalPath());
|
||||
}%>
|
||||
</td>
|
||||
</tr>
|
||||
|
||||
<tr>
|
||||
<td></td>
|
||||
<td><% if (detailsMap != null)
|
||||
out.println("Size: " + detailsMap.get("indexSize"));
|
||||
%>
|
||||
</td>
|
||||
</tr>
|
||||
|
||||
<%
|
||||
if ("true".equals(detailsMap.get("isMaster")))
|
||||
if (detailsMap != null) {
|
||||
%>
|
||||
|
||||
<tr>
|
||||
<td></td>
|
||||
<td>
|
||||
<%out.println("Config Files To Replicate: " + detailsMap.get("confFiles"));%>
|
||||
</td>
|
||||
</tr>
|
||||
|
||||
<tr>
|
||||
<td></td>
|
||||
<td>
|
||||
<%out.println("Trigger Replication On: " + detailsMap.get("replicateAfter")); %>
|
||||
</td>
|
||||
</tr>
|
||||
<%}%>
|
||||
|
||||
<%
|
||||
if ("false".equals(detailsMap.get("isMaster")))
|
||||
if (detailsMap != null) {%>
|
||||
<tr>
|
||||
<td>
|
||||
</td>
|
||||
<td>
|
||||
<%
|
||||
out.println("Times Replicated Since Startup: " + detailsMap.get("timesIndexReplicated"));
|
||||
%>
|
||||
</td>
|
||||
</tr>
|
||||
|
||||
<tr>
|
||||
<td>
|
||||
</td>
|
||||
<td>
|
||||
<%
|
||||
out.println("Previous Replication Done At: " + detailsMap.get("indexReplicatedAt"));
|
||||
%>
|
||||
</td>
|
||||
</tr>
|
||||
|
||||
<tr>
|
||||
<td>
|
||||
</td>
|
||||
<td>
|
||||
<%
|
||||
out.println("Config Files Replicated At: " + detailsMap.get("confFilesReplicatedAt"));
|
||||
%>
|
||||
</td>
|
||||
</tr>
|
||||
|
||||
<tr>
|
||||
<td>
|
||||
</td>
|
||||
<td>
|
||||
<%
|
||||
out.println("Config Files Replicated: " + detailsMap.get("confFilesReplicated"));
|
||||
%>
|
||||
</td>
|
||||
</tr>
|
||||
|
||||
<tr>
|
||||
<td>
|
||||
</td>
|
||||
<td>
|
||||
<%
|
||||
out.println("Times Config Files Replicated Since Startup: " + detailsMap.get("timesConfigReplicated"));
|
||||
%>
|
||||
</td>
|
||||
</tr>
|
||||
|
||||
<tr>
|
||||
<td>
|
||||
</td>
|
||||
<td>
|
||||
<%
|
||||
if (detailsMap.get("nextExecutionAt") != null)
|
||||
if (detailsMap.get("nextExecutionAt") != "")
|
||||
out.println("Next Replication Cycle At: " + detailsMap.get("nextExecutionAt"));
|
||||
else if ("true".equals(detailsMap.get("isPollingDisabled")))
|
||||
out.println("Next Replication Cycle At: Polling disabled.");
|
||||
else {
|
||||
NamedList nl1 = (NamedList) detailsMap.get("masterDetails");
|
||||
out.println("Next Replication Cycle At: After " + nl1.get("replicateAfter") + " on master.");
|
||||
}
|
||||
%>
|
||||
</td>
|
||||
</tr>
|
||||
|
||||
<%
|
||||
if ("true".equals(detailsMap.get("isReplicating"))) {
|
||||
%>
|
||||
<tr>
|
||||
<td><strong>Current Replication Status</strong>
|
||||
|
||||
<td>
|
||||
<%out.println("Start Time: " + detailsMap.get("replicationStartTime"));%>
|
||||
</td>
|
||||
</tr>
|
||||
|
||||
<tr>
|
||||
<td></td>
|
||||
<td>
|
||||
<%
|
||||
out.println("Files Downloaded: " + detailsMap.get("numFilesDownloaded") + " / " + detailsMap.get("numFilesToDownload"));%>
|
||||
</td>
|
||||
</tr>
|
||||
|
||||
<tr>
|
||||
<td></td>
|
||||
<td>
|
||||
<%
|
||||
out.println("Downloaded: " + detailsMap.get("bytesDownloaded") + " / " + detailsMap.get("bytesToDownload") + " [" + detailsMap.get("totalPercent") + "%]");%>
|
||||
</td>
|
||||
</tr>
|
||||
|
||||
<tr>
|
||||
<td></td>
|
||||
<td>
|
||||
<%
|
||||
out.println("Downloading File: " + detailsMap.get("currentFile") + ", Downloaded: " + detailsMap.get("currentFileSizeDownloaded") + " / " + detailsMap.get("currentFileSize") + " [" + detailsMap.get("currentFileSizePercent") + "%]");%>
|
||||
</td>
|
||||
</tr>
|
||||
|
||||
<tr>
|
||||
<td></td>
|
||||
<td>
|
||||
<%
|
||||
out.println("Time Elapsed: " + detailsMap.get("timeElapsed") + ", Estimated Time Remaining: " + detailsMap.get("timeRemaining") + ", Speed: " + detailsMap.get("downloadSpeed") + "/s");%>
|
||||
</td>
|
||||
</tr>
|
||||
<%}%>
|
||||
|
||||
<tr>
|
||||
<td><strong>Controls</strong>
|
||||
</td>
|
||||
<td><%
|
||||
String pollVal = request.getParameter("poll");
|
||||
if (pollVal != null)
|
||||
if (pollVal.equals("disable"))
|
||||
executeCommand("disablepoll", core, rh);
|
||||
else if (pollVal.equals("enable"))
|
||||
executeCommand("enablepoll", core, rh);
|
||||
if ("false".equals(detailsMap.get("isPollingDisabled"))) {
|
||||
%>
|
||||
|
||||
<form name=polling method="POST" action="./index.jsp" accept-charset="UTF-8">
|
||||
<input name="poll" type="hidden" value="disable">
|
||||
<input class="stdbutton" type="submit" value="Disable Poll">
|
||||
</form>
|
||||
|
||||
<%}%>
|
||||
<%
|
||||
if ("true".equals(detailsMap.get("isPollingDisabled"))) {
|
||||
%>
|
||||
|
||||
<form name=polling method="POST" action="./index.jsp" accept-charset="UTF-8">
|
||||
<input name="poll" type="hidden" value="enable">
|
||||
<input class="stdbutton" type="submit" value="Enable Poll">
|
||||
</form>
|
||||
<%
|
||||
}
|
||||
%>
|
||||
|
||||
</td>
|
||||
</tr>
|
||||
|
||||
<tr>
|
||||
<td></td>
|
||||
<td>
|
||||
<form name=replicate method="POST" action="./index.jsp" accept-charset="UTF-8">
|
||||
<input name="replicate" type="hidden" value="now">
|
||||
<input name="replicateButton" class="stdbutton" type="submit" value="Replicate Now">
|
||||
</form>
|
||||
<%
|
||||
if ("true".equals(detailsMap.get("isReplicating"))) {
|
||||
%>
|
||||
<script type="text/javascript">
|
||||
document["replicate"].replicateButton.disabled = true;
|
||||
document["replicate"].replicateButton.className = 'stdbuttondis';
|
||||
</script>
|
||||
<form name=abort method="POST" action="./index.jsp" accept-charset="UTF-8">
|
||||
<input name="abort" type="hidden" value="stop">
|
||||
<input name="abortButton" class="stdbutton" type="submit" value="Abort">
|
||||
</form>
|
||||
|
||||
<%} else {%>
|
||||
<script type="text/javascript">
|
||||
document["replicate"].replicateButton.disabled = false;
|
||||
document["replicate"].replicateButton.className = 'stdbutton';
|
||||
</script>
|
||||
<%
|
||||
}
|
||||
String replicateParam = request.getParameter("replicate");
|
||||
String abortParam = request.getParameter("abort");
|
||||
if (replicateParam != null)
|
||||
if (replicateParam.equals("now")) {
|
||||
new Thread() {
|
||||
public void run() {
|
||||
executeCommand("snappull", solrcore, rh);
|
||||
}
|
||||
}.start();
|
||||
}
|
||||
if (abortParam != null)
|
||||
if (abortParam.equals("stop")) {
|
||||
new Thread() {
|
||||
public void run() {
|
||||
executeCommand("abortsnappull", solrcore, rh);
|
||||
}
|
||||
}.start();
|
||||
}
|
||||
|
||||
%>
|
||||
</td>
|
||||
|
||||
</tr>
|
||||
|
||||
<%}%>
|
||||
|
||||
<%-- List the cores (that arent this one) so we can switch --%>
|
||||
<% org.apache.solr.core.CoreContainer cores = (org.apache.solr.core.CoreContainer) request.getAttribute("org.apache.solr.CoreContainer");
|
||||
if (cores != null) {
|
||||
Collection<String> names = cores.getCoreNames();
|
||||
if (names.size() > 1) {%>
|
||||
<tr>
|
||||
<td><strong>Cores:</strong><br></td>
|
||||
<td><%
|
||||
for (String name : names) {
|
||||
%>[<a href="../../<%=name%>/admin/"><%=name%>
|
||||
</a>]<%
|
||||
}%></td>
|
||||
</tr>
|
||||
<%
|
||||
}
|
||||
}%>
|
||||
|
||||
|
||||
</table>
|
||||
<P>
|
||||
|
||||
<p>
|
||||
|
||||
<table>
|
||||
<tr>
|
||||
<td>
|
||||
</td>
|
||||
<td>
|
||||
Current Time: <%= new Date() %>
|
||||
</td>
|
||||
</tr>
|
||||
<tr>
|
||||
<td>
|
||||
</td>
|
||||
<td>
|
||||
Server Start At: <%= new Date(core.getStartTime()) %>
|
||||
</td>
|
||||
</tr>
|
||||
</table>
|
||||
|
||||
<br>
|
||||
<a href="..">Return to Admin Page</a>
|
||||
</body>
|
||||
</html>
|
|
@ -51,6 +51,15 @@ input.stdbutton:hover {
|
|||
border: groove #0000ff;
|
||||
}
|
||||
|
||||
input.stdbuttondis{
|
||||
font-family: ITC Officina Sans Book, Helvetica, Arial, sans-serif;
|
||||
font-style: bold;
|
||||
font-size: 11;
|
||||
text-transform: capitalize;
|
||||
color: #8B8B83;
|
||||
background-color: #dddddd;
|
||||
border: groove #8B8B83;
|
||||
}
|
||||
|
||||
body {
|
||||
background-color: #bbbbbb;
|
||||
|
|
Loading…
Reference in New Issue