SOLR-215 - remove static singleton for SolrCore. This commit enables multiple solr cores, but does not yet provide an interface to manage them. This is a rather substantial change.

git-svn-id: https://svn.apache.org/repos/asf/lucene/solr/trunk@573893 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Ryan McKinley 2007-09-08 19:03:54 +00:00
parent e595f5e7ce
commit 5f003559b5
83 changed files with 1105 additions and 659 deletions

View File

@ -32,7 +32,6 @@ import org.apache.solr.common.params.DefaultSolrParams;
import org.apache.solr.common.params.ModifiableSolrParams;
import org.apache.solr.common.params.SolrParams;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.core.SolrConfig;
import org.apache.solr.core.SolrCore;
import org.apache.solr.request.QueryResponseWriter;
import org.apache.solr.request.SolrQueryRequest;
@ -60,7 +59,7 @@ public class EmbeddedSolrServer extends BaseSolrServer
public EmbeddedSolrServer( SolrCore core )
{
this.core = core;
this.parser = new SolrRequestParsers( core, SolrConfig.config );
this.parser = new SolrRequestParsers( core );
// by default use the XML one
_processor = new XMLResponseParser();

View File

@ -36,7 +36,7 @@ public class TestEmbeddedSolrServer extends SolrExampleTestBase {
super.setUp();
// setup the server...
server = new EmbeddedSolrServer( SolrCore.getSolrCore() );
server = new EmbeddedSolrServer( h.getCore() );
}
@Override

View File

@ -34,6 +34,11 @@ public class TestJettySolrRunner extends SolrExampleTestBase {
SolrServer server;
JettySolrRunner jetty;
@Override
public String getCoreName() {
return null;
}
@Override public void setUp() throws Exception
{
super.setUp();

View File

@ -19,16 +19,32 @@ package org.apache.solr.analysis;
import java.util.Map;
import java.util.logging.Logger;
import org.apache.solr.core.SolrConfig;
/**
* Simple abstract implementation that handles init arg processing.
*
* @version $Id$
*/
public abstract class BaseTokenFilterFactory implements TokenFilterFactory {
public abstract class BaseTokenFilterFactory implements TokenFilterFactory, SolrConfig.Initializable {
final static Logger log = Logger.getLogger(BaseTokenFilterFactory.class.getName());
/** The init args */
protected Map<String,String> args;
@Deprecated
public void init(Map<String,String> args) {
log.warning("calling the deprecated form of init; should be calling init(SolrConfig solrConfig, Map<String,String> args)");
this.args=args;
}
/**
* @since solr 1.3
*/
public void init(SolrConfig solrConfig, Map<String,String> args) {
this.args=args;
}

View File

@ -18,16 +18,28 @@
package org.apache.solr.analysis;
import java.util.Map;
import java.util.logging.Logger;
import org.apache.solr.core.SolrConfig;
/**
* Simple abstract implementation that handles init arg processing.
*
* @version $Id$
*/
public abstract class BaseTokenizerFactory implements TokenizerFactory {
public abstract class BaseTokenizerFactory implements TokenizerFactory, SolrConfig.Initializable {
final static Logger log = Logger.getLogger(BaseTokenizerFactory.class.getName());
/** The init args */
protected Map<String,String> args;
@Deprecated
public void init(Map<String,String> args) {
log.warning("calling the deprecated form of init; should be calling init(SolrConfig solrConfig, Map<String,String> args)");
this.args=args;
}
public void init(SolrConfig solrConfig, Map<String,String> args) {
this.args=args;
}

View File

@ -27,6 +27,7 @@ import java.util.StringTokenizer;
import org.apache.lucene.analysis.Token;
import org.apache.lucene.analysis.TokenFilter;
import org.apache.lucene.analysis.TokenStream;
import org.apache.solr.core.SolrConfig;
import org.apache.solr.analysis.BaseTokenFilterFactory;
/**
@ -71,8 +72,8 @@ public class CapitalizationFilterFactory extends BaseTokenFilterFactory
boolean forceFirstLetter = true; // make sure the first letter is capitol even if it is in the keep list
@Override
public void init(Map<String,String> args) {
super.init( args );
public void init(SolrConfig solrConfig, Map<String,String> args) {
super.init( solrConfig, args );
String k = args.get( KEEP );
if( k != null ) {

View File

@ -1,55 +1,55 @@
package org.apache.solr.analysis;
/**
* 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.
*/
import java.util.Map;
import org.apache.lucene.analysis.TokenStream;
import org.apache.lucene.analysis.ngram.EdgeNGramTokenFilter;
/**
* Creates new instances of {@link EdgeNGramTokenFilter}.
*/
public class EdgeNGramFilterFactory extends BaseTokenFilterFactory {
private int maxGramSize = 0;
private int minGramSize = 0;
private String side;
@Override
public void init(Map<String, String> args) {
super.init(args);
String maxArg = args.get("maxGramSize");
maxGramSize = (maxArg != null ? Integer.parseInt(maxArg)
: EdgeNGramTokenFilter.DEFAULT_MAX_GRAM_SIZE);
String minArg = args.get("minGramSize");
minGramSize = (minArg != null ? Integer.parseInt(minArg)
: EdgeNGramTokenFilter.DEFAULT_MIN_GRAM_SIZE);
side = args.get("side");
if (side == null) {
side = EdgeNGramTokenFilter.Side.FRONT.getLabel();
}
}
public TokenStream create(TokenStream input) {
return new EdgeNGramTokenFilter(input, side, minGramSize, maxGramSize);
}
}
package org.apache.solr.analysis;
/**
* 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.
*/
import java.util.Map;
import org.apache.solr.core.SolrConfig;
import org.apache.lucene.analysis.TokenStream;
import org.apache.lucene.analysis.ngram.EdgeNGramTokenFilter;
/**
* Creates new instances of {@link EdgeNGramTokenFilter}.
*/
public class EdgeNGramFilterFactory extends BaseTokenFilterFactory {
private int maxGramSize = 0;
private int minGramSize = 0;
private String side;
@Override
public void init(SolrConfig solrConfig, Map<String, String> args) {
super.init(solrConfig, args);
String maxArg = args.get("maxGramSize");
maxGramSize = (maxArg != null ? Integer.parseInt(maxArg)
: EdgeNGramTokenFilter.DEFAULT_MAX_GRAM_SIZE);
String minArg = args.get("minGramSize");
minGramSize = (minArg != null ? Integer.parseInt(minArg)
: EdgeNGramTokenFilter.DEFAULT_MIN_GRAM_SIZE);
side = args.get("side");
if (side == null) {
side = EdgeNGramTokenFilter.Side.FRONT.getLabel();
}
}
public TokenStream create(TokenStream input) {
return new EdgeNGramTokenFilter(input, side, minGramSize, maxGramSize);
}
}

View File

@ -17,6 +17,7 @@ package org.apache.solr.analysis;
* limitations under the License.
*/
import org.apache.solr.core.SolrConfig;
import org.apache.lucene.analysis.TokenStream;
import org.apache.lucene.analysis.ngram.EdgeNGramTokenizer;
@ -33,8 +34,8 @@ public class EdgeNGramTokenizerFactory extends BaseTokenizerFactory {
private String side;
public void init(Map<String, String> args) {
super.init(args);
public void init(SolrConfig solrConfig, Map<String, String> args) {
super.init(solrConfig, args);
String maxArg = args.get("maxGramSize");
maxGramSize = (maxArg != null ? Integer.parseInt(maxArg) : EdgeNGramTokenizer.DEFAULT_MAX_GRAM_SIZE);

View File

@ -17,7 +17,7 @@
package org.apache.solr.analysis;
import org.apache.solr.core.Config;
import org.apache.solr.core.SolrConfig;
import org.apache.lucene.analysis.StopFilter;
import org.apache.lucene.analysis.TokenStream;
import org.apache.lucene.analysis.TokenFilter;
@ -32,12 +32,13 @@ import java.io.IOException;
* @version $Id$
*/
public class EnglishPorterFilterFactory extends BaseTokenFilterFactory {
public void init(Map<String, String> args) {
super.init(args);
@Override
public void init(SolrConfig solrConfig, Map<String, String> args) {
super.init(solrConfig, args);
String wordFile = args.get("protected");
if (wordFile != null) {
try {
List<String> wlist = Config.getLines(wordFile);
List<String> wlist = solrConfig.getLines(wordFile);
protectedWords = StopFilter.makeStopSet((String[])wlist.toArray(new String[0]));
} catch (IOException e) {
throw new RuntimeException(e);

View File

@ -17,7 +17,7 @@
package org.apache.solr.analysis;
import org.apache.solr.core.Config;
import org.apache.solr.core.SolrConfig;
import org.apache.lucene.analysis.StopFilter;
import org.apache.lucene.analysis.TokenStream;
@ -37,14 +37,14 @@ public class KeepWordFilterFactory extends BaseTokenFilterFactory {
@Override
@SuppressWarnings("unchecked")
public void init(Map<String, String> args) {
super.init(args);
public void init(SolrConfig config, Map<String, String> args) {
super.init(config, args);
String wordFile = args.get("words");
ignoreCase = getBoolean("ignoreCase",false);
if (wordFile != null) {
try {
List<String> wlist = Config.getLines(wordFile);
List<String> wlist = config.getLines(wordFile);
words = StopFilter.makeStopSet(
(String[])wlist.toArray(new String[0]), ignoreCase);
}

View File

@ -17,6 +17,7 @@
package org.apache.solr.analysis;
import org.apache.solr.core.SolrConfig;
import org.apache.lucene.analysis.TokenStream;
import org.apache.lucene.analysis.LengthFilter;
@ -27,8 +28,8 @@ import java.util.Map;
*/
public class LengthFilterFactory extends BaseTokenFilterFactory {
int min,max;
public void init(Map<String, String> args) {
super.init(args);
public void init(SolrConfig solrConfig, Map<String, String> args) {
super.init(solrConfig, args);
min=Integer.parseInt(args.get("min"));
max=Integer.parseInt(args.get("max"));
}

View File

@ -1,49 +1,49 @@
package org.apache.solr.analysis;
/**
* 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.
*/
import java.util.Map;
import org.apache.lucene.analysis.TokenStream;
import org.apache.lucene.analysis.ngram.NGramTokenFilter;
/**
* Creates new instances of {@link NGramTokenFilter}.
*/
public class NGramFilterFactory extends BaseTokenFilterFactory {
private int maxGramSize = 0;
private int minGramSize = 0;
/** Initializes the n-gram min and max sizes and the side from which one should start tokenizing. */
@Override
public void init(Map<String, String> args) {
super.init(args);
String maxArg = args.get("maxGramSize");
maxGramSize = (maxArg != null ? Integer.parseInt(maxArg)
: NGramTokenFilter.DEFAULT_MAX_NGRAM_SIZE);
String minArg = args.get("minGramSize");
minGramSize = (minArg != null ? Integer.parseInt(minArg)
: NGramTokenFilter.DEFAULT_MIN_NGRAM_SIZE);
}
public TokenStream create(TokenStream input) {
return new NGramTokenFilter(input, minGramSize, maxGramSize);
}
}
package org.apache.solr.analysis;
/**
* 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.
*/
import java.util.Map;
import org.apache.solr.core.SolrConfig;
import org.apache.lucene.analysis.TokenStream;
import org.apache.lucene.analysis.ngram.NGramTokenFilter;
/**
* Creates new instances of {@link NGramTokenFilter}.
*/
public class NGramFilterFactory extends BaseTokenFilterFactory {
private int maxGramSize = 0;
private int minGramSize = 0;
/** Initializes the n-gram min and max sizes and the side from which one should start tokenizing. */
@Override
public void init(SolrConfig solrConfig, Map<String, String> args) {
super.init(solrConfig, args);
String maxArg = args.get("maxGramSize");
maxGramSize = (maxArg != null ? Integer.parseInt(maxArg)
: NGramTokenFilter.DEFAULT_MAX_NGRAM_SIZE);
String minArg = args.get("minGramSize");
minGramSize = (minArg != null ? Integer.parseInt(minArg)
: NGramTokenFilter.DEFAULT_MIN_NGRAM_SIZE);
}
public TokenStream create(TokenStream input) {
return new NGramTokenFilter(input, minGramSize, maxGramSize);
}
}

View File

@ -17,6 +17,7 @@ package org.apache.solr.analysis;
* limitations under the License.
*/
import org.apache.solr.core.SolrConfig;
import org.apache.lucene.analysis.TokenStream;
import org.apache.lucene.analysis.ngram.NGramTokenizer;
@ -31,8 +32,8 @@ public class NGramTokenizerFactory extends BaseTokenizerFactory {
private int minGramSize = 0;
/** Initializes the n-gram min and max sizes and the side from which one should start tokenizing. */
public void init(Map<String, String> args) {
super.init(args);
public void init(SolrConfig solrConfig, Map<String, String> args) {
super.init(solrConfig, args);
String maxArg = args.get("maxGramSize");
maxGramSize = (maxArg != null ? Integer.parseInt(maxArg) : NGramTokenizer.DEFAULT_MAX_NGRAM_SIZE);

View File

@ -16,7 +16,7 @@
*/
package org.apache.solr.analysis;
import org.apache.solr.core.SolrConfig;
import org.apache.lucene.analysis.TokenStream;
import java.util.Map;
@ -32,8 +32,8 @@ public class PatternReplaceFilterFactory extends BaseTokenFilterFactory {
String replacement;
boolean all = true;
public void init(Map<String, String> args) {
super.init(args);
public void init(SolrConfig solrConfig, Map<String, String> args) {
super.init(solrConfig, args);
try {
p = Pattern.compile(args.get("pattern"));
} catch (PatternSyntaxException e) {

View File

@ -21,6 +21,7 @@ import org.apache.commons.io.IOUtils;
import org.apache.lucene.analysis.Token;
import org.apache.lucene.analysis.TokenStream;
import org.apache.solr.common.SolrException;
import org.apache.solr.core.SolrConfig;
import java.io.IOException;
import java.io.Reader;
@ -57,7 +58,7 @@ import java.util.regex.Pattern;
* @since solr1.2
* @version $Id:$
*/
public class PatternTokenizerFactory implements TokenizerFactory
public class PatternTokenizerFactory extends BaseTokenizerFactory
{
public static final String PATTERN = "pattern";
public static final String GROUP = "group";
@ -69,7 +70,7 @@ public class PatternTokenizerFactory implements TokenizerFactory
/**
* Require a configured pattern
*/
public void init(Map<String,String> args)
public void init(SolrConfig solrConfig, Map<String,String> args)
{
this.args = args;
String regex = args.get( PATTERN );

View File

@ -20,6 +20,7 @@ package org.apache.solr.analysis;
import java.util.HashMap;
import java.util.Map;
import org.apache.solr.core.SolrConfig;
import org.apache.commons.codec.Encoder;
import org.apache.commons.codec.language.DoubleMetaphone;
import org.apache.commons.codec.language.Metaphone;
@ -60,8 +61,8 @@ public class PhoneticFilterFactory extends BaseTokenFilterFactory
protected Encoder encoder = null;
@Override
public void init(Map<String,String> args) {
super.init( args );
public void init(SolrConfig solrConfig, Map<String,String> args) {
super.init( solrConfig, args );
if( args.get( "inject" ) != null ) {
inject = Boolean.getBoolean( args.get( INJECT ) );

View File

@ -21,6 +21,7 @@ import java.util.Map;
import org.apache.lucene.analysis.TokenStream;
import org.apache.lucene.analysis.snowball.SnowballFilter;
import org.apache.solr.core.SolrCore;
import org.apache.solr.core.SolrConfig;
/**
* Factory for SnowballFilters, with configurable language
@ -34,8 +35,8 @@ import org.apache.solr.core.SolrCore;
public class SnowballPorterFilterFactory extends BaseTokenFilterFactory {
private String language = "English";
public void init(Map<String, String> args) {
super.init(args);
public void init(SolrConfig solrConfig, Map<String, String> args) {
super.init(solrConfig, args);
final String cfgLanguage = args.get("language");
if(cfgLanguage!=null) language = cfgLanguage;
SolrCore.log.fine("SnowballPorterFilterFactory: language=" + language);

View File

@ -17,7 +17,7 @@
package org.apache.solr.analysis;
import org.apache.solr.core.Config;
import org.apache.solr.core.SolrConfig;
import org.apache.lucene.analysis.StopFilter;
import org.apache.lucene.analysis.StopAnalyzer;
import org.apache.lucene.analysis.TokenStream;
@ -31,14 +31,16 @@ import java.io.IOException;
* @version $Id$
*/
public class StopFilterFactory extends BaseTokenFilterFactory {
public void init(Map<String, String> args) {
super.init(args);
@Override
public void init(SolrConfig solrConfig, Map<String, String> args) {
super.init(solrConfig, args);
String stopWordFile = args.get("words");
ignoreCase = getBoolean("ignoreCase",false);
if (stopWordFile != null) {
try {
List<String> wlist = Config.getLines(stopWordFile);
List<String> wlist = solrConfig.getLines(stopWordFile);
stopWords = StopFilter.makeStopSet((String[])wlist.toArray(new String[0]), ignoreCase);
} catch (IOException e) {
throw new RuntimeException(e);

View File

@ -18,8 +18,8 @@
package org.apache.solr.analysis;
import org.apache.lucene.analysis.TokenStream;
import org.apache.solr.core.SolrConfig;
import org.apache.solr.common.util.StrUtils;
import org.apache.solr.core.Config;
import org.apache.solr.core.SolrCore;
import java.io.IOException;
@ -30,8 +30,9 @@ import java.util.Map;
* @version $Id$
*/
public class SynonymFilterFactory extends BaseTokenFilterFactory {
public void init(Map<String, String> args) {
super.init(args);
@Override
public void init(SolrConfig solrConfig, Map<String, String> args) {
super.init(solrConfig, args);
String synonyms = args.get("synonyms");
ignoreCase = getBoolean("ignoreCase",false);
@ -40,7 +41,7 @@ public class SynonymFilterFactory extends BaseTokenFilterFactory {
if (synonyms != null) {
List<String> wlist=null;
try {
wlist = Config.getLines(synonyms);
wlist = solrConfig.getLines(synonyms);
} catch (IOException e) {
throw new RuntimeException(e);
}
@ -50,7 +51,6 @@ public class SynonymFilterFactory extends BaseTokenFilterFactory {
SolrCore.log.fine("SynonymMap "+synonyms +":"+synMap);
}
}
}
private SynonymMap synMap;

View File

@ -18,7 +18,7 @@
package org.apache.solr.analysis;
import org.apache.lucene.analysis.TokenStream;
import org.apache.solr.core.SolrConfig;
import java.util.Map;
/**
@ -53,7 +53,10 @@ public interface TokenFilterFactory {
* <p>The args are user-level initialization parameters that
* may be specified when declaring a the factory in the
* schema.xml
* This method is deprecated; the replacement is to have your class implement SolrConfig.Initializable
* and implement 'void init(SolrConfig solrConfig, Map<String,String> args);'
*/
@Deprecated
public void init(Map<String,String> args);
/**
* Accessor method for reporting the args used to initialize this factory.

View File

@ -19,7 +19,7 @@ package org.apache.solr.analysis;
import java.io.*;
import java.util.Map;
import org.apache.solr.core.SolrConfig;
import org.apache.lucene.analysis.*;
@ -52,7 +52,10 @@ public interface TokenizerFactory {
* <p>The args are user-level initialization parameters that
* may be specified when declaring a the factory in the
* schema.xml
* This method is deprecated; the replacement is to have your class implement SolrConfig.Initializable
* and implement 'void init(SolrConfig solrConfig, Map<String,String> args);'
*/
@Deprecated
public void init(Map<String,String> args);
/**
* Accessor method for reporting the args used to initialize this factory.

View File

@ -19,6 +19,7 @@ package org.apache.solr.analysis;
import java.util.Map;
import org.apache.solr.core.SolrConfig;
import org.apache.lucene.analysis.TokenStream;
import org.apache.solr.common.SolrException;
@ -31,8 +32,8 @@ public class TrimFilterFactory extends BaseTokenFilterFactory {
protected boolean updateOffsets = false;
@Override
public void init(Map<String,String> args) {
super.init( args );
public void init(SolrConfig solrConfig, Map<String,String> args) {
super.init(solrConfig, args );
String v = args.get( "updateOffsets" );
if( v != null ) {

View File

@ -16,7 +16,7 @@
*/
package org.apache.solr.analysis;
import org.apache.solr.core.SolrConfig;
import org.apache.lucene.analysis.TokenStream;
import java.util.Map;
@ -32,8 +32,8 @@ public class WordDelimiterFilterFactory extends BaseTokenFilterFactory {
int catenateAll=0;
int splitOnCaseChange=0;
public void init(Map<String, String> args) {
super.init(args);
public void init(SolrConfig solrConfig, Map<String, String> args) {
super.init(solrConfig, args);
generateWordParts = getInt("generateWordParts", 1);
generateNumberParts = getInt("generateNumberParts", 1);
catenateWords = getInt("catenateWords", 0);

View File

@ -23,6 +23,10 @@ import org.apache.solr.search.SolrIndexSearcher;
/**
*/
class AbstractSolrEventListener implements SolrEventListener {
protected final SolrCore core;
public AbstractSolrEventListener(SolrCore core) {
this.core = core;
}
protected NamedList args;
public void init(NamedList args) {

View File

@ -58,16 +58,22 @@ public class Config {
this.name = name;
this.prefix = prefix;
if (prefix!=null && !prefix.endsWith("/")) prefix += '/';
javax.xml.parsers.DocumentBuilder builder = DocumentBuilderFactory.newInstance().newDocumentBuilder();
doc = builder.parse(is);
InputStream lis = is;
try{
if (lis == null)
lis = openResource(name);
javax.xml.parsers.DocumentBuilder builder = DocumentBuilderFactory.newInstance().newDocumentBuilder();
doc = builder.parse(lis);
DOMUtil.substituteSystemProperties(doc);
}
catch( SolrException e ){
} catch( SolrException e ){
SolrException.log(log,"Error in "+name,e);
throw e;
} finally {
// if this opens the resource, it also closes it
if (lis != is)
lis.close();
}
}
@ -185,6 +191,75 @@ public class Config {
}
// The directory where solr will look for config files by default.
// defaults to "./solr/conf/"
public String getConfigDir() {
return getInstanceDir() + "conf/";
}
public InputStream openResource(String resource) {
InputStream is=null;
try {
File f = new File(resource);
if (!f.isAbsolute()) {
// try $CWD/solrconf/
f = new File(getConfigDir() + resource);
}
if (f.isFile() && f.canRead()) {
return new FileInputStream(f);
} else {
// try $CWD
f = new File(resource);
if (f.isFile() && f.canRead()) {
return new FileInputStream(f);
}
}
ClassLoader loader = getClassLoader();
is = loader.getResourceAsStream(resource);
} catch (Exception e) {
throw new RuntimeException("Error opening " + resource, e);
}
if (is==null) {
throw new RuntimeException("Can't find resource '" + resource + "' in classpath or '" + getConfigDir() + "', cwd="+System.getProperty("user.dir"));
}
return is;
}
/**
* Accesses a resource by name and returns the (non comment) lines
* containing data.
*
* <p>
* A comment line is any line that starts with the character "#"
* </p>
*
* @param resource
* @return a list of non-blank non-comment lines with whitespace trimmed
* from front and back.
* @throws IOException
*/
public List<String> getLines(String resource) throws IOException {
BufferedReader input = null;
try {
// todo - allow configurable charset?
input = new BufferedReader(new InputStreamReader(openResource(resource), "UTF-8"));
} catch (UnsupportedEncodingException e) {
throw new RuntimeException(e);
}
ArrayList<String> lines = new ArrayList<String>();
for (String word=null; (word=input.readLine())!=null;) {
// skip comments
if (word.startsWith("#")) continue;
word=word.trim();
// skip blank lines
if (word.length()==0) continue;
lines.add(word);
}
return lines;
}
//
// classloader related functions
//
@ -280,17 +355,10 @@ public class Config {
return instanceDir;
}
public static boolean isInstanceDirInitialized()
{
public static boolean isInstanceDirInitialized() {
return instanceDir != null;
}
// The directory where solr will look for config files by default.
// defaults to "./solr/conf/"
static String getConfigDir() {
return getInstanceDir() + "conf/";
}
/** Singleton classloader loading resources specified in any configs */
private static ClassLoader classLoader = null;
@ -306,7 +374,8 @@ public class Config {
*/
static ClassLoader getClassLoader() {
if (null == classLoader) {
classLoader = Thread.currentThread().getContextClassLoader();
// NB5.5/win32/1.5_10: need to go thru local var or classLoader is not set!
ClassLoader loader = Thread.currentThread().getContextClassLoader();
File f = new File(getInstanceDir() + "lib/");
if (f.canRead() && f.isDirectory()) {
@ -317,78 +386,14 @@ public class Config {
jars[j] = jarFiles[j].toURI().toURL();
log.info("Adding '" + jars[j].toString() + "' to Solr classloader");
}
classLoader = URLClassLoader.newInstance(jars, classLoader);
loader = URLClassLoader.newInstance(jars, loader);
} catch (MalformedURLException e) {
SolrException.log(log,"Can't construct solr lib class loader", e);
}
}
classLoader = loader;
}
return classLoader;
}
public static InputStream openResource(String resource) {
InputStream is=null;
try {
File f = new File(resource);
if (!f.isAbsolute()) {
// try $CWD/solrconf/
f = new File(getConfigDir() + resource);
}
if (f.isFile() && f.canRead()) {
return new FileInputStream(f);
} else {
// try $CWD
f = new File(resource);
if (f.isFile() && f.canRead()) {
return new FileInputStream(f);
}
}
ClassLoader loader = getClassLoader();
is = loader.getResourceAsStream(resource);
} catch (Exception e) {
throw new RuntimeException("Error opening " + resource, e);
}
if (is==null) {
throw new RuntimeException("Can't find resource '" + resource + "' in classpath or '" + getConfigDir() + "', cwd="+System.getProperty("user.dir"));
}
return is;
}
/**
* Accesses a resource by name and returns the (non comment) lines
* containing data.
*
* <p>
* A comment line is any line that starts with the character "#"
* </p>
*
* @param resource
* @return a list of non-blank non-comment lines with whitespace trimmed
* from front and back.
* @throws IOException
*/
public static List<String> getLines(String resource) throws IOException {
BufferedReader input = null;
try {
// todo - allow configurable charset?
input = new BufferedReader(new InputStreamReader(openResource(resource), "UTF-8"));
} catch (UnsupportedEncodingException e) {
throw new RuntimeException(e);
}
ArrayList<String> lines = new ArrayList<String>();
for (String word=null; (word=input.readLine())!=null;) {
// skip comments
if (word.startsWith("#")) continue;
word=word.trim();
// skip blank lines
if (word.length()==0) continue;
lines.add(word);
}
return lines;
}
}

View File

@ -30,7 +30,11 @@ import java.util.List;
* @version $Id$
*/
class QuerySenderListener extends AbstractSolrEventListener {
public QuerySenderListener(SolrCore core) {
super(core);
}
@Override
public void newSearcher(SolrIndexSearcher newSearcher, SolrIndexSearcher currentSearcher) {
final SolrIndexSearcher searcher = newSearcher;
SolrCore core = SolrCore.getSolrCore();
@ -39,11 +43,8 @@ class QuerySenderListener extends AbstractSolrEventListener {
try {
// bind the request to a particular searcher (the newSearcher)
LocalSolrQueryRequest req = new LocalSolrQueryRequest(core,nlst) {
public SolrIndexSearcher getSearcher() {
return searcher;
}
public void close() {
}
@Override public SolrIndexSearcher getSearcher() { return searcher; }
@Override public void close() { }
};
SolrQueryResponse rsp = new SolrQueryResponse();

View File

@ -43,7 +43,7 @@ final class RequestHandlers {
public static Logger log = Logger.getLogger(RequestHandlers.class.getName());
public static final String DEFAULT_HANDLER_NAME="standard";
protected final SolrCore core;
// Use a synchronized map - since the handlers can be changed at runtime,
// the map implementation should be thread safe
private final Map<String, SolrRequestHandler> handlers = Collections.synchronizedMap(
@ -66,6 +66,10 @@ final class RequestHandlers {
return p;
}
public RequestHandlers(SolrCore core) {
this.core = core;
}
/**
* @return the RequestHandler registered at the given name
*/
@ -89,7 +93,7 @@ final class RequestHandlers {
SolrRequestHandler old = handlers.put(norm, handler);
if (handlerName != null && handlerName != "") {
if (handler instanceof SolrInfoMBean) {
SolrInfoRegistry.getRegistry().put(handlerName, (SolrInfoMBean)handler);
core.getInfoRegistry().put(handlerName, (SolrInfoMBean)handler);
}
}
return old;
@ -129,20 +133,20 @@ final class RequestHandlers {
new AbstractPluginLoader<SolrRequestHandler>( "[solrconfig.xml] requestHandler", true, true )
{
@Override
protected SolrRequestHandler create( String name, String className, Node node ) throws Exception
protected SolrRequestHandler create( SolrCore core, String name, String className, Node node ) throws Exception
{
String startup = DOMUtil.getAttr( node, "startup" );
if( startup != null ) {
if( "lazy".equals( startup ) ) {
log.info("adding lazy requestHandler: " + className );
NamedList args = DOMUtil.childNodesToNamedList(node);
return new LazyRequestHandlerWrapper( className, args );
return new LazyRequestHandlerWrapper( core, className, args );
}
else {
throw new Exception( "Unknown startup value: '"+startup+"' for: "+className );
}
}
return super.create( name, className, node );
return super.create( core, name, className, node );
}
@Override
@ -159,7 +163,7 @@ final class RequestHandlers {
NodeList nodes = (NodeList)config.evaluate("requestHandler", XPathConstants.NODESET);
// Load the handlers and get the default one
SolrRequestHandler defaultHandler = loader.load( nodes );
SolrRequestHandler defaultHandler = loader.load( core, nodes );
if( defaultHandler == null ) {
defaultHandler = get(RequestHandlers.DEFAULT_HANDLER_NAME);
if( defaultHandler == null ) {
@ -194,12 +198,14 @@ final class RequestHandlers {
*/
private static final class LazyRequestHandlerWrapper implements SolrRequestHandler, SolrInfoMBean
{
private final SolrCore core;
private String _className;
private NamedList _args;
private SolrRequestHandler _handler;
public LazyRequestHandlerWrapper( String className, NamedList args )
public LazyRequestHandlerWrapper( SolrCore core, String className, NamedList args )
{
this.core = core;
_className = className;
_args = args;
_handler = null; // don't initialize
@ -223,8 +229,7 @@ final class RequestHandlers {
{
if( _handler == null ) {
try {
Class clazz = Config.findClass( _className, new String[]{} );
_handler = (SolrRequestHandler)clazz.newInstance();
_handler = (SolrRequestHandler)core.createRequestHandler(_className);
_handler.init( _args );
}
catch( Exception ex ) {

View File

@ -30,6 +30,9 @@ import java.util.logging.Level;
/**
*/
class RunExecutableListener extends AbstractSolrEventListener {
public RunExecutableListener(SolrCore core) {
super(core);
}
protected String[] cmd;
protected File dir;
protected String[] envp;

View File

@ -21,10 +21,15 @@ import org.apache.solr.common.util.NamedList;
import org.apache.solr.request.LocalSolrQueryRequest;
import org.apache.solr.request.SolrQueryRequest;
import org.apache.solr.search.CacheConfig;
import org.apache.solr.update.SolrIndexConfig;
import org.apache.lucene.search.BooleanQuery;
import org.xml.sax.SAXException;
import javax.xml.parsers.ParserConfigurationException;
import java.util.Map;
import java.util.Collection;
import java.util.HashSet;
import java.util.StringTokenizer;
@ -39,63 +44,119 @@ import java.io.InputStream;
*
* @version $Id$
*/
public class SolrConfig {
public class SolrConfig extends Config {
public static final String DEFAULT_CONF_FILE = "solrconfig.xml";
/**
* Singleton containing all configuration.
* Compatibility feature for single-core (pre-solr215 patch) code.
* Most usage should be converted by:
* - using the configuration directly when used in Abstract{Tokeinizer,TokenFilter}Factory.init().
* - getting the configuration through the owning core if accessible (SolrCore.getSolrConfig()).
* - getting the core by name then its configuration as above
*/
public static Config config;
@Deprecated
public static SolrConfig config = null;
/** An interface to denote objects that need a SolrConfig to be initialized.
* These are mainly TokenFilterFactory and TokenizerFactory subclasses.
*/
public interface Initializable {
/** <code>init</code> will be called just once, immediately after creation.
*/
void init(SolrConfig solrConfig, Map<String,String> args);
}
public final String configFile;
/**
* Singleton keeping track of configuration errors
*/
public static final Collection<Throwable> severeErrors = new HashSet<Throwable>();
/**
* (Re)loads the static configuration information from the specified file.
*
* <p>
* This method is called implicitly on ClassLoad, but it may be
* called explicitly to change the Configuration used for the purpose
* of testing - in which case it should be called prior to initializing
* a SolrCore.
* </p>
*
* <p>
* This method should <b>only</b> be called for testing purposes.
* Because it modifies a singleton, it is not suitable for running
* multi-threaded tests.
* </p>
*
* Creates a configation information from the specified file.
* Add any exception that might occur to the severeErrors singleton.
* @param file file name to load
* @see Config#openResource
*/
public static synchronized void initConfig(String file)
throws ParserConfigurationException, IOException, SAXException {
InputStream is = Config.openResource(file);
config=new Config(file, is, "/config/");
is.close();
Config.log.info("Loaded SolrConfig: " + file);
}
static {
public static SolrConfig createInstance(String file) {
try {
initConfig(DEFAULT_CONF_FILE);
return new SolrConfig(file);
} catch (Exception ee) {
severeErrors.add( ee );
throw new RuntimeException("Error in " + DEFAULT_CONF_FILE, ee);
throw new RuntimeException("Error in " + file, ee);
}
}
/**
* Returns a Request object based on the admin/pingQuery section
* of the Solr config file.
*/
public static SolrQueryRequest getPingQueryRequest(SolrCore core) {
/** Creates a default instance from the solrconfig.xml. */
public SolrConfig()
throws ParserConfigurationException, IOException, SAXException {
this(DEFAULT_CONF_FILE);
}
/** Creates a configuration instance from a file. */
public SolrConfig(String file)
throws ParserConfigurationException, IOException, SAXException {
this(file, null);
}
/** Creates a configuration instance from an input stream. */
public SolrConfig(String file, InputStream is)
throws ParserConfigurationException, IOException, SAXException {
super(file, is, "/config/");
this.configFile = file;
defaultIndexConfig = new SolrIndexConfig(this, null, null);
mainIndexConfig = new SolrIndexConfig(this, "mainIndex", defaultIndexConfig);
booleanQueryMaxClauseCount = getInt("query/maxBooleanClauses", BooleanQuery.getMaxClauseCount());
filtOptEnabled = getBool("query/boolTofilterOptimizer/@enabled", false);
filtOptCacheSize = getInt("query/boolTofilterOptimizer/@cacheSize",32);
filtOptThreshold = getFloat("query/boolTofilterOptimizer/@threshold",.05f);
useFilterForSortedQuery = getBool("query/useFilterForSortedQuery", false);
queryResultWindowSize = getInt("query/queryResultWindowSize", 1);
queryResultMaxDocsCached = getInt("query/queryResultMaxDocsCached", Integer.MAX_VALUE);
enableLazyFieldLoading = getBool("query/enableLazyFieldLoading", false);
filterCacheConfig = CacheConfig.getConfig(this, "query/filterCache");
queryResultCacheConfig = CacheConfig.getConfig(this, "query/queryResultCache");
documentCacheConfig = CacheConfig.getConfig(this, "query/documentCache");
userCacheConfigs = CacheConfig.getMultipleConfigs(this, "query/cache");
org.apache.solr.search.SolrIndexSearcher.initRegenerators(this);
hashSetInverseLoadFactor = 1.0f / getFloat("//HashDocSet/@loadFactor",0.75f);
hashDocSetMaxSize= getInt("//HashDocSet/@maxSize",-1);
pingQueryParams = readPingQueryParams(this);
Config.log.info("Loaded SolrConfig: " + file);
}
/* The set of materialized parameters: */
public final int booleanQueryMaxClauseCount;
// SolrIndexSearcher - nutch optimizer
public final boolean filtOptEnabled;
public final int filtOptCacheSize;
public final float filtOptThreshold;
// SolrIndexSearcher - caches configurations
public final CacheConfig filterCacheConfig ;
public final CacheConfig queryResultCacheConfig;
public final CacheConfig documentCacheConfig;
public final CacheConfig[] userCacheConfigs;
// SolrIndexSearcher - more...
public final boolean useFilterForSortedQuery;
public final int queryResultWindowSize;
public final int queryResultMaxDocsCached;
public final boolean enableLazyFieldLoading;
// DocSet
public final float hashSetInverseLoadFactor;
public final int hashDocSetMaxSize;
// default & main index configurations
public final SolrIndexConfig defaultIndexConfig;
public final SolrIndexConfig mainIndexConfig;
// ping query request parameters
private final NamedList pingQueryParams;
static private NamedList readPingQueryParams(SolrConfig config) {
// TODO: check for nested tags and parse as a named list instead
String urlSnippet = config.get("admin/pingQuery", "").trim();
@ -107,6 +168,14 @@ public class SolrConfig {
String[] split = tok.split("=", 2);
params.add(split[0], split[1]);
}
return new LocalSolrQueryRequest(core, params);
return params;
}
/**
* Returns a Request object based on the admin/pingQuery section
* of the Solr config file.
*/
public SolrQueryRequest getPingQueryRequest(SolrCore core) {
return new LocalSolrQueryRequest(core, pingQueryParams);
}
}

View File

@ -19,7 +19,9 @@ package org.apache.solr.core;
import java.io.File;
import java.io.IOException;
import java.util.Collections;
import java.util.ArrayList;
import java.util.LinkedHashMap;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
@ -75,30 +77,59 @@ public final class SolrCore {
public static final String version="1.0";
public static Logger log = Logger.getLogger(SolrCore.class.getName());
private final String name;
private final IndexSchema schema;
private final String dataDir;
private final String index_path;
private final UpdateHandler updateHandler;
private static final long startTime = System.currentTimeMillis();
private final RequestHandlers reqHandlers = new RequestHandlers();
private final RequestHandlers reqHandlers;
private final SolrHighlighter highlighter;
private final Map<String,UpdateRequestProcessorFactory> updateProcessors;
public long getStartTime() { return startTime; }
public static SolrIndexConfig mainIndexConfig = new SolrIndexConfig("mainIndex");
static {
BooleanQuery.setMaxClauseCount(SolrConfig.config.getInt("query/maxBooleanClauses",BooleanQuery.getMaxClauseCount()));
if (mainIndexConfig.writeLockTimeout != -1) IndexWriter.setDefaultWriteLockTimeout(mainIndexConfig.writeLockTimeout);
static int boolean_query_max_clause_count = Integer.MIN_VALUE;
// only change the BooleanQuery maxClauseCount once for ALL cores...
void booleanQueryMaxClauseCount() {
synchronized(SolrCore.class) {
if (boolean_query_max_clause_count == Integer.MIN_VALUE) {
boolean_query_max_clause_count = solrConfig.booleanQueryMaxClauseCount;
BooleanQuery.setMaxClauseCount(boolean_query_max_clause_count);
} else if (boolean_query_max_clause_count != solrConfig.booleanQueryMaxClauseCount )
log.fine("BooleanQuery.maxClauseCount= " +boolean_query_max_clause_count+ ", ignoring " +solrConfig.booleanQueryMaxClauseCount);
}
}
public SolrConfig getSolrConfig() {
return solrConfig;
}
public static List<SolrEventListener> parseListener(String path) {
public String getConfigFile() {
return solrConfig.configFile;
}
public String getSchemaFile() {
return schema.getSchemaFile();
}
/** The configuration used by this core (to load resources, find classes, etc). */
private final SolrConfig solrConfig;
private Map<String,SolrInfoMBean> infoRegistry = new java.util.HashMap<String,SolrInfoMBean>();
/**
* @since solr 1.3
*/
public Map<String,SolrInfoMBean> getInfoRegistry() {
return infoRegistry;
}
public List<SolrEventListener> parseListener(String path) {
List<SolrEventListener> lst = new ArrayList<SolrEventListener>();
log.info("Searching for listeners: " +path);
NodeList nodes = (NodeList)SolrConfig.config.evaluate(path, XPathConstants.NODESET);
NodeList nodes = (NodeList)solrConfig.evaluate(path, XPathConstants.NODESET);
if (nodes!=null) {
for (int i=0; i<nodes.getLength(); i++) {
Node node = nodes.item(i);
@ -119,13 +150,14 @@ public final class SolrCore {
newSearcherListeners = parseListener("//listener[@event=\"newSearcher\"]");
}
public String getName() { return name; }
public IndexSchema getSchema() { return schema; }
public String getDataDir() { return dataDir; }
public String getIndexDir() { return index_path; }
// gets a non-caching searcher
public SolrIndexSearcher newSearcher(String name) throws IOException {
return new SolrIndexSearcher(schema, name,getIndexDir(),false);
return new SolrIndexSearcher(this, schema, name,getIndexDir(),false);
}
@ -134,7 +166,7 @@ public final class SolrCore {
File dirFile = new File(getIndexDir());
boolean indexExists = dirFile.canRead();
boolean removeLocks = SolrConfig.config.getBool("mainIndex/unlockOnStartup", false);
boolean removeLocks = solrConfig.getBool("mainIndex/unlockOnStartup", false);
if (removeLocks) {
// to remove locks, the directory must already exist... so we create it
// if it didn't exist already...
@ -151,7 +183,7 @@ public final class SolrCore {
log.warning("Solr index directory '" + dirFile + "' doesn't exist."
+ " Creating new index...");
SolrIndexWriter writer = new SolrIndexWriter("SolrCore.initIndex",getIndexDir(), true, schema, mainIndexConfig);
SolrIndexWriter writer = new SolrIndexWriter("SolrCore.initIndex",getIndexDir(), true, schema, solrConfig.mainIndexConfig);
writer.close();
}
@ -161,52 +193,151 @@ public final class SolrCore {
}
}
private UpdateHandler createUpdateHandler(String className) {
/** Creates an instance by trying a constructor that accepts a SolrCore before
* trying the default (no arg) constructor.
*@param className the instance class to create
*@cast the class or interface that the instance should extend or implement
*@param msg a message helping compose the exception error if any occurs.
*@return the desired instance
*@throws SolrException if the object could not be instantiated
*/
public <T extends Object> T createInstance(String className, Class<T> cast, String msg) {
Class clazz = null;
if (msg == null) msg = "SolrCore Object";
try {
Class handlerClass = Config.findClass(className);
java.lang.reflect.Constructor cons = handlerClass.getConstructor(new Class[]{SolrCore.class});
return (UpdateHandler)cons.newInstance(new Object[]{this});
try {
clazz = solrConfig.findClass(className);
if (cast != null && !cast.isAssignableFrom(clazz))
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,"Error Instantiating "+msg+", "+className+ " is not a " +cast.getName());
java.lang.reflect.Constructor cons = clazz.getConstructor(new Class[]{SolrCore.class});
return (T) cons.newInstance(new Object[]{this});
} catch(NoSuchMethodException xnomethod) {
return (T) clazz.newInstance();
}
} catch (SolrException e) {
throw e;
} catch (Exception e) {
throw new SolrException( SolrException.ErrorCode.SERVER_ERROR,"Error Instantiating Update Handler "+className, e);
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,"Error Instantiating "+msg+", "+className+ " failed to instantiate " +cast.getName(), e);
}
}
public SolrEventListener createEventListener(String className) {
return createInstance(className, SolrEventListener.class, "Event Listener");
}
// Singleton for now...
private static SolrCore core;
public SolrRequestHandler createRequestHandler(String className) {
return createInstance(className, SolrRequestHandler.class, "Request Handler");
}
private UpdateHandler createUpdateHandler(String className) {
return createInstance(className, UpdateHandler.class, "Update Handler");
}
// The registry of known cores
private static Map<String, SolrCore> cores = new HashMap();
/** Alias for SolrCore.getSolrCore(null). */
public static SolrCore getSolrCore() {
synchronized (SolrCore.class) {
if (core==null) core = new SolrCore(null,null);
return getSolrCore(null);
}
/**
* Retrieves a core instance by name.
*@param name the core name
*@return the core instance or null if none exist with that name.
*/
public static SolrCore getSolrCore(String name) {
if (name != null && name.length() == 0)
name = null;
synchronized (cores) {
SolrCore core = cores.get(name);
if (core==null && name==null)
try {
core = new SolrCore(null, new SolrConfig(), null);
} catch(Exception xany) {
log.throwing("SolrCore", "getSolrCore", xany);
return null;
}
return core;
}
}
/**
* Returns an unmodifieable Map containing the registered cores
*/
public Map<String,SolrCore> getSolrCores() {
return Collections.unmodifiableMap( cores );
}
public SolrCore(String dataDir, IndexSchema schema) {
synchronized (SolrCore.class) {
// this is for backward compatibility (and also the reason
// the sync block is needed)
core = this; // set singleton
/** The array of known core names. */
public String[] getSolrCoreNames() {
synchronized(cores) {
String[] names = new String[cores.size()];
int count = 0;
java.util.Iterator<String> itnames = cores.keySet().iterator();
while(itnames.hasNext()) {
names[count++] = itnames.next();
}
return names;
}
}
if (dataDir ==null) {
dataDir = SolrConfig.config.get("dataDir",Config.getInstanceDir()+"data");
public String toString() {
return name!=null? "core{" + name + "}" : super.toString();
}
log.info("Opening new SolrCore at " + Config.getInstanceDir() + ", dataDir="+dataDir);
/** The single-core mode compatibility constructor; the core is named 'null'. */
public SolrCore(String dataDir, SolrConfig config, IndexSchema schema) {
this(null, dataDir, config, schema);
}
if (schema==null) {
schema = new IndexSchema("schema.xml");
/** Ensures that a name does not contain a '/' or a '\' to avoid any potential
* issues with file pathes.
*@param name the core name to check
*@return the name
*@throws SolrException if the name is not valid
*/
private static String checkName(String name) {
if (name != null) for(int i = 0, length = name.length(); i < length; ++i) {
char c = name.charAt(i);
if (c == '/' || c == '\\' || Character.isSpaceChar(c))
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,"Invalid core name '"+name+"'");
}
return name;
}
/**
* Creates a new core and register it in the list of cores.
* If a core with the same name already exists, it will be stopped and replaced by this one.
*@param name the unique name of the core (null is accepted)
*@param dataDir the index directory
*@param config a solr config instance
*@param schema a solr schema instance
*/
public SolrCore(String name, String dataDir, SolrConfig config, IndexSchema schema) {
this.name = checkName(name);
this.solrConfig = config;
// compatibility code with pre-solr215-patch in case some custom code relies on SolrConfig.config existence.
if (this.name == null) SolrConfig.config = config;
if (dataDir ==null)
dataDir = solrConfig.get("dataDir",solrConfig.getInstanceDir()+"data");
if (schema==null)
this.schema = new IndexSchema(config, "schema.xml");
else
this.schema = schema;
this.dataDir = dataDir;
this.index_path = dataDir + "/" + "index";
if (name == null)
this.index_path = dataDir + "/index";
else
this.index_path = dataDir + "/index-" + name;
this.maxWarmingSearchers = SolrConfig.config.getInt("query/maxWarmingSearchers",Integer.MAX_VALUE);
log.info("Opening new SolrCore at " + solrConfig.getInstanceDir() + ", dataDir="+dataDir + ", indexPath=" + index_path);
booleanQueryMaxClauseCount();
this.maxWarmingSearchers = solrConfig.getInt("query/maxWarmingSearchers",Integer.MAX_VALUE);
parseListeners();
@ -216,12 +347,12 @@ public final class SolrCore {
// Processors initialized before the handlers
updateProcessors = loadUpdateProcessors();
reqHandlers.initHandlersFromConfig( SolrConfig.config );
reqHandlers = new RequestHandlers(this);
reqHandlers.initHandlersFromConfig( solrConfig );
// TODO? could select the highlighter implementation
highlighter = new SolrHighlighter();
highlighter.initalize( SolrConfig.config );
highlighter.initalize( solrConfig );
try {
// Open the searcher *before* the handler so we don't end up opening
@ -229,12 +360,19 @@ public final class SolrCore {
getSearcher(false,false,null);
updateHandler = createUpdateHandler(
SolrConfig.config.get("updateHandler/@class", DirectUpdateHandler.class.getName())
solrConfig.get("updateHandler/@class", DirectUpdateHandler.class.getName())
);
} catch (IOException e) {
throw new RuntimeException(e);
}
// register this core
synchronized(cores) {
SolrCore previous = cores.get(name);
if (previous != null) {
previous.close();
}
cores.put(name, this);
}
}
@ -259,7 +397,7 @@ public final class SolrCore {
}
};
NodeList nodes = (NodeList)SolrConfig.config.evaluate("updateRequestProcessor/factory", XPathConstants.NODESET);
NodeList nodes = (NodeList)solrConfig.evaluate("updateRequestProcessor/factory", XPathConstants.NODESET);
UpdateRequestProcessorFactory def = loader.load( nodes );
if( def == null ) {
def = new ChainedUpdateProcessorFactory(); // the default
@ -284,7 +422,21 @@ public final class SolrCore {
}
public void close() {
log.info("CLOSING SolrCore!");
close(true);
}
private void close(boolean remove) {
if (name == null)
log.info("CLOSING default SolrCore!");
else
log.info("CLOSING SolrCore "+ name);
if (remove) synchronized(cores) {
SolrCore core = cores.remove(name);
if (core == null) {
log.info("Core " + core + " already closed");
return;
}
}
try {
closeSearcher();
} catch (Exception e) {
@ -302,10 +454,28 @@ public final class SolrCore {
}
}
/** Stops all cores. */
public static void shutdown() {
synchronized(cores) {
java.util.Iterator< java.util.Map.Entry<String,SolrCore> > it = cores.entrySet().iterator();
while(it.hasNext()) {
SolrCore core = it.next().getValue();
core.close(false);
}
cores.clear();
}
}
@Override
protected void finalize() { close(); }
/**
* Returns a Request object based on the admin/pingQuery section
* of the Solr config file.
*/
public SolrQueryRequest getPingQueryRequest() {
return solrConfig.getPingQueryRequest(this);
}
////////////////////////////////////////////////////////////////////////////////
// Request Handler
////////////////////////////////////////////////////////////////////////////////
@ -364,8 +534,7 @@ public final class SolrCore {
* RequestHandlers need access to the updateHandler so they can all talk to the
* same RAM indexer.
*/
public UpdateHandler getUpdateHandler()
{
public UpdateHandler getUpdateHandler() {
return updateHandler;
}
@ -486,7 +655,7 @@ public final class SolrCore {
// if this fails, we need to decrement onDeckSearchers again.
SolrIndexSearcher tmp;
try {
tmp = new SolrIndexSearcher(schema, "main", index_path, true);
tmp = new SolrIndexSearcher(this, schema, "main", index_path, true);
} catch (Throwable th) {
synchronized(searcherLock) {
onDeckSearchers--;
@ -515,7 +684,7 @@ public final class SolrCore {
if (_searcher == null) {
// if there isn't a current searcher then we may
// want to register this one before warming is complete instead of waiting.
if (SolrConfig.config.getBool("query/useColdSearcher",false)) {
if (solrConfig.getBool("query/useColdSearcher",false)) {
registerSearcher(newSearchHolder);
decrementOnDeckCount[0]=false;
alreadyRegistered=true;
@ -628,8 +797,7 @@ public final class SolrCore {
// callers may wait on the waitSearcher future returned.
return returnSearcher ? newSearchHolder : null;
}
catch (Exception e) {
} catch (Exception e) {
SolrException.logOnce(log,null,e);
if (currSearcherHolder != null) currSearcherHolder.decref();
@ -655,8 +823,7 @@ public final class SolrCore {
private RefCounted<SolrIndexSearcher> newHolder(SolrIndexSearcher newSearcher) {
RefCounted<SolrIndexSearcher> holder = new RefCounted<SolrIndexSearcher>(newSearcher)
{
RefCounted<SolrIndexSearcher> holder = new RefCounted<SolrIndexSearcher>(newSearcher) {
public void close() {
try {
resource.close();
@ -710,13 +877,17 @@ public final class SolrCore {
if (_searcher != null) {
_searcher.decref(); // dec refcount for this._searcher
_searcher=null;
SolrInfoRegistry.getRegistry().remove("currentSearcher");
infoRegistry.remove("currentSearcher");
}
}
}
public void execute(SolrRequestHandler handler, SolrQueryRequest req, SolrQueryResponse rsp) {
if (handler==null) {
log.warning("Null Request Handler '" + req.getQueryType() +"' :" + req);
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,"Null Request Handler '" + req.getQueryType() + "'", true);
}
// setup response header and handle request
final NamedList<Object> responseHeader = new SimpleOrderedMap<Object>();
rsp.add("responseHeader", responseHeader);
@ -760,8 +931,7 @@ public final class SolrCore {
}
if( echoParams == EchoParamStyle.EXPLICIT ) {
responseHeader.add("params", req.getOriginalParams().toNamedList());
}
else if( echoParams == EchoParamStyle.ALL ) {
} else if( echoParams == EchoParamStyle.ALL ) {
responseHeader.add("params", req.getParams().toNamedList());
}
}
@ -781,12 +951,12 @@ public final class SolrCore {
* writers may also be configured. */
private void initWriters() {
String xpath = "queryResponseWriter";
NodeList nodes = (NodeList) SolrConfig.config.evaluate(xpath, XPathConstants.NODESET);
NodeList nodes = (NodeList) solrConfig.evaluate(xpath, XPathConstants.NODESET);
NamedListPluginLoader<QueryResponseWriter> loader =
new NamedListPluginLoader<QueryResponseWriter>( "[solrconfig.xml] "+xpath, responseWriters );
defaultResponseWriter = loader.load( nodes );
defaultResponseWriter = loader.load( this, nodes );
// configure the default response writer; this one should never be null
if (defaultResponseWriter == null) {

View File

@ -18,7 +18,7 @@
package org.apache.solr.core;
import org.apache.solr.core.SolrInfoMBean;
import org.apache.solr.core.SolrCore;
import java.util.*;
/**
@ -27,10 +27,11 @@ import java.util.*;
// A Registry to hold a collection of SolrInfo objects
@Deprecated
public class SolrInfoRegistry {
private static final Map<String,SolrInfoMBean> inst = Collections.synchronizedMap(new LinkedHashMap<String,SolrInfoMBean>());
@Deprecated
public static Map<String, SolrInfoMBean> getRegistry() {
return inst;
return SolrCore.getSolrCore().getInfoRegistry();
}
}

View File

@ -357,9 +357,9 @@ public class XmlUpdateRequestHandler extends RequestHandlerBase
* @since solr 1.2
*/
@Deprecated
public void doLegacyUpdate(Reader input, Writer output) {
public void doLegacyUpdate(SolrCore core, Reader input, Writer output) {
try {
SolrCore core = SolrCore.getSolrCore();
//SolrCore core = SolrCore.getSolrCore();
// Old style requests do not choose a custom handler
UpdateRequestProcessorFactory processorFactory = core.getUpdateProcessorFactory( null );

View File

@ -23,8 +23,8 @@ import java.util.Map;
import org.apache.solr.common.params.SolrParams;
import org.apache.solr.common.util.SimpleOrderedMap;
import org.apache.solr.core.SolrCore;
import org.apache.solr.core.SolrInfoMBean;
import org.apache.solr.core.SolrInfoRegistry;
import org.apache.solr.handler.RequestHandlerBase;
import org.apache.solr.handler.RequestHandlerUtils;
import org.apache.solr.request.SolrQueryRequest;
@ -49,17 +49,17 @@ public class PluginInfoHandler extends RequestHandlerBase
SolrParams params = req.getParams();
boolean stats = params.getBool( "stats", false );
rsp.add( "plugins", getSolrInfoBeans( stats ) );
rsp.add( "plugins", getSolrInfoBeans( req.getCore(), stats ) );
}
private static SimpleOrderedMap<Object> getSolrInfoBeans( boolean stats )
private static SimpleOrderedMap<Object> getSolrInfoBeans( SolrCore core, boolean stats )
{
SimpleOrderedMap<Object> list = new SimpleOrderedMap<Object>();
for (SolrInfoMBean.Category cat : SolrInfoMBean.Category.values())
{
SimpleOrderedMap<Object> category = new SimpleOrderedMap<Object>();
list.add( cat.name(), category );
Map<String, SolrInfoMBean> reg = SolrInfoRegistry.getRegistry();
Map<String, SolrInfoMBean> reg = core.getInfoRegistry();
synchronized(reg) {
for (Map.Entry<String,SolrInfoMBean> entry : reg.entrySet()) {
SolrInfoMBean m = entry.getValue();

View File

@ -31,6 +31,7 @@ import javax.xml.transform.TransformerException;
import javax.xml.transform.stream.StreamResult;
import javax.xml.transform.stream.StreamSource;
import org.apache.solr.core.SolrConfig;
import org.apache.solr.common.params.SolrParams;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.util.xslt.TransformerProvider;
@ -106,13 +107,14 @@ public class XSLTResponseWriter implements QueryResponseWriter {
if(xslt==null) {
throw new IOException("'" + TRANSFORM_PARAM + "' request parameter is required to use the XSLTResponseWriter");
}
// not the cleanest way to achieve this
SolrConfig solrConfig = request.getSearcher().getSchema().getSolrConfig();
// no need to synchronize access to context, right?
// Nothing else happens with it at the same time
final Map<Object,Object> ctx = request.getContext();
Transformer result = (Transformer)ctx.get(CONTEXT_TRANSFORMER_KEY);
if(result==null) {
result = TransformerProvider.instance.getTransformer(xslt,xsltCacheLifetimeSeconds.intValue());
result = TransformerProvider.instance.getTransformer(solrConfig, xslt,xsltCacheLifetimeSeconds.intValue());
ctx.put(CONTEXT_TRANSFORMER_KEY,result);
}
return result;

View File

@ -25,6 +25,7 @@ import org.apache.lucene.search.Similarity;
import org.apache.lucene.queryParser.QueryParser;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.util.DOMUtil;
import org.apache.solr.core.SolrCore;
import org.apache.solr.core.SolrConfig;
import org.apache.solr.core.Config;
import org.apache.solr.analysis.TokenFilterFactory;
@ -55,7 +56,7 @@ import java.util.logging.Logger;
public final class IndexSchema {
final static Logger log = Logger.getLogger(IndexSchema.class.getName());
private final SolrConfig solrConfig;
private final String schemaFile;
private String name;
private float version;
@ -66,20 +67,28 @@ public final class IndexSchema {
*
* @see Config#openResource
*/
public IndexSchema(String schemaFile) {
public IndexSchema(SolrConfig solrConfig, String schemaFile) {
this.solrConfig = solrConfig;
this.schemaFile=schemaFile;
readConfig();
readSchema(solrConfig);
}
public SolrConfig getSolrConfig() {
return solrConfig;
}
/**
* Direct acess to the InputStream for the schemaFile used by this instance.
*
* @see Config#openResource
*/
public InputStream getInputStream() {
return Config.openResource(schemaFile);
return solrConfig.openResource(schemaFile);
}
/** Gets the name of the schema file. */
public String getSchemaFile() {
return schemaFile;
}
float getVersion() {
return version;
@ -284,7 +293,7 @@ public final class IndexSchema {
}
private void readConfig() {
private void readSchema(SolrConfig solrConfig) {
log.info("Reading Solr Schema");
try {
@ -293,9 +302,9 @@ public final class IndexSchema {
Document document = builder.parse(getInputStream());
***/
Config config = new Config("schema", getInputStream(), "/schema/");
Document document = config.getDocument();
final XPath xpath = config.getXPath();
Config schemaConf = new Config("schema", getInputStream(), "/schema/");
Document document = schemaConf.getDocument();
final XPath xpath = schemaConf.getXPath();
Node nd = (Node) xpath.evaluate("/schema/@name", document, XPathConstants.NODE);
if (nd==null) {
@ -305,13 +314,13 @@ public final class IndexSchema {
log.info("Schema name=" + name);
}
version = config.getFloat("/schema/@version", 1.0f);
version = schemaConf.getFloat("/schema/@version", 1.0f);
final IndexSchema schema = this;
AbstractPluginLoader<FieldType> loader = new AbstractPluginLoader<FieldType>( "[schema.xml] fieldType" ) {
@Override
protected FieldType create( String name, String className, Node node ) throws Exception
protected FieldType create( SolrCore core, String name, String className, Node node ) throws Exception
{
FieldType ft = (FieldType)Config.newInstance(className);
ft.setTypeName(name);
@ -448,7 +457,7 @@ public final class IndexSchema {
similarity = new DefaultSimilarity();
log.fine("using default similarity");
} else {
similarity = (Similarity)Config.newInstance(node.getNodeValue().trim());
similarity = (Similarity)solrConfig.newInstance(node.getNodeValue().trim());
log.fine("using similarity " + similarity.getClass().getName());
}
@ -583,7 +592,7 @@ public final class IndexSchema {
NamedNodeMap attrs = node.getAttributes();
String analyzerName = DOMUtil.getAttr(attrs,"class");
if (analyzerName != null) {
return (Analyzer)Config.newInstance(analyzerName);
return (Analyzer)solrConfig.newInstance(analyzerName);
}
XPath xpath = XPathFactory.newInstance().newXPath();
@ -593,7 +602,7 @@ public final class IndexSchema {
if (tokNode==null){
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,"analyzer without class or tokenizer & filter list");
}
TokenizerFactory tfac = readTokenizerFactory(tokNode);
TokenizerFactory tfac = readTokenizerFactory(solrConfig, tokNode);
/******
// oops, getChildNodes() includes text (newlines, etc) in addition
@ -607,7 +616,7 @@ public final class IndexSchema {
ArrayList<TokenFilterFactory> filters = new ArrayList<TokenFilterFactory>();
for (int i=0; i<nList.getLength(); i++) {
TokenFilterFactory filt = readTokenFilterFactory(nList.item(i));
TokenFilterFactory filt = readTokenFilterFactory(solrConfig, nList.item(i));
if (filt != null) filters.add(filt);
}
@ -615,21 +624,27 @@ public final class IndexSchema {
};
// <tokenizer class="solr.StandardFilterFactory"/>
private TokenizerFactory readTokenizerFactory(Node node) {
private TokenizerFactory readTokenizerFactory(SolrConfig solrConfig, Node node) {
// if (node.getNodeName() != "tokenizer") return null;
NamedNodeMap attrs = node.getAttributes();
String className = DOMUtil.getAttr(attrs,"class","tokenizer");
TokenizerFactory tfac = (TokenizerFactory)Config.newInstance(className);
TokenizerFactory tfac = (TokenizerFactory)solrConfig.newInstance(className);
if (tfac instanceof SolrConfig.Initializable)
((SolrConfig.Initializable)tfac).init(solrConfig, DOMUtil.toMapExcept(attrs,"class"));
else
tfac.init(DOMUtil.toMapExcept(attrs,"class"));
return tfac;
}
// <tokenizer class="solr.StandardFilterFactory"/>
private TokenFilterFactory readTokenFilterFactory(Node node) {
private TokenFilterFactory readTokenFilterFactory(SolrConfig solrConfig, Node node) {
// if (node.getNodeName() != "filter") return null;
NamedNodeMap attrs = node.getAttributes();
String className = DOMUtil.getAttr(attrs,"class","token filter");
TokenFilterFactory tfac = (TokenFilterFactory)Config.newInstance(className);
TokenFilterFactory tfac = (TokenFilterFactory)solrConfig.newInstance(className);
if (tfac instanceof SolrConfig.Initializable)
((SolrConfig.Initializable)tfac).init(solrConfig, DOMUtil.toMapExcept(attrs,"class"));
else
tfac.init(DOMUtil.toMapExcept(attrs,"class"));
return tfac;
}

View File

@ -36,7 +36,7 @@ import javax.xml.xpath.XPathConstants;
*
* @version $Id$
*/
class CacheConfig {
public class CacheConfig {
private String nodeName;
private Map args;
@ -56,8 +56,8 @@ class CacheConfig {
this.regenerator = regenerator;
}
public static CacheConfig[] getMultipleConfigs(String configPath) {
NodeList nodes = (NodeList)SolrConfig.config.evaluate(configPath, XPathConstants.NODESET);
public static CacheConfig[] getMultipleConfigs(SolrConfig solrConfig, String configPath) {
NodeList nodes = (NodeList)solrConfig.evaluate(configPath, XPathConstants.NODESET);
if (nodes==null || nodes.getLength()==0) return null;
CacheConfig[] configs = new CacheConfig[nodes.getLength()];
for (int i=0; i<nodes.getLength(); i++) {
@ -67,8 +67,8 @@ class CacheConfig {
}
public static CacheConfig getConfig(String xpath) {
Node node = (Node)SolrConfig.config.getNode(xpath, false);
public static CacheConfig getConfig(SolrConfig solrConfig, String xpath) {
Node node = (Node)solrConfig.getNode(xpath, false);
return getConfig(node);
}

View File

@ -27,8 +27,8 @@ import org.apache.solr.core.SolrConfig;
final class DocSetHitCollector extends HitCollector {
static float HASHSET_INVERSE_LOAD_FACTOR = 1.0f / SolrConfig.config.getFloat("//HashDocSet/@loadFactor",0.75f);
static int HASHDOCSET_MAXSIZE= SolrConfig.config.getInt("//HashDocSet/@maxSize",-1);
final float HASHSET_INVERSE_LOAD_FACTOR;
final int HASHDOCSET_MAXSIZE;
int pos=0;
OpenBitSet bits;
@ -37,11 +37,14 @@ final class DocSetHitCollector extends HitCollector {
// in case there aren't that many hits, we may not want a very sparse
// bit array. Optimistically collect the first few docs in an array
// in case there are only a few.
final int[] scratch = new int[HASHDOCSET_MAXSIZE];
final int[] scratch;
// todo - could pass in bitset and an operation also...
DocSetHitCollector(int maxDoc) {
DocSetHitCollector(float inverseLoadFactor, int maxSize, int maxDoc) {
this.maxDoc = maxDoc;
HASHSET_INVERSE_LOAD_FACTOR = inverseLoadFactor;
HASHDOCSET_MAXSIZE = maxSize;
scratch = new int[HASHDOCSET_MAXSIZE];
}
public void collect(int doc, float score) {

View File

@ -56,7 +56,7 @@ import java.util.logging.Logger;
public class SolrIndexSearcher extends Searcher implements SolrInfoMBean {
private static Logger log = Logger.getLogger(SolrIndexSearcher.class.getName());
private final SolrCore core;
private final IndexSchema schema;
private final String name;
@ -66,11 +66,21 @@ public class SolrIndexSearcher extends Searcher implements SolrInfoMBean {
private final IndexReader reader;
private final boolean closeReader;
private final int queryResultWindowSize;
private final int queryResultMaxDocsCached;
private final boolean useFilterForSortedQuery;
public final boolean enableLazyFieldLoading;
private final boolean cachingEnabled;
private final SolrCache filterCache;
private final SolrCache queryResultCache;
private final SolrCache documentCache;
private final LuceneQueryOptimizer optimizer;
private final float HASHSET_INVERSE_LOAD_FACTOR;
private final int HASHDOCSET_MAXSIZE;
// map of generic caches - not synchronized since it's read-only after the constructor.
private final HashMap<String, SolrCache> cacheMap;
private static final HashMap<String, SolrCache> noGenericCaches=new HashMap<String,SolrCache>(0);
@ -80,21 +90,22 @@ public class SolrIndexSearcher extends Searcher implements SolrInfoMBean {
private static final SolrCache[] noCaches = new SolrCache[0];
/** Creates a searcher searching the index in the named directory. */
public SolrIndexSearcher(IndexSchema schema, String name, String path, boolean enableCache) throws IOException {
this(schema,name,IndexReader.open(path), true, enableCache);
public SolrIndexSearcher(SolrCore core, IndexSchema schema, String name, String path, boolean enableCache) throws IOException {
this(core, schema,name,IndexReader.open(path), true, enableCache);
}
/** Creates a searcher searching the index in the provided directory. */
public SolrIndexSearcher(IndexSchema schema, String name, Directory directory, boolean enableCache) throws IOException {
this(schema,name,IndexReader.open(directory), true, enableCache);
public SolrIndexSearcher(SolrCore core, IndexSchema schema, String name, Directory directory, boolean enableCache) throws IOException {
this(core, schema,name,IndexReader.open(directory), true, enableCache);
}
/** Creates a searcher searching the provided index. */
public SolrIndexSearcher(IndexSchema schema, String name, IndexReader r, boolean enableCache) {
this(schema,name,r, false, enableCache);
public SolrIndexSearcher(SolrCore core, IndexSchema schema, String name, IndexReader r, boolean enableCache) {
this(core, schema,name,r, false, enableCache);
}
private SolrIndexSearcher(IndexSchema schema, String name, IndexReader r, boolean closeReader, boolean enableCache) {
private SolrIndexSearcher(SolrCore core, IndexSchema schema, String name, IndexReader r, boolean closeReader, boolean enableCache) {
this.core = core;
this.schema = schema;
this.name = "Searcher@" + Integer.toHexString(hashCode()) + (name!=null ? " "+name : "");
@ -105,21 +116,27 @@ public class SolrIndexSearcher extends Searcher implements SolrInfoMBean {
this.closeReader = closeReader;
searcher.setSimilarity(schema.getSimilarity());
SolrConfig solrConfig = schema.getSolrConfig();
queryResultWindowSize = solrConfig.queryResultWindowSize;
queryResultMaxDocsCached = solrConfig.queryResultMaxDocsCached;
useFilterForSortedQuery = solrConfig.useFilterForSortedQuery;
enableLazyFieldLoading = solrConfig.enableLazyFieldLoading;
cachingEnabled=enableCache;
if (cachingEnabled) {
ArrayList<SolrCache> clist = new ArrayList<SolrCache>();
filterCache= filterCacheConfig==null ? null : filterCacheConfig.newInstance();
filterCache= solrConfig.filterCacheConfig==null ? null : solrConfig.filterCacheConfig.newInstance();
if (filterCache!=null) clist.add(filterCache);
queryResultCache = queryResultCacheConfig==null ? null : queryResultCacheConfig.newInstance();
queryResultCache = solrConfig.queryResultCacheConfig==null ? null : solrConfig.queryResultCacheConfig.newInstance();
if (queryResultCache!=null) clist.add(queryResultCache);
documentCache = documentCacheConfig==null ? null : documentCacheConfig.newInstance();
documentCache = solrConfig.documentCacheConfig==null ? null : solrConfig.documentCacheConfig.newInstance();
if (documentCache!=null) clist.add(documentCache);
if (userCacheConfigs == null) {
if (solrConfig.userCacheConfigs == null) {
cacheMap = noGenericCaches;
} else {
cacheMap = new HashMap<String,SolrCache>(userCacheConfigs.length);
for (CacheConfig userCacheConfig : userCacheConfigs) {
cacheMap = new HashMap<String,SolrCache>(solrConfig.userCacheConfigs.length);
for (CacheConfig userCacheConfig : solrConfig.userCacheConfigs) {
SolrCache cache = null;
if (userCacheConfig != null) cache = userCacheConfig.newInstance();
if (cache != null) {
@ -137,9 +154,13 @@ public class SolrIndexSearcher extends Searcher implements SolrInfoMBean {
cacheMap = noGenericCaches;
cacheList= noCaches;
}
optimizer = solrConfig.filtOptEnabled ? new LuceneQueryOptimizer(solrConfig.filtOptCacheSize,solrConfig.filtOptThreshold) : null;
// for DocSets
HASHSET_INVERSE_LOAD_FACTOR = solrConfig.hashSetInverseLoadFactor;
HASHDOCSET_MAXSIZE = solrConfig.hashDocSetMaxSize;
// register self
SolrInfoRegistry.getRegistry().put(this.name, this);
core.getInfoRegistry().put(this.name, this);
}
@ -153,7 +174,7 @@ public class SolrIndexSearcher extends Searcher implements SolrInfoMBean {
public void register() {
for (SolrCache cache : cacheList) {
cache.setState(SolrCache.State.LIVE);
SolrInfoRegistry.getRegistry().put(cache.name(), cache);
core.getInfoRegistry().put(cache.name(), cache);
}
registerTime=System.currentTimeMillis();
}
@ -165,7 +186,7 @@ public class SolrIndexSearcher extends Searcher implements SolrInfoMBean {
*/
public void close() throws IOException {
// unregister first, so no management actions are tried on a closing searcher.
SolrInfoRegistry.getRegistry().remove(name);
core.getInfoRegistry().remove(name);
if (cachingEnabled) {
StringBuilder sb = new StringBuilder();
@ -193,27 +214,12 @@ public class SolrIndexSearcher extends Searcher implements SolrInfoMBean {
public IndexReader getReader() { return reader; }
/** Direct access to the IndexSchema for use with this searcher */
public IndexSchema getSchema() { return schema; }
// params for the "nutch" query optimizer
private static boolean filtOptEnabled=SolrConfig.config.getBool("query/boolTofilterOptimizer/@enabled",false);
private static int filtOptCacheSize=SolrConfig.config.getInt("query/boolTofilterOptimizer/@cacheSize",32);
private static float filtOptThreshold= SolrConfig.config.getFloat("query/boolTofilterOptimizer/@threshold",.05f);
private LuceneQueryOptimizer optimizer = filtOptEnabled ? new LuceneQueryOptimizer(filtOptCacheSize,filtOptThreshold) : null;
private static final CacheConfig filterCacheConfig = CacheConfig.getConfig("query/filterCache");
private static final CacheConfig queryResultCacheConfig = CacheConfig.getConfig("query/queryResultCache");
private static final CacheConfig documentCacheConfig = CacheConfig.getConfig("query/documentCache");
private static final CacheConfig[] userCacheConfigs = CacheConfig.getMultipleConfigs("query/cache");
//
// Set default regenerators on filter and query caches if they don't have any
//
static {
if (filterCacheConfig != null && filterCacheConfig.getRegenerator() == null) {
filterCacheConfig.setRegenerator(
public static void initRegenerators(SolrConfig solrConfig) {
if (solrConfig.filterCacheConfig != null && solrConfig.filterCacheConfig.getRegenerator() == null) {
solrConfig.filterCacheConfig.setRegenerator(
new CacheRegenerator() {
public boolean regenerateItem(SolrIndexSearcher newSearcher, SolrCache newCache, SolrCache oldCache, Object oldKey, Object oldVal) throws IOException {
newSearcher.cacheDocSet((Query)oldKey, null, false);
@ -223,8 +229,9 @@ public class SolrIndexSearcher extends Searcher implements SolrInfoMBean {
);
}
if (queryResultCacheConfig != null && queryResultCacheConfig.getRegenerator() == null) {
queryResultCacheConfig.setRegenerator(
if (solrConfig.queryResultCacheConfig != null && solrConfig.queryResultCacheConfig.getRegenerator() == null) {
final int queryResultWindowSize = solrConfig.queryResultWindowSize;
solrConfig.queryResultCacheConfig.setRegenerator(
new CacheRegenerator() {
public boolean regenerateItem(SolrIndexSearcher newSearcher, SolrCache newCache, SolrCache oldCache, Object oldKey, Object oldVal) throws IOException {
QueryResultKey key = (QueryResultKey)oldKey;
@ -253,12 +260,6 @@ public class SolrIndexSearcher extends Searcher implements SolrInfoMBean {
}
}
private static boolean useFilterForSortedQuery=SolrConfig.config.getBool("query/useFilterForSortedQuery", false);
private static int queryResultWindowSize=SolrConfig.config.getInt("query/queryResultWindowSize", 1);
private static int queryResultMaxDocsCached=SolrConfig.config.getInt("query/queryResultMaxDocsCached", Integer.MAX_VALUE);
public Hits search(Query query, Filter filter, Sort sort) throws IOException {
// todo - when Solr starts accepting filters, need to
// change this conditional check (filter!=null) and create a new filter
@ -326,9 +327,6 @@ public class SolrIndexSearcher extends Searcher implements SolrInfoMBean {
}
}
/* solrconfig lazyfields setting */
public static final boolean enableLazyFieldLoading = SolrConfig.config.getBool("query/enableLazyFieldLoading", false);
/**
* Retrieve the {@link Document} instance corresponding to the document id.
*/
@ -579,7 +577,7 @@ public class SolrIndexSearcher extends Searcher implements SolrInfoMBean {
// query must be positive
protected DocSet getDocSetNC(Query query, DocSet filter) throws IOException {
if (filter==null) {
DocSetHitCollector hc = new DocSetHitCollector(maxDoc());
DocSetHitCollector hc = new DocSetHitCollector(HASHSET_INVERSE_LOAD_FACTOR, HASHDOCSET_MAXSIZE, maxDoc());
if (query instanceof TermQuery) {
Term t = ((TermQuery)query).getTerm();
TermDocs tdocs = null;
@ -596,7 +594,7 @@ public class SolrIndexSearcher extends Searcher implements SolrInfoMBean {
} else {
// FUTURE: if the filter is sorted by docid, could use skipTo (SkipQueryFilter)
final DocSetHitCollector hc = new DocSetHitCollector(maxDoc());
final DocSetHitCollector hc = new DocSetHitCollector(HASHSET_INVERSE_LOAD_FACTOR, HASHDOCSET_MAXSIZE, maxDoc());
final DocSet filt = filter;
searcher.search(query, null, new HitCollector() {
public void collect(int doc, float score) {
@ -987,7 +985,7 @@ public class SolrIndexSearcher extends Searcher implements SolrInfoMBean {
float maxScore;
int[] ids;
float[] scores;
final DocSetHitCollector setHC = new DocSetHitCollector(maxDoc());
final DocSetHitCollector setHC = new DocSetHitCollector(HASHSET_INVERSE_LOAD_FACTOR, HASHDOCSET_MAXSIZE, maxDoc());
query = QueryUtils.makeQueryable(query);

View File

@ -40,10 +40,14 @@ import org.apache.solr.request.SolrQueryRequest;
public class OldRequestHandler implements SolrRequestHandler {
SolrCore core;
long numRequests;
long numErrors;
public OldRequestHandler(SolrCore core) {
this.core = core;
}
public void init(NamedList args) {
SolrCore.log.log(Level.INFO, "Unused request handler arguments:" + args);
}

View File

@ -45,6 +45,11 @@ import org.apache.solr.request.SolrQueryResponse;
public class TestRequestHandler implements SolrRequestHandler {
private static Logger log = Logger.getLogger(SolrIndexSearcher.class.getName());
SolrCore core;
public TestRequestHandler(SolrCore core) {
this.core = core;
}
public void init(NamedList args) {
SolrCore.log.log(Level.INFO, "Unused request handler arguments:" + args);
}

View File

@ -143,8 +143,8 @@ public class DirectUpdateHandler2 extends UpdateHandler {
// The key is the id, the value (Integer) is the number
// of docs to save (delete all except the last "n" added)
protected final Map<String,Integer> pset;
protected int maxPendingDeletes = SolrConfig.config.getInt("updateHandler/maxPendingDeletes", -1);
protected int maxPendingDeletes = -1;
// commonly used constants for the count in the pset
protected final static Integer ZERO = 0;
protected final static Integer ONE = 1;
@ -164,6 +164,7 @@ public class DirectUpdateHandler2 extends UpdateHandler {
which makes commits more efficient
*/
pset = new TreeMap<String,Integer>();
maxPendingDeletes = core.getSolrConfig().getInt("updateHandler/maxPendingDeletes", -1);
ReadWriteLock rwl = new ReentrantReadWriteLock();
iwAccess = rwl.readLock();
@ -616,8 +617,8 @@ public class DirectUpdateHandler2 extends UpdateHandler {
docsSinceCommit = 0;
pending = null;
docsUpperBound = SolrConfig.config.getInt("updateHandler/autoCommit/maxDocs", -1);
timeUpperBound = SolrConfig.config.getInt("updateHandler/autoCommit/maxTime", -1);
docsUpperBound = core.getSolrConfig().getInt("updateHandler/autoCommit/maxDocs", -1);
timeUpperBound = core.getSolrConfig().getInt("updateHandler/autoCommit/maxTime", -1);
SolrCore.log.info("AutoCommit: " + this);
}

View File

@ -29,24 +29,18 @@ import org.apache.solr.core.SolrConfig;
*/
public class SolrIndexConfig {
public static final String defaultsName ="indexDefaults";
static final SolrIndexConfig defaultDefaults = new SolrIndexConfig();
//default values
public static final boolean defUseCompoundFile=SolrConfig.config.getBool(defaultsName +"/useCompoundFile", true);
public static final int defMaxBufferedDocs=SolrConfig.config.getInt(defaultsName +"/maxBufferedDocs", -1);
public static final int defMaxMergeDocs=SolrConfig.config.getInt(defaultsName +"/maxMergeDocs", -1);
public static final int defMergeFactor=SolrConfig.config.getInt(defaultsName +"/mergeFactor", -1);
public static final int defMaxFieldLength=SolrConfig.config.getInt(defaultsName +"/maxFieldLength", -1);
public static final int defWriteLockTimeout=SolrConfig.config.getInt(defaultsName +"/writeLockTimeout", -1);
public static final int defCommitLockTimeout=SolrConfig.config.getInt(defaultsName +"/commitLockTimeout", -1);
public static final String defLockType=SolrConfig.config.get(defaultsName +"/lockType", null);
/*** These are "final" in lucene 1.9
static {
if (writeLockTimeout != -1) IndexWriter.WRITE_LOCK_TIMEOUT=writeLockTimeout;
if (commitLockTimeout != -1) IndexWriter.COMMIT_LOCK_TIMEOUT=commitLockTimeout;
private SolrIndexConfig() {
useCompoundFile = true;
maxBufferedDocs = -1;
maxMergeDocs = -1;
mergeFactor = -1;
maxFieldLength = -1;
writeLockTimeout = -1;
commitLockTimeout = -1;
lockType = null;
}
***/
public final boolean useCompoundFile;
public final int maxBufferedDocs;
@ -57,14 +51,18 @@ public class SolrIndexConfig {
public final int commitLockTimeout;
public final String lockType;
public SolrIndexConfig(String prefix) {
useCompoundFile=SolrConfig.config.getBool(prefix+"/useCompoundFile", defUseCompoundFile);
maxBufferedDocs=SolrConfig.config.getInt(prefix+"/maxBufferedDocs",defMaxBufferedDocs);
maxMergeDocs=SolrConfig.config.getInt(prefix+"/maxMergeDocs",defMaxMergeDocs);
mergeFactor=SolrConfig.config.getInt(prefix+"/mergeFactor",defMergeFactor);
maxFieldLength= SolrConfig.config.getInt(prefix+"/maxFieldLength",defMaxFieldLength);
writeLockTimeout= SolrConfig.config.getInt(prefix+"/writeLockTimeout", defWriteLockTimeout);
commitLockTimeout= SolrConfig.config.getInt(prefix+"/commitLockTimeout", defCommitLockTimeout);
lockType=SolrConfig.config.get(prefix+"/lockType", defLockType);
public SolrIndexConfig(SolrConfig solrConfig, String prefix, SolrIndexConfig def) {
if (prefix == null)
prefix = defaultsName;
if (def == null)
def = defaultDefaults;
useCompoundFile=solrConfig.getBool(prefix+"/useCompoundFile", def.useCompoundFile);
maxBufferedDocs=solrConfig.getInt(prefix+"/maxBufferedDocs",def.maxBufferedDocs);
maxMergeDocs=solrConfig.getInt(prefix+"/maxMergeDocs",def.maxMergeDocs);
mergeFactor=solrConfig.getInt(prefix+"/mergeFactor",def.mergeFactor);
maxFieldLength=solrConfig.getInt(prefix+"/maxFieldLength",def.maxFieldLength);
writeLockTimeout=solrConfig.getInt(prefix+"/writeLockTimeout", def.writeLockTimeout);
commitLockTimeout=solrConfig.getInt(prefix+"/commitLockTimeout", def.commitLockTimeout);
lockType=solrConfig.get(prefix+"/lockType", def.lockType);
}
}

View File

@ -61,13 +61,14 @@ public abstract class UpdateHandler implements SolrInfoMBean {
protected Vector<SolrEventListener> optimizeCallbacks = new Vector<SolrEventListener>();
private void parseEventListeners() {
NodeList nodes = (NodeList) SolrConfig.config.evaluate("updateHandler/listener[@event=\"postCommit\"]", XPathConstants.NODESET);
final SolrConfig solrConfig = core.getSolrConfig();
NodeList nodes = (NodeList) solrConfig.evaluate("updateHandler/listener[@event=\"postCommit\"]", XPathConstants.NODESET);
if (nodes!=null) {
for (int i=0; i<nodes.getLength(); i++) {
Node node = nodes.item(i);
try {
String className = DOMUtil.getAttr(node,"class");
SolrEventListener listener = (SolrEventListener)Config.newInstance(className);
SolrEventListener listener = core.createEventListener(className);
listener.init(DOMUtil.childNodesToNamedList(node));
// listener.init(DOMUtil.toMapExcept(node.getAttributes(),"class","synchronized"));
commitCallbacks.add(listener);
@ -77,13 +78,13 @@ public abstract class UpdateHandler implements SolrInfoMBean {
}
}
}
nodes = (NodeList)SolrConfig.config.evaluate("updateHandler/listener[@event=\"postOptimize\"]", XPathConstants.NODESET);
nodes = (NodeList) solrConfig.evaluate("updateHandler/listener[@event=\"postOptimize\"]", XPathConstants.NODESET);
if (nodes!=null) {
for (int i=0; i<nodes.getLength(); i++) {
Node node = nodes.item(i);
try {
String className = DOMUtil.getAttr(node,"class");
SolrEventListener listener = (SolrEventListener)Config.newInstance(className);
SolrEventListener listener = core.createEventListener(className);
listener.init(DOMUtil.childNodesToNamedList(node));
optimizeCallbacks.add(listener);
log.info("added SolarEventListener for postOptimize: " + listener);
@ -113,11 +114,11 @@ public abstract class UpdateHandler implements SolrInfoMBean {
idFieldType = idField!=null ? idField.getType() : null;
parseEventListeners();
SolrInfoRegistry.getRegistry().put("updateHandler", this);
core.getInfoRegistry().put("updateHandler", this);
}
protected SolrIndexWriter createMainIndexWriter(String name) throws IOException {
SolrIndexWriter writer = new SolrIndexWriter(name,core.getIndexDir(), false, schema,SolrCore.mainIndexConfig);
SolrIndexWriter writer = new SolrIndexWriter(name,core.getIndexDir(), false, schema, core.getSolrConfig().mainIndexConfig);
return writer;
}

View File

@ -18,6 +18,8 @@
package org.apache.solr.util;
import org.apache.solr.core.SolrConfig;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.util.XML;
import org.apache.solr.request.*;
@ -43,7 +45,7 @@ import java.io.*;
* @see #tearDown
*/
public abstract class AbstractSolrTestCase extends TestCase {
protected SolrConfig solrConfig;
/**
* Harness initialized by initTestHarness.
*
@ -61,6 +63,13 @@ public abstract class AbstractSolrTestCase extends TestCase {
* </p>
*/
protected TestHarness.LocalRequestFactory lrf;
/**
* Subclasses may define this method to return the name of the
* Solr core they wish to use.
*/
public String getCoreName() {
return this.getClass().getPackage().getName();
}
/**
* Subclasses must define this method to return the name of the
@ -90,14 +99,23 @@ public abstract class AbstractSolrTestCase extends TestCase {
*
*/
public void setUp() throws Exception {
String coreName = getCoreName();
if (coreName != null) {
dataDir = new File(System.getProperty("java.io.tmpdir")
+ System.getProperty("file.separator")
+ getClass().getName() + "-" + getName() + "-"
+ System.currentTimeMillis());
} else {
dataDir = new File(System.getProperty("java.io.tmpdir")
+ System.getProperty("file.separator")
+ getClass().getName() + "-" + System.currentTimeMillis());
}
dataDir.mkdirs();
h = new TestHarness(dataDir.getAbsolutePath(),
getSolrConfigFile(),
solrConfig = h.createConfig(getSolrConfigFile());
h = new TestHarness(coreName,
dataDir.getAbsolutePath(),
solrConfig,
getSchemaFile());
lrf = h.getRequestFactory
("standard",0,20,"version","2.2");

View File

@ -66,14 +66,26 @@ public class TestHarness {
private DocumentBuilder builder;
public XmlUpdateRequestHandler updater;
public static SolrConfig createConfig(String confFile) {
// set some system properties for use by tests
System.setProperty("solr.test.sys.prop1", "propone");
System.setProperty("solr.test.sys.prop2", "proptwo");
try {
return new SolrConfig(confFile);
}
catch(Exception xany) {
throw new RuntimeException(xany);
}
}
/**
* Assumes "solrconfig.xml" is the config file to use, and
* "schema.xml" is the schema path to use.
*
* @param dataDirectory path for index data, will not be cleaned up
*/
public TestHarness(String dataDirectory) {
this(dataDirectory, "schema.xml");
public TestHarness(String name, String dataDirectory) {
this(name, dataDirectory, "schema.xml");
}
/**
* Assumes "solrconfig.xml" is the config file to use.
@ -81,24 +93,42 @@ public class TestHarness {
* @param dataDirectory path for index data, will not be cleaned up
* @param schemaFile path of schema file
*/
public TestHarness(String dataDirectory, String schemaFile) {
this(dataDirectory, "solrconfig.xml", schemaFile);
public TestHarness(String name, String dataDirectory, String schemaFile) {
this(name, dataDirectory, "solrconfig.xml", schemaFile);
}
/**
* @param name the core name
* @param dataDirectory path for index data, will not be cleaned up
* @param confFile solrconfig filename
* @param configFile solrconfig filename
* @param schemaFile schema filename
*/
public TestHarness(String dataDirectory,
String confFile,
public TestHarness(String name, String dataDirectory, String configFile, String schemaFile) {
this(name, dataDirectory, createConfig(configFile), schemaFile);
}
/**
* @param name the core name
* @param dataDirectory path for index data, will not be cleaned up
* @param solrConfig solronfig instance
* @param schemaFile schema filename
*/
public TestHarness(String name,
String dataDirectory,
SolrConfig solrConfig,
String schemaFile) {
this(name, dataDirectory, solrConfig, new IndexSchema(solrConfig, schemaFile));
}
/**
* @param name the core name
* @param dataDirectory path for index data, will not be cleaned up
* @param solrConfig solrconfig instance
* @param schema schema instance
*/
public TestHarness(String name,
String dataDirectory,
SolrConfig solrConfig,
IndexSchema indexSchema) {
try {
// set some system properties for use by tests
System.setProperty("solr.test.sys.prop1", "propone");
System.setProperty("solr.test.sys.prop2", "proptwo");
SolrConfig.initConfig(confFile);
core = new SolrCore(dataDirectory, new IndexSchema(schemaFile));
core = new SolrCore(name, dataDirectory, solrConfig, indexSchema);
builder = DocumentBuilderFactory.newInstance().newDocumentBuilder();
updater = new XmlUpdateRequestHandler();
@ -108,7 +138,9 @@ public class TestHarness {
}
}
public SolrCore getCore() {
return core;
}
/**
* Processes an "update" (add, commit or optimize) and
@ -125,7 +157,7 @@ public class TestHarness {
StringReader req = new StringReader(xml);
StringWriter writer = new StringWriter(32000);
updater.doLegacyUpdate(req, writer);
updater.doLegacyUpdate(core, req, writer);
return writer.toString();
}
@ -287,10 +319,6 @@ public class TestHarness {
}
public SolrCore getCore() {
return core;
}
/**
* Shuts down and frees any resources
*/

View File

@ -24,6 +24,7 @@ import java.util.logging.Logger;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.util.DOMUtil;
import org.apache.solr.core.Config;
import org.apache.solr.core.SolrCore;
import org.apache.solr.core.SolrConfig;
import org.w3c.dom.Node;
import org.w3c.dom.NodeList;
@ -77,8 +78,10 @@ public abstract class AbstractPluginLoader<T>
* @param node - the XML node defining this plugin
*/
@SuppressWarnings("unchecked")
protected T create( String name, String className, Node node ) throws Exception
protected T create( SolrCore core, String name, String className, Node node ) throws Exception
{
if (core != null)
return (T) core.createInstance(className, null, "plug-in");
return (T) Config.newInstance( className, getDefaultPackages() );
}
@ -120,7 +123,10 @@ public abstract class AbstractPluginLoader<T>
* If a default element is defined, it will be returned from this function.
*
*/
public T load( NodeList nodes )
public T load( NodeList nodes ) {
return load(null, nodes);
}
public T load( SolrCore core, NodeList nodes )
{
List<PluginInitInfo> info = new ArrayList<PluginInitInfo>();
T defaultPlugin = null;
@ -136,7 +142,7 @@ public abstract class AbstractPluginLoader<T>
String className = DOMUtil.getAttr(node,"class", type);
String defaultStr = DOMUtil.getAttr(node,"default", null );
T plugin = create(name, className, node );
T plugin = create(core, name, className, node );
log.info("created "+name+": " + plugin.getClass().getName() );
// Either initialize now or wait till everything has been registered

View File

@ -28,7 +28,7 @@ import javax.xml.transform.TransformerConfigurationException;
import javax.xml.transform.TransformerFactory;
import javax.xml.transform.stream.StreamSource;
import org.apache.solr.core.Config;
import org.apache.solr.core.SolrConfig;
/** Singleton that creates a Transformer for the XSLTServletFilter.
* For now, only caches the last created Transformer, but
@ -64,7 +64,7 @@ public class TransformerProvider {
/** Return a new Transformer, possibly created from our cached Templates object
* @throws TransformerConfigurationException
*/
public synchronized Transformer getTransformer(String filename,int cacheLifetimeSeconds) throws IOException {
public synchronized Transformer getTransformer(SolrConfig solrConfig, String filename,int cacheLifetimeSeconds) throws IOException {
// For now, the Templates are blindly reloaded once cacheExpires is over.
// It'd be better to check the file modification time to reload only if needed.
if(lastTemplates!=null && filename.equals(lastFilename) && System.currentTimeMillis() < cacheExpires) {
@ -72,7 +72,7 @@ public class TransformerProvider {
log.fine("Using cached Templates:" + filename);
}
} else {
lastTemplates = getTemplates(filename,cacheLifetimeSeconds);
lastTemplates = getTemplates(solrConfig, filename,cacheLifetimeSeconds);
}
Transformer result = null;
@ -90,7 +90,7 @@ public class TransformerProvider {
}
/** Return a Templates object for the given filename */
private Templates getTemplates(String filename,int cacheLifetimeSeconds) throws IOException {
private Templates getTemplates(SolrConfig solrConfig, String filename,int cacheLifetimeSeconds) throws IOException {
Templates result = null;
lastFilename = null;
@ -98,7 +98,7 @@ public class TransformerProvider {
if(log.isLoggable(Level.FINE)) {
log.fine("compiling XSLT templates:" + filename);
}
final InputStream xsltStream = Config.openResource("xslt/" + filename);
final InputStream xsltStream = solrConfig.openResource("xslt/" + filename);
result = tFactory.newTemplates(new StreamSource(xsltStream));
} catch (Exception e) {
log.throwing(getClass().getName(), "newTemplates", e);

View File

@ -48,6 +48,7 @@ public class BasicFunctionalityTest extends AbstractSolrTestCase {
public String getSchemaFile() { return "schema.xml"; }
public String getSolrConfigFile() { return "solrconfig.xml"; }
public String getCoreName() { return "basic"; }
public void setUp() throws Exception {
// if you override setUp or tearDown, you better call
@ -292,7 +293,7 @@ public class BasicFunctionalityTest extends AbstractSolrTestCase {
public void testTermVectorFields() {
IndexSchema ischema = new IndexSchema(getSchemaFile());
IndexSchema ischema = new IndexSchema(solrConfig, getSchemaFile());
SchemaField f; // Solr field type
Field luf; // Lucene field
@ -1172,7 +1173,7 @@ public class BasicFunctionalityTest extends AbstractSolrTestCase {
}
public void testCompressableFieldType() {
IndexSchema ischema = new IndexSchema(getSchemaFile());
IndexSchema ischema = new IndexSchema(solrConfig, getSchemaFile());
SchemaField f; // Solr field type
Field luf; // Lucene field

View File

@ -0,0 +1,40 @@
/**
* 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.analysis;
import org.apache.solr.core.SolrConfig;
import org.apache.solr.util.TestHarness;
import junit.framework.TestCase;
/**
*
* @author hbiestro
*/
public class AnalysisTestCase extends TestCase {
protected SolrConfig solrConfig;
/** Creates a new instance of AnalysisTestCase */
public AnalysisTestCase() {
}
public String getSolrConfigFile() { return "solrconfig.xml"; }
public void setUp() throws Exception {
// if you override setUp or tearDown, you better call
// the super classes version
super.setUp();
solrConfig = TestHarness.createConfig(getSolrConfigFile());
}
}

View File

@ -31,7 +31,7 @@ import junit.framework.TestCase;
/**
* General token testing helper functions
*/
public abstract class BaseTokenTestCase extends TestCase
public abstract class BaseTokenTestCase extends AnalysisTestCase
{
public static String tsToString(TokenStream in) throws IOException {
StringBuffer out = new StringBuffer();

View File

@ -22,7 +22,7 @@ import java.util.Map;
/**
* @version $Id:$
* @version $Id$
*/
public class TestCapitalizationFilter extends BaseTokenTestCase {
@ -33,7 +33,7 @@ public class TestCapitalizationFilter extends BaseTokenTestCase {
args.put( CapitalizationFilterFactory.ONLY_FIRST_WORD, "true" );
CapitalizationFilterFactory factory = new CapitalizationFilterFactory();
factory.init( args );
factory.init( solrConfig, args );
assertEquals( "Kitten", factory.processWord( "kiTTEN", 0 ) );
factory.forceFirstLetter = true;
@ -63,7 +63,7 @@ public class TestCapitalizationFilter extends BaseTokenTestCase {
// Now try some prefixes
factory = new CapitalizationFilterFactory();
args.put( "okPrefix", "McK" ); // all words
factory.init( args );
factory.init( solrConfig, args );
out = tsToString( factory.create( new IterTokenStream( "McKinley" ) ) );
assertEquals( "McKinley", out );

View File

@ -44,7 +44,7 @@ public class TestKeepWordFilter extends BaseTokenTestCase {
// Test Stopwords
KeepWordFilterFactory factory = new KeepWordFilterFactory();
args.put( "ignoreCase", "true" );
factory.init( args );
factory.init( solrConfig, args );
factory.setWords( words );
List<Token> expect = tokens( "aaa BBB" );
@ -53,7 +53,7 @@ public class TestKeepWordFilter extends BaseTokenTestCase {
// Now force case
args.put( "ignoreCase", "false" );
factory.init( args );
factory.init( solrConfig, args );
expect = tokens( "aaa" );
real = getTokens(factory.create( new IterTokenStream(input) ));

View File

@ -27,7 +27,7 @@ import org.apache.lucene.analysis.WhitespaceTokenizer;
/**
* @version $Id:$
*/
public class TestPatternReplaceFilter extends TestCase {
public class TestPatternReplaceFilter extends AnalysisTestCase {
public void testReplaceAll() throws Exception {
String input = "aabfooaabfooabfoob ab caaaaaaaaab";

View File

@ -26,7 +26,7 @@ import junit.framework.TestCase;
import org.apache.lucene.analysis.Token;
import org.apache.lucene.analysis.TokenStream;
public class TestPatternTokenizerFactory extends TestCase
public class TestPatternTokenizerFactory extends AnalysisTestCase
{
public void testSplitting() throws Exception
{
@ -50,7 +50,7 @@ public class TestPatternTokenizerFactory extends TestCase
args.put( PatternTokenizerFactory.PATTERN, test[1] );
PatternTokenizerFactory tokenizer = new PatternTokenizerFactory();
tokenizer.init( args );
tokenizer.init(solrConfig, args );
TokenStream stream = tokenizer.create( new StringReader( test[2] ) );
String out = TestHyphenatedWordsFilter.tsToString( stream );

View File

@ -40,24 +40,24 @@ public class TestPhoneticFilter extends BaseTokenTestCase {
PhoneticFilterFactory ff = new PhoneticFilterFactory();
try {
ff.init( args );
ff.init( solrConfig, args );
fail( "missing encoder parameter" );
}
catch( Exception ex ) {}
args.put( PhoneticFilterFactory.ENCODER, "XXX" );
try {
ff.init( args );
ff.init( solrConfig, args );
fail( "unknown encoder parameter" );
}
catch( Exception ex ) {}
args.put( PhoneticFilterFactory.ENCODER, "Metaphone" );
ff.init( args );
ff.init( solrConfig, args );
assertTrue( ff.encoder instanceof Metaphone );
assertTrue( ff.inject ); // default
args.put( PhoneticFilterFactory.INJECT, "false" );
ff.init( args );
ff.init( solrConfig, args );
assertFalse( ff.inject );
}

View File

@ -23,7 +23,7 @@ import org.apache.lucene.analysis.TokenStream;
import java.util.Iterator;
import java.util.Arrays;
public class TestRemoveDuplicatesTokenFilter extends TestCase {
public class TestRemoveDuplicatesTokenFilter extends AnalysisTestCase {
public static Token tok(int pos, String t, int start, int end) {
Token tok = new Token(t,start,end);

View File

@ -28,7 +28,7 @@ public class RequestHandlersTest extends AbstractSolrTestCase {
public void testLazyLoading() {
SolrCore core = SolrCore.getSolrCore();
SolrCore core = h.getCore();
SolrRequestHandler handler = core.getRequestHandler( "lazy" );
assertFalse( handler instanceof StandardRequestHandler );
@ -65,7 +65,7 @@ public class RequestHandlersTest extends AbstractSolrTestCase {
public void testPathNormalization()
{
SolrCore core = SolrCore.getSolrCore();
SolrCore core = h.getCore();
SolrRequestHandler h1 = core.getRequestHandler("/update/csv" );
assertNotNull( h1 );

View File

@ -30,7 +30,7 @@ public class SolrCoreTest extends AbstractSolrTestCase {
public void testRequestHandlerRegistry() {
// property values defined in build.xml
SolrCore core = SolrCore.getSolrCore();
SolrCore core = h.getCore();
EmptyRequestHandler handler1 = new EmptyRequestHandler();
EmptyRequestHandler handler2 = new EmptyRequestHandler();

View File

@ -1,51 +1,52 @@
/**
* 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.core;
import org.apache.solr.util.AbstractSolrTestCase;
import org.apache.solr.util.TestHarness;
import java.io.File;
public class TestBadConfig extends AbstractSolrTestCase {
public String getSchemaFile() { return "schema.xml"; }
public String getSolrConfigFile() { return "bad_solrconfig.xml"; }
public void setUp() throws Exception {
dataDir = new File(System.getProperty("java.io.tmpdir")
+ System.getProperty("file.separator")
+ getClass().getName() + "-" + getName() + "-"
+ System.currentTimeMillis());
dataDir.mkdirs();
try {
h = new TestHarness(dataDir.getAbsolutePath(),
getSolrConfigFile(),
getSchemaFile());
fail("Exception should have been thrown");
} catch (Exception e) {
assertTrue(e.getMessage().contains("unset.sys.property"));
}
}
public void testNothing() {
// Empty test case as the real test is that the initialization of the TestHarness fails
assertTrue(true);
}
/**
* 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.core;
import org.apache.solr.util.AbstractSolrTestCase;
import org.apache.solr.util.TestHarness;
import java.io.File;
public class TestBadConfig extends AbstractSolrTestCase {
public String getSchemaFile() { return "schema.xml"; }
public String getSolrConfigFile() { return "bad_solrconfig.xml"; }
public void setUp() throws Exception {
dataDir = new File(System.getProperty("java.io.tmpdir")
+ System.getProperty("file.separator")
+ getClass().getName());
dataDir.mkdirs();
try {
solrConfig = new SolrConfig(getSolrConfigFile());
h = new TestHarness(getName() + "-" + System.currentTimeMillis(),
dataDir.getAbsolutePath(),
solrConfig,
getSchemaFile());
fail("Exception should have been thrown");
} catch (Exception e) {
assertTrue(e.getMessage().contains("unset.sys.property"));
}
}
public void testNothing() {
// Empty test case as the real test is that the initialization of the TestHarness fails
assertTrue(true);
}
}

View File

@ -1,53 +1,53 @@
/**
* 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.core;
import org.apache.solr.util.AbstractSolrTestCase;
import org.w3c.dom.Node;
import org.w3c.dom.NodeList;
import javax.xml.xpath.XPathConstants;
public class TestConfig extends AbstractSolrTestCase {
public String getSchemaFile() { return "schema.xml"; }
public String getSolrConfigFile() { return "solrconfig.xml"; }
public void testJavaProperty() {
// property values defined in build.xml
String s = SolrConfig.config.get("propTest");
assertEquals("prefix-proptwo-suffix", s);
s = SolrConfig.config.get("propTest/@attr1", "default");
assertEquals("propone-${literal}", s);
s = SolrConfig.config.get("propTest/@attr2", "default");
assertEquals("default-from-config", s);
s = SolrConfig.config.get("propTest[@attr2='default-from-config']", "default");
assertEquals("prefix-proptwo-suffix", s);
NodeList nl = (NodeList) SolrConfig.config.evaluate("propTest", XPathConstants.NODESET);
assertEquals(1, nl.getLength());
assertEquals("prefix-proptwo-suffix", nl.item(0).getTextContent());
Node node = SolrConfig.config.getNode("propTest", true);
assertEquals("prefix-proptwo-suffix", node.getTextContent());
}
/**
* 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.core;
import org.apache.solr.util.AbstractSolrTestCase;
import org.w3c.dom.Node;
import org.w3c.dom.NodeList;
import javax.xml.xpath.XPathConstants;
public class TestConfig extends AbstractSolrTestCase {
public String getSchemaFile() { return "schema.xml"; }
public String getSolrConfigFile() { return "solrconfig.xml"; }
public void testJavaProperty() {
// property values defined in build.xml
String s = solrConfig.get("propTest");
assertEquals("prefix-proptwo-suffix", s);
s = solrConfig.get("propTest/@attr1", "default");
assertEquals("propone-${literal}", s);
s = solrConfig.get("propTest/@attr2", "default");
assertEquals("default-from-config", s);
s = solrConfig.get("propTest[@attr2='default-from-config']", "default");
assertEquals("prefix-proptwo-suffix", s);
NodeList nl = (NodeList) solrConfig.evaluate("propTest", XPathConstants.NODESET);
assertEquals(1, nl.getLength());
assertEquals("prefix-proptwo-suffix", nl.item(0).getTextContent());
Node node = solrConfig.getNode("propTest", true);
assertEquals("prefix-proptwo-suffix", node.getTextContent());
}
}

View File

@ -45,8 +45,8 @@ public class MoreLikeThisHandlerTest extends AbstractSolrTestCase {
public void testInterface()
{
SolrCore core = h.getCore();
MoreLikeThisHandler mlt = new MoreLikeThisHandler();
SolrCore core = SolrCore.getSolrCore();
Map<String,String[]> params = new HashMap<String,String[]>();
MultiMapSolrParams mmparams = new MultiMapSolrParams( params );

View File

@ -41,6 +41,7 @@ public class StandardRequestHandlerTest extends AbstractSolrTestCase {
}
public void testSorting() throws Exception {
SolrCore core = h.getCore();
assertU(adoc("id", "10", "title", "test", "val_s", "aaa"));
assertU(adoc("id", "11", "title", "test", "val_s", "bbb"));
assertU(adoc("id", "12", "title", "test", "val_s", "ccc"));
@ -49,7 +50,7 @@ public class StandardRequestHandlerTest extends AbstractSolrTestCase {
Map<String,String> args = new HashMap<String, String>();
args.put( CommonParams.Q, "title:test" );
args.put( "indent", "true" );
SolrQueryRequest req = new LocalSolrQueryRequest( SolrCore.getSolrCore(), new MapSolrParams( args) );
SolrQueryRequest req = new LocalSolrQueryRequest( core, new MapSolrParams( args) );
assertQ("Make sure they got in", req

View File

@ -1,5 +1,6 @@
package org.apache.solr.handler;
import org.apache.solr.util.AbstractSolrTestCase;
import java.io.StringReader;
import java.util.Collection;
@ -12,14 +13,18 @@ import junit.framework.TestCase;
import org.apache.solr.common.SolrInputDocument;
public class XmlUpdateRequestHandlerTest extends TestCase
public class XmlUpdateRequestHandlerTest extends AbstractSolrTestCase
{
private XMLInputFactory inputFactory = BaseXMLInputFactory.newInstance();
protected XmlUpdateRequestHandler handler = new XmlUpdateRequestHandler();
protected XmlUpdateRequestHandler handler;
@Override public String getSchemaFile() { return "schema.xml"; }
@Override public String getSolrConfigFile() { return "solrconfig.xml"; }
@Override
public void setUp() throws Exception {
super.setUp();
handler = new XmlUpdateRequestHandler();
}
@Override

View File

@ -55,7 +55,7 @@ public class BadIndexSchemaTest extends AbstractSolrTestCase {
public void testSevereErrorsForDuplicateNames()
{
SolrCore core = SolrCore.getSolrCore();
SolrCore core = h.getCore();
IndexSchema schema = core.getSchema();
for( Throwable t : SolrConfig.severeErrors ) {

View File

@ -50,13 +50,14 @@ public class IndexSchemaTest extends AbstractSolrTestCase {
*/
public void testDynamicCopy()
{
SolrCore core = h.getCore();
assertU(adoc("id", "10", "title", "test", "aaa_dynamic", "aaa"));
assertU(commit());
Map<String,String> args = new HashMap<String, String>();
args.put( CommonParams.Q, "title:test" );
args.put( "indent", "true" );
SolrQueryRequest req = new LocalSolrQueryRequest( SolrCore.getSolrCore(), new MapSolrParams( args) );
SolrQueryRequest req = new LocalSolrQueryRequest( core, new MapSolrParams( args) );
assertQ("Make sure they got in", req
,"//*[@numFound='1']"
@ -66,7 +67,7 @@ public class IndexSchemaTest extends AbstractSolrTestCase {
args = new HashMap<String, String>();
args.put( CommonParams.Q, "aaa_dynamic:aaa" );
args.put( "indent", "true" );
req = new LocalSolrQueryRequest( SolrCore.getSolrCore(), new MapSolrParams( args) );
req = new LocalSolrQueryRequest( core, new MapSolrParams( args) );
assertQ("dynamic source", req
,"//*[@numFound='1']"
,"//result/doc[1]/int[@name='id'][.='10']"
@ -75,7 +76,7 @@ public class IndexSchemaTest extends AbstractSolrTestCase {
args = new HashMap<String, String>();
args.put( CommonParams.Q, "dynamic_aaa:aaa" );
args.put( "indent", "true" );
req = new LocalSolrQueryRequest( SolrCore.getSolrCore(), new MapSolrParams( args) );
req = new LocalSolrQueryRequest( core, new MapSolrParams( args) );
assertQ("dynamic destination", req
,"//*[@numFound='1']"
,"//result/doc[1]/int[@name='id'][.='10']"

View File

@ -45,7 +45,7 @@ public class NotRequiredUniqueKeyTest extends AbstractSolrTestCase {
public void testSchemaLoading()
{
SolrCore core = SolrCore.getSolrCore();
SolrCore core = h.getCore();
IndexSchema schema = core.getSchema();
SchemaField uniqueKey = schema.getUniqueKeyField();

View File

@ -43,8 +43,7 @@ public class RequiredFieldsTest extends AbstractSolrTestCase {
public void testRequiredFieldsConfig() {
SolrCore core = SolrCore.getSolrCore();
SolrCore core = h.getCore();
IndexSchema schema = core.getSchema();
SchemaField uniqueKey = schema.getUniqueKeyField();
@ -59,6 +58,7 @@ public class RequiredFieldsTest extends AbstractSolrTestCase {
}
public void testRequiredFieldsSingleAdd() {
SolrCore core = h.getCore();
// Add a single document
assertU("adding document",
adoc("id", "529", "name", "document with id, name, and subject", "field_t", "what's inside?", "subject", "info"));
@ -74,12 +74,12 @@ public class RequiredFieldsTest extends AbstractSolrTestCase {
assertU(commit());
// Add another document without a subject, which has a default in schema
String subjectDefault = SolrCore.getSolrCore().getSchema().getField("subject").getDefaultValue();
String subjectDefault = core.getSchema().getField("subject").getDefaultValue();
assertNotNull("subject has no default value", subjectDefault);
assertQ("should find one with subject="+subjectDefault, req("id:530 subject:"+subjectDefault) ,"//result[@numFound=1]" );
// Add another document without a required name, which has no default
assertNull(SolrCore.getSolrCore().getSchema().getField("name").getDefaultValue());
assertNull(core.getSchema().getField("name").getDefaultValue());
assertFailedU("adding doc without required field",
adoc("id", "531", "subject", "no name document", "field_t", "what's inside?") );
assertU(commit());

View File

@ -33,7 +33,7 @@ public class DirectSolrConnectionTest extends AbstractSolrTestCase
public void setUp() throws Exception
{
super.setUp();
direct = new DirectSolrConnection();
direct = new DirectSolrConnection(h.getCore());
}
// Check that a request gets back the echoParams call

View File

@ -48,7 +48,7 @@ public class SolrRequestParserTest extends AbstractSolrTestCase {
public void setUp() throws Exception {
super.setUp();
parser = new SolrRequestParsers( SolrCore.getSolrCore(), SolrConfig.config );
parser = new SolrRequestParsers(h.getCore() );
}
public void testStreamBody() throws Exception

View File

@ -54,8 +54,8 @@ public class AutoCommitTest extends AbstractSolrTestCase {
}
public void testMaxDocs() throws Exception {
DirectUpdateHandler2 updater = (DirectUpdateHandler2)SolrCore.getSolrCore().getUpdateHandler();
SolrCore core = h.getCore();
DirectUpdateHandler2 updater = (DirectUpdateHandler2)core.getUpdateHandler();
DirectUpdateHandler2.CommitTracker tracker = updater.tracker;
tracker.timeUpperBound = 100000;
tracker.docsUpperBound = 14;
@ -63,7 +63,6 @@ public class AutoCommitTest extends AbstractSolrTestCase {
XmlUpdateRequestHandler handler = new XmlUpdateRequestHandler();
handler.init( null );
SolrCore core = SolrCore.getSolrCore();
MapSolrParams params = new MapSolrParams( new HashMap<String, String>() );
// Add a single document
@ -122,8 +121,8 @@ public class AutoCommitTest extends AbstractSolrTestCase {
}
public void testMaxTime() throws Exception {
DirectUpdateHandler2 updater = (DirectUpdateHandler2)SolrCore.getSolrCore().getUpdateHandler();
SolrCore core = h.getCore();
DirectUpdateHandler2 updater = (DirectUpdateHandler2) core.getUpdateHandler();
DirectUpdateHandler2.CommitTracker tracker = updater.tracker;
tracker.timeUpperBound = 500;
tracker.docsUpperBound = -1;
@ -131,7 +130,6 @@ public class AutoCommitTest extends AbstractSolrTestCase {
XmlUpdateRequestHandler handler = new XmlUpdateRequestHandler();
handler.init( null );
SolrCore core = SolrCore.getSolrCore();
MapSolrParams params = new MapSolrParams( new HashMap<String, String>() );
// Add a single document
@ -165,9 +163,9 @@ public class AutoCommitTest extends AbstractSolrTestCase {
// now make the call 10 times really fast and make sure it
// only commits once
req.setContentStreams( toContentStreams(
adoc("id", "500" ), null ) );
adoc("id", "500" ), null ) );
for( int i=0;i<10; i++ ) {
handler.handleRequest( req, rsp );
handler.handleRequest( req, rsp );
}
assertQ("should not be there yet", req("id:500") ,"//result[@numFound=0]" );
assertEquals( 2, tracker.autoCommitCount );

View File

@ -35,7 +35,7 @@ public class DocumentBuilderTest extends AbstractSolrTestCase {
public void testBuildDocument() throws Exception
{
SolrCore core = SolrCore.getSolrCore();
SolrCore core = h.getCore();
// undefined field
try {

View File

@ -32,7 +32,7 @@ public class UpdateRequestProcessorFactoryTest extends AbstractSolrTestCase {
public void testConfiguration() throws Exception
{
SolrCore core = SolrCore.getSolrCore();
SolrCore core = h.getCore();
// make sure it loaded the factories
ChainedUpdateProcessorFactory chained =

View File

@ -44,6 +44,7 @@
<writeLockTimeout>1000</writeLockTimeout>
<commitLockTimeout>10000</commitLockTimeout>
<lockType>single</lockType>
</indexDefaults>
<mainIndex>

View File

@ -25,7 +25,9 @@
<%@ page import="org.apache.lucene.LucenePackage"%>
<%
SolrCore core = SolrCore.getSolrCore();
Object ocore = request.getAttribute("org.apache.solr.SolrCore");
SolrCore core = ocore instanceof SolrCore? (SolrCore) ocore : SolrCore.getSolrCore();
SolrConfig solrConfig = core.getSolrConfig();
int port = request.getServerPort();
IndexSchema schema = core.getSchema();
@ -33,7 +35,7 @@
// and has no effect on local server function. If there is no healthcheck
// configured, don't put any status on the admin pages.
String enabledStatus = null;
String enabledFile = SolrConfig.config.get("admin/healthcheck/text()",null);
String enabledFile = solrConfig.get("admin/healthcheck/text()",null);
boolean isEnabled = false;
if (enabledFile!=null) {
isEnabled = new File(enabledFile).exists();
@ -47,7 +49,7 @@
{
StringWriter tmp = new StringWriter();
XML.escapeCharData
(SolrConfig.config.get("admin/defaultQuery/text()", null), tmp);
(solrConfig.get("admin/defaultQuery/text()", null), tmp);
defaultSearch = tmp.toString();
}

View File

@ -16,15 +16,18 @@
limitations under the License.
--%>
<%@ page import="org.apache.solr.core.Config,
org.apache.solr.core.SolrCore,
org.apache.solr.core.SolrConfig,
java.io.InputStream,
java.io.InputStreamReader,
java.io.Reader,
java.util.StringTokenizer"%>
<%
Object ocore = request.getAttribute("org.apache.solr.SolrCore");
SolrCore core = ocore instanceof SolrCore? (SolrCore) ocore : SolrCore.getSolrCore();
String fname = request.getParameter("file");
String optional = request.getParameter("optional");
String gettableFiles = SolrConfig.config.get("admin/gettableFiles","");
String gettableFiles = core.getSolrConfig().get("admin/gettableFiles","");
StringTokenizer st = new StringTokenizer(gettableFiles);
InputStream is;
boolean isValid = false;
@ -40,7 +43,7 @@
}
if (isValid) {
try {
is= Config.openResource(fname);
is= core.getSolrConfig().openResource(fname);
Reader input = new InputStreamReader(is);
char[] buf = new char[4096];
while (true) {

View File

@ -34,8 +34,8 @@
<h3>Solr</h3>
</td>
<td>
[<a href="get-file.jsp?file=schema.xml">Schema</a>]
[<a href="get-file.jsp?file=solrconfig.xml">Config</a>]
[<a href="get-file.jsp?file=<%=core.getSchemaFile()%>">Schema</a>]
[<a href="get-file.jsp?file=<%=core.getConfigFile()%>">Config</a>]
[<a href="analysis.jsp?highlight=on">Analysis</a>]
<br>
[<a href="stats.jsp">Statistics</a>]

View File

@ -17,7 +17,7 @@
--%>
<%@ page import="org.apache.solr.core.SolrConfig,
org.apache.solr.core.SolrCore,
org.apache.solr.util.XML,
org.apache.solr.common.util.XML,
org.apache.solr.core.SolrException"%>
<%@ page import="org.apache.solr.request.LocalSolrQueryRequest"%>
<%@ page import="org.apache.solr.request.SolrQueryResponse"%>
@ -29,12 +29,13 @@
<solr>
<ping>
<%
SolrCore core = SolrCore.getSolrCore();
Object ocore = request.getAttribute("org.apache.solr.SolrCore");
SolrCore core = ocore instanceof SolrCore? (SolrCore) ocore : SolrCore.getSolrCore();
SolrQueryRequest req = null;
if (null == request.getQueryString()) {
req = SolrConfig.getPingQueryRequest(core);
req = core.getPingQueryRequest();
} else {
req = new LocalSolrQueryRequest(core, new ServletSolrParams(request));
}

View File

@ -21,7 +21,8 @@
<%@ page import="java.io.Reader"%>
<%@ page contentType="text/plain;charset=UTF-8" language="java" %>
<%
SolrCore core = SolrCore.getSolrCore();
Object ocore = request.getAttribute("org.apache.solr.SolrCore");
SolrCore core = ocore instanceof SolrCore? (SolrCore) ocore : SolrCore.getSolrCore();
IndexSchema schema = core.getSchema();
Reader input = new InputStreamReader(schema.getInputStream());
char[] buf = new char[4096];

View File

@ -35,7 +35,7 @@ for (SolrInfoMBean.Category cat : SolrInfoMBean.Category.values()) {
%>
<<%= cat.toString() %>>
<%
Map<String,SolrInfoMBean> reg = SolrInfoRegistry.getRegistry();
Map<String,SolrInfoMBean> reg = core.getInfoRegistry();
synchronized(reg) {
for (Map.Entry<String,SolrInfoMBean> entry : reg.entrySet()) {
String key = entry.getKey();

View File

@ -59,7 +59,7 @@ public class DirectSolrConnection
public DirectSolrConnection()
{
core = SolrCore.getSolrCore();
parser = new SolrRequestParsers( core, SolrConfig.config );
parser = new SolrRequestParsers( core );
}
/**
@ -68,7 +68,7 @@ public class DirectSolrConnection
public DirectSolrConnection( SolrCore c )
{
core = c;
parser = new SolrRequestParsers( core, SolrConfig.config );
parser = new SolrRequestParsers( core );
}
/**
@ -110,15 +110,17 @@ public class DirectSolrConnection
}
Config.setInstanceDir( instanceDir );
}
SolrConfig config = SolrConfig.createInstance("solrconfig.xml");
// If the Data directory is specified, initalize SolrCore directly
// If the Data directory is specified, initialize SolrCore directly
if( dataDir != null ) {
core = new SolrCore( dataDir, new IndexSchema(instanceDir+"/conf/schema.xml"));
IndexSchema schema = new IndexSchema(config, instanceDir+"/conf/schema.xml");
core = new SolrCore( dataDir, config, schema );
}
else {
core = SolrCore.getSolrCore();
}
parser = new SolrRequestParsers( core, SolrConfig.config );
parser = new SolrRequestParsers( core );
}

View File

@ -68,7 +68,7 @@ public class SolrDispatchFilter implements Filter
log.info("user.dir=" + System.getProperty("user.dir"));
core = SolrCore.getSolrCore();
parsers = new SolrRequestParsers( core, SolrConfig.config );
parsers = new SolrRequestParsers( core );
}
catch( Throwable t ) {
// catch this so our filter still works

View File

@ -65,15 +65,16 @@ public class SolrRequestParsers
private boolean enableRemoteStreams = false;
private StandardRequestParser standard;
public SolrRequestParsers( SolrCore core, Config config )
public SolrRequestParsers( SolrCore core )
{
this.core = core;
Config config = core.getSolrConfig();
// Read the configuration
long uploadLimitKB = SolrConfig.config.getInt(
long uploadLimitKB = config.getInt(
"requestDispatcher/requestParsers/@multipartUploadLimitInKB", 2000 ); // 2MB default
this.enableRemoteStreams = SolrConfig.config.getBool(
this.enableRemoteStreams = config.getBool(
"requestDispatcher/requestParsers/@enableRemoteStreaming", false );
MultipartRequestParser multi = new MultipartRequestParser( uploadLimitKB );

View File

@ -25,6 +25,7 @@ import javax.servlet.http.HttpServlet;
import javax.servlet.http.HttpServletRequest;
import javax.servlet.http.HttpServletResponse;
import org.apache.solr.core.SolrCore;
import org.apache.solr.handler.XmlUpdateRequestHandler;
import org.apache.solr.request.QueryResponseWriter;
import org.apache.solr.request.XMLResponseWriter;
@ -51,7 +52,8 @@ public class SolrUpdateServlet extends HttpServlet {
BufferedReader requestReader = request.getReader();
response.setContentType(QueryResponseWriter.CONTENT_TYPE_XML_UTF8);
SolrCore core = SolrCore.getSolrCore();
PrintWriter writer = response.getWriter();
legacyUpdateHandler.doLegacyUpdate(requestReader, writer);
legacyUpdateHandler.doLegacyUpdate(core,requestReader, writer);
}
}