LUCENE-7754: Inner classes should be static whenever possible.

This commit is contained in:
Adrien Grand 2017-03-28 15:15:45 +02:00
parent 68eb078af8
commit 53064e46dd
152 changed files with 253 additions and 250 deletions

View File

@ -81,7 +81,11 @@ Other
* LUCENE-7681: MemoryIndex uses new DocValues API (Alan Woodward)
======================= Lucene 6.6.0 =======================
(No Changes)
Other
* LUCENE-7754: Inner classes should be static whenever possible.
(Daniel Jelinski via Adrien Grand)
======================= Lucene 6.5.0 =======================

View File

@ -30021,7 +30021,7 @@ public final class HTMLStripCharFilter extends BaseCharFilter {
return ZZ_BUFFERSIZE;
}
private class TextSegment extends OpenStringBuilder {
private static class TextSegment extends OpenStringBuilder {
/** The position from which the next char will be read. */
int pos = 0;

View File

@ -19,13 +19,12 @@ package org.apache.lucene.analysis.charfilter;
import java.io.IOException;
import java.io.Reader;
import java.util.Arrays;
import java.util.HashMap;
import java.util.Map;
import java.util.Set;
import org.apache.lucene.analysis.util.CharArrayMap;
import org.apache.lucene.analysis.util.CharArraySet;
import org.apache.lucene.analysis.CharArrayMap;
import org.apache.lucene.analysis.CharArraySet;
import org.apache.lucene.analysis.util.OpenStringBuilder;
/**
@ -238,7 +237,7 @@ InlineElment = ( [aAbBiIqQsSuU] |
return ZZ_BUFFERSIZE;
}
private class TextSegment extends OpenStringBuilder {
private static class TextSegment extends OpenStringBuilder {
/** The position from which the next char will be read. */
int pos = 0;

View File

@ -583,7 +583,7 @@ public final class ShingleFilter extends TokenFilter {
}
}
private class InputWindowToken {
private static class InputWindowToken {
final AttributeSource attSource;
final CharTermAttribute termAtt;
final OffsetAttribute offsetAtt;

View File

@ -157,7 +157,7 @@ public class TestDictionary extends LuceneTestCase {
tempDir.close();
}
private class CloseCheckInputStream extends FilterInputStream {
private static class CloseCheckInputStream extends FilterInputStream {
private boolean closed = false;
public CloseCheckInputStream(InputStream delegate) {

View File

@ -184,7 +184,7 @@ public class TestWordDelimiterFilter extends BaseTokenStreamTestCase {
/*
* Set a large position increment gap of 10 if the token is "largegap" or "/"
*/
private final class LargePosIncTokenFilter extends TokenFilter {
private static final class LargePosIncTokenFilter extends TokenFilter {
private CharTermAttribute termAtt = addAttribute(CharTermAttribute.class);
private PositionIncrementAttribute posIncAtt = addAttribute(PositionIncrementAttribute.class);

View File

@ -171,7 +171,7 @@ public class TestWordDelimiterGraphFilter extends BaseTokenStreamTestCase {
/*
* Set a large position increment gap of 10 if the token is "largegap" or "/"
*/
private final class LargePosIncTokenFilter extends TokenFilter {
private static final class LargePosIncTokenFilter extends TokenFilter {
private CharTermAttribute termAtt = addAttribute(CharTermAttribute.class);
private PositionIncrementAttribute posIncAtt = addAttribute(PositionIncrementAttribute.class);

View File

@ -57,7 +57,7 @@ public class NumericPayloadTokenFilterTest extends BaseTokenStreamTestCase {
assertTrue(seenDogs + " does not equal: " + true, seenDogs == true);
}
private final class WordTokenFilter extends TokenFilter {
private static final class WordTokenFilter extends TokenFilter {
private final CharTermAttribute termAtt = addAttribute(CharTermAttribute.class);
private final TypeAttribute typeAtt = addAttribute(TypeAttribute.class);

View File

@ -47,7 +47,7 @@ public class TypeAsPayloadTokenFilterTest extends BaseTokenStreamTestCase {
assertTrue(count + " does not equal: " + 10, count == 10);
}
private final class WordTokenFilter extends TokenFilter {
private static final class WordTokenFilter extends TokenFilter {
private final CharTermAttribute termAtt = addAttribute(CharTermAttribute.class);
private final TypeAttribute typeAtt = addAttribute(TypeAttribute.class);

View File

@ -223,7 +223,7 @@ public class TestTeeSinkTokenFilter extends BaseTokenStreamTestCase {
}
class ModuloTokenFilter extends TokenFilter {
static class ModuloTokenFilter extends TokenFilter {
int modCount;
@ -248,7 +248,7 @@ public class TestTeeSinkTokenFilter extends BaseTokenStreamTestCase {
}
}
class ModuloSinkFilter extends FilteringTokenFilter {
static class ModuloSinkFilter extends FilteringTokenFilter {
int count = 0;
int modCount;

View File

@ -71,7 +71,7 @@ public class TestSnowball extends BaseTokenStreamTestCase {
assertEquals(new BytesRef(new byte[]{0,1,2,3}), payloadAtt.getPayload());
}
private final class TestTokenStream extends TokenStream {
private static final class TestTokenStream extends TokenStream {
private final CharTermAttribute termAtt = addAttribute(CharTermAttribute.class);
private final OffsetAttribute offsetAtt = addAttribute(OffsetAttribute.class);
private final TypeAttribute typeAtt = addAttribute(TypeAttribute.class);

View File

@ -127,7 +127,7 @@ public class TestUAX29URLEmailTokenizer extends BaseTokenStreamTestCase {
}
/** Passes through tokens with type "<URL>" and blocks all other types. */
private class URLFilter extends TokenFilter {
private static class URLFilter extends TokenFilter {
private final TypeAttribute typeAtt = addAttribute(TypeAttribute.class);
public URLFilter(TokenStream in) {
super(in);
@ -146,7 +146,7 @@ public class TestUAX29URLEmailTokenizer extends BaseTokenStreamTestCase {
}
/** Passes through tokens with type "<EMAIL>" and blocks all other types. */
private class EmailFilter extends TokenFilter {
private static class EmailFilter extends TokenFilter {
private final TypeAttribute typeAtt = addAttribute(TypeAttribute.class);
public EmailFilter(TokenStream in) {
super(in);

View File

@ -112,7 +112,7 @@ public final class ICUTransformFilter extends TokenFilter {
/**
* Wrap a {@link CharTermAttribute} with the Replaceable API.
*/
final class ReplaceableTermAttribute implements Replaceable {
static final class ReplaceableTermAttribute implements Replaceable {
private char buffer[];
private int length;
private CharTermAttribute token;

View File

@ -82,7 +82,7 @@ public class CachingNaiveBayesClassifierTest extends ClassificationTestBase<Byte
}
}
private class NGramAnalyzer extends Analyzer {
private static class NGramAnalyzer extends Analyzer {
@Override
protected TokenStreamComponents createComponents(String fieldName) {
final Tokenizer tokenizer = new KeywordTokenizer();

View File

@ -83,7 +83,7 @@ public class SimpleNaiveBayesClassifierTest extends ClassificationTestBase<Bytes
}
}
private class NGramAnalyzer extends Analyzer {
private static class NGramAnalyzer extends Analyzer {
@Override
protected TokenStreamComponents createComponents(String fieldName) {
final Tokenizer tokenizer = new KeywordTokenizer();

View File

@ -217,8 +217,8 @@ public final class BloomFilteringPostingsFormat extends PostingsFormat {
public int size() {
return delegateFieldsProducer.size();
}
class BloomFilteredTerms extends Terms {
static class BloomFilteredTerms extends Terms {
private Terms delegateTerms;
private FuzzySet filter;
@ -288,8 +288,8 @@ public final class BloomFilteringPostingsFormat extends PostingsFormat {
return delegateTerms.getMax();
}
}
final class BloomFilteredTermsEnum extends TermsEnum {
static final class BloomFilteredTermsEnum extends TermsEnum {
private Terms delegateTerms;
private TermsEnum delegateTermsEnum;
private final FuzzySet filter;

View File

@ -267,7 +267,7 @@ public final class MemoryPostingsFormat extends PostingsFormat {
private static final int VERSION_START = 1;
private static final int VERSION_CURRENT = VERSION_START;
private class MemoryFieldsConsumer extends FieldsConsumer {
private static class MemoryFieldsConsumer extends FieldsConsumer {
private final SegmentWriteState state;
private final IndexOutput out;

View File

@ -239,7 +239,7 @@ public class SimpleTextTermVectorsReader extends TermVectorsReader {
return scratchUTF16.toString();
}
private class SimpleTVFields extends Fields {
private static class SimpleTVFields extends Fields {
private final SortedMap<String,SimpleTVTerms> fields;
SimpleTVFields(SortedMap<String,SimpleTVTerms> fields) {

View File

@ -742,7 +742,7 @@ public final class CompressingTermVectorsReader extends TermVectorsReader implem
}
private class TVTerms extends Terms {
private static class TVTerms extends Terms {
private final int numTerms, flags;
private final int[] prefixLengths, suffixLengths, termFreqs, positionIndex, positions, startOffsets, lengths, payloadIndex;

View File

@ -160,7 +160,7 @@ public class ParallelLeafReader extends LeafReader {
}
// Single instance of this, per ParallelReader instance
private final class ParallelFields extends Fields {
private static final class ParallelFields extends Fields {
final Map<String,Terms> fields = new TreeMap<>();
ParallelFields() {

View File

@ -68,7 +68,7 @@ public final class ConstantScoreQuery extends Query {
* wraps a query with its own optimized top-level
* scorer (e.g. BooleanScorer) we can use that
* top-level scorer. */
protected class ConstantBulkScorer extends BulkScorer {
protected static class ConstantBulkScorer extends BulkScorer {
final BulkScorer bulkScorer;
final Weight weight;
final float theScore;

View File

@ -75,7 +75,7 @@ public class TestCodecs extends LuceneTestCase {
NUM_TEST_ITER = atLeast(20);
}
class FieldData implements Comparable<FieldData> {
static class FieldData implements Comparable<FieldData> {
final FieldInfo fieldInfo;
final TermData[] terms;
final boolean omitTF;
@ -107,7 +107,7 @@ public class TestCodecs extends LuceneTestCase {
}
}
class PositionData {
static class PositionData {
int pos;
BytesRef payload;
@ -117,7 +117,7 @@ public class TestCodecs extends LuceneTestCase {
}
}
class TermData implements Comparable<TermData> {
static class TermData implements Comparable<TermData> {
String text2;
final BytesRef text;
int[] docs;
@ -300,7 +300,7 @@ public class TestCodecs extends LuceneTestCase {
dir.close();
}
private class Verify extends Thread {
private static class Verify extends Thread {
final Fields termsDict;
final FieldData[] fields;
final SegmentInfo si;

View File

@ -290,7 +290,7 @@ public class TestFlushByRamOrCountsPolicy extends LuceneTestCase {
assertEquals(bytesUsed, flushControl.activeBytes());
}
public class IndexThread extends Thread {
public static class IndexThread extends Thread {
IndexWriter writer;
LiveIndexWriterConfig iwc;
LineFileDocs docs;

View File

@ -346,7 +346,7 @@ public class TestIndexWriterExceptions extends LuceneTestCase {
private static String CRASH_FAIL_MESSAGE = "I'm experiencing problems";
private class CrashingFilter extends TokenFilter {
private static class CrashingFilter extends TokenFilter {
String fieldName;
int count;

View File

@ -308,7 +308,7 @@ public class TestIndexWriterMerging extends LuceneTestCase {
// Just intercepts all merges & verifies that we are never
// merging a segment with >= 20 (maxMergeDocs) docs
private class MyMergeScheduler extends MergeScheduler {
private static class MyMergeScheduler extends MergeScheduler {
@Override
synchronized public void merge(IndexWriter writer, MergeTrigger trigger, boolean newMergesFound) throws IOException {

View File

@ -51,7 +51,7 @@ import org.apache.lucene.util.LuceneTestCase.Slow;
public class TestIndexWriterWithThreads extends LuceneTestCase {
// Used by test cases below
private class IndexerThread extends Thread {
private static class IndexerThread extends Thread {
boolean diskFull;
Throwable error;

View File

@ -41,7 +41,7 @@ import org.apache.lucene.util.TestUtil;
public class TestIndexableField extends LuceneTestCase {
private class MyField implements IndexableField {
private static class MyField implements IndexableField {
private final int counter;
private final IndexableFieldType fieldType = new IndexableFieldType() {

View File

@ -99,7 +99,7 @@ public class TestMaxTermFrequency extends LuceneTestCase {
/**
* Simple similarity that encodes maxTermFrequency directly as a byte
*/
class TestSimilarity extends TFIDFSimilarity {
static class TestSimilarity extends TFIDFSimilarity {
@Override
public float lengthNorm(FieldInvertState state) {

View File

@ -50,7 +50,7 @@ import org.apache.lucene.util.TestUtil;
public class TestNorms extends LuceneTestCase {
final String byteTestField = "normsTestByte";
class CustomNormEncodingSimilarity extends TFIDFSimilarity {
static class CustomNormEncodingSimilarity extends TFIDFSimilarity {
@Override
public long encodeNormValue(float f) {

View File

@ -242,7 +242,7 @@ public class TestPerSegmentDeletes extends LuceneTestCase {
return Arrays.copyOf(docs, numDocs);
}
public class RangeMergePolicy extends MergePolicy {
public static class RangeMergePolicy extends MergePolicy {
boolean doMerge = false;
int start;
int length;

View File

@ -686,7 +686,7 @@ public class TestStressIndexing2 extends LuceneTestCase {
assertFalse(fieldsEnum2.hasNext());
}
private class IndexingThread extends Thread {
private static class IndexingThread extends Thread {
IndexWriter w;
int base;
int range;

View File

@ -50,7 +50,7 @@ public class TestTermVectorsReader extends LuceneTestCase {
private FieldInfos fieldInfos = new FieldInfos(new FieldInfo[0]);
private static int TERM_FREQ = 3;
private class TestToken implements Comparable<TestToken> {
private static class TestToken implements Comparable<TestToken> {
String text;
int pos;
int startOffset;

View File

@ -157,7 +157,7 @@ public class TestTransactionRollback extends LuceneTestCase {
}
// Rolls back to previous commit point
class RollbackDeletionPolicy extends IndexDeletionPolicy {
static class RollbackDeletionPolicy extends IndexDeletionPolicy {
private int rollbackPoint;
public RollbackDeletionPolicy(int rollbackPoint) {
@ -197,7 +197,7 @@ public class TestTransactionRollback extends LuceneTestCase {
}
}
class DeleteLastCommitPolicy extends IndexDeletionPolicy {
static class DeleteLastCommitPolicy extends IndexDeletionPolicy {
@Override
public void onCommit(List<? extends IndexCommit> commits) throws IOException {}
@ -222,7 +222,7 @@ public class TestTransactionRollback extends LuceneTestCase {
}
// Keeps all commit points (used to build index)
class KeepAllDeletionPolicy extends IndexDeletionPolicy {
static class KeepAllDeletionPolicy extends IndexDeletionPolicy {
@Override
public void onCommit(List<? extends IndexCommit> commits) throws IOException {}
@Override

View File

@ -98,7 +98,7 @@ public class TestUniqueTermCount extends LuceneTestCase {
/**
* Simple similarity that encodes maxTermFrequency directly
*/
class TestSimilarity extends Similarity {
static class TestSimilarity extends Similarity {
@Override
public long computeNorm(FieldInvertState state) {

View File

@ -183,7 +183,7 @@ public class TestCustomSearcherSort extends LuceneTestCase {
if (VERBOSE) System.out.println(message);
}
public class CustomSearcher extends IndexSearcher {
public static class CustomSearcher extends IndexSearcher {
private int switcher;
public CustomSearcher(IndexReader r, int switcher) {
@ -212,7 +212,7 @@ public class TestCustomSearcherSort extends LuceneTestCase {
}
}
private class RandomGen {
private static class RandomGen {
RandomGen(Random random) {
this.random = random;
base.set(1980, 1, 1);

View File

@ -201,7 +201,7 @@ public class TestEarlyTerminatingSortingCollector extends LuceneTestCase {
}
}
private class TestTerminatedEarlySimpleCollector extends SimpleCollector {
private static class TestTerminatedEarlySimpleCollector extends SimpleCollector {
private boolean collectedSomething;
public boolean collectedSomething() {
return collectedSomething;
@ -216,7 +216,7 @@ public class TestEarlyTerminatingSortingCollector extends LuceneTestCase {
}
}
private class TestEarlyTerminatingSortingcollectorQueryTimeout implements QueryTimeout {
private static class TestEarlyTerminatingSortingcollectorQueryTimeout implements QueryTimeout {
final private boolean shouldExit;
public TestEarlyTerminatingSortingcollectorQueryTimeout(boolean shouldExit) {
this.shouldExit = shouldExit;

View File

@ -75,7 +75,7 @@ public class TestPrefixRandom extends LuceneTestCase {
}
/** a stupid prefix query that just blasts thru the terms */
private class DumbPrefixQuery extends MultiTermQuery {
private static class DumbPrefixQuery extends MultiTermQuery {
private final BytesRef prefix;
DumbPrefixQuery(Term term) {
@ -88,7 +88,7 @@ public class TestPrefixRandom extends LuceneTestCase {
return new SimplePrefixTermsEnum(terms.iterator(), prefix);
}
private class SimplePrefixTermsEnum extends FilteredTermsEnum {
private static class SimplePrefixTermsEnum extends FilteredTermsEnum {
private final BytesRef prefix;
private SimplePrefixTermsEnum(TermsEnum tenum, BytesRef prefix) {

View File

@ -102,7 +102,7 @@ public class TestRegexpRandom2 extends LuceneTestCase {
}
/** a stupid regexp query that just blasts thru the terms */
private class DumbRegexpQuery extends MultiTermQuery {
private static class DumbRegexpQuery extends MultiTermQuery {
private final Automaton automaton;
DumbRegexpQuery(Term term, int flags) {

View File

@ -90,7 +90,7 @@ public class TestSimilarityProvider extends LuceneTestCase {
assertTrue(foodocs.scoreDocs[0].score < bardocs.scoreDocs[0].score);
}
private class ExampleSimilarityProvider extends PerFieldSimilarityWrapper {
private static class ExampleSimilarityProvider extends PerFieldSimilarityWrapper {
private Similarity sim1 = new Sim1();
private Similarity sim2 = new Sim2();
@ -104,7 +104,7 @@ public class TestSimilarityProvider extends LuceneTestCase {
}
}
private class Sim1 extends TFIDFSimilarity {
private static class Sim1 extends TFIDFSimilarity {
@Override
public long encodeNormValue(float f) {
@ -142,7 +142,7 @@ public class TestSimilarityProvider extends LuceneTestCase {
}
}
private class Sim2 extends TFIDFSimilarity {
private static class Sim2 extends TFIDFSimilarity {
@Override
public long encodeNormValue(float f) {

View File

@ -153,7 +153,7 @@ public class TestTermScorer extends LuceneTestCase {
assertTrue("doc should be number 5", ts.docID() == 5);
}
private class TestHit {
private static class TestHit {
public int doc;
public float score;

View File

@ -320,7 +320,7 @@ public class TestTimeLimitingCollector extends LuceneTestCase {
}
// counting collector that can slow down at collect().
private class MyHitCollector extends SimpleCollector {
private static class MyHitCollector extends SimpleCollector {
private final BitSet bits = new BitSet();
private int slowdown = 0;
private int lastDocCollected = -1;

View File

@ -76,7 +76,7 @@ public class TestLockFactory extends LuceneTestCase {
}
}
class MockLockFactory extends LockFactory {
static class MockLockFactory extends LockFactory {
public Map<String,Lock> locksCreated = Collections.synchronizedMap(new HashMap<String,Lock>());
@ -87,7 +87,7 @@ public class TestLockFactory extends LuceneTestCase {
return lock;
}
public class MockLock extends Lock {
public static class MockLock extends Lock {
@Override
public void close() {

View File

@ -40,7 +40,7 @@ public class TestCloseableThreadLocal extends LuceneTestCase {
assertNull(ctl.get());
}
public class InitValueThreadLocal extends CloseableThreadLocal<Object> {
public static class InitValueThreadLocal extends CloseableThreadLocal<Object> {
@Override
protected Object initialValue() {
return TEST_VALUE;

View File

@ -266,7 +266,7 @@ public class TestQueryBuilder extends LuceneTestCase {
}
}
private class SimpleCJKAnalyzer extends Analyzer {
private static class SimpleCJKAnalyzer extends Analyzer {
@Override
public TokenStreamComponents createComponents(String fieldName) {
return new TokenStreamComponents(new SimpleCJKTokenizer());

View File

@ -1458,7 +1458,7 @@ public class TestFSTs extends LuceneTestCase {
}
// used by slowcompletor
class TwoLongs {
static class TwoLongs {
long a;
long b;

View File

@ -185,7 +185,7 @@ public class CollisionMap {
return memoryUsage;
}
private class EntryIterator implements Iterator<Entry> {
private static class EntryIterator implements Iterator<Entry> {
Entry next; // next entry to return
int index; // current slot
Entry[] ents;

View File

@ -720,7 +720,7 @@ public class GroupFacetCollectorTest extends AbstractGroupingTestCase {
return null;
}
private class IndexContext {
private static class IndexContext {
final int numDocs;
final DirectoryReader indexReader;
@ -744,7 +744,7 @@ public class GroupFacetCollectorTest extends AbstractGroupingTestCase {
}
}
private class GroupedFacetResult {
private static class GroupedFacetResult {
final int totalCount;
final int totalMissingCount;

View File

@ -202,7 +202,7 @@ public class TestUnifiedHighlighterExtensibility extends LuceneTestCase {
}
/** Tests maintaining extensibility/visibility of {@link org.apache.lucene.search.uhighlight.FieldHighlighter} out of package. */
private class CustomFieldHighlighter extends FieldHighlighter {
private static class CustomFieldHighlighter extends FieldHighlighter {
CustomFieldHighlighter(String field, FieldOffsetStrategy fieldOffsetStrategy, BreakIterator breakIterator, PassageScorer passageScorer, int maxPassages, int maxNoHighlightPassages, PassageFormatter passageFormatter) {
super(field, fieldOffsetStrategy, breakIterator, passageScorer, maxPassages, maxNoHighlightPassages, passageFormatter);
}

View File

@ -238,7 +238,7 @@ public class CustomScoreQuery extends Query implements Cloneable {
/**
* A scorer that applies a (callback) function on scores of the subQuery.
*/
private class CustomScorer extends FilterScorer {
private static class CustomScorer extends FilterScorer {
private final float qWeight;
private final Scorer subQueryScorer;
private final Scorer[] valSrcScorers;

View File

@ -153,7 +153,7 @@ public class TestCustomScoreQuery extends FunctionTestSetup {
}
}
private final class CustomExternalQuery extends CustomScoreQuery {
private static final class CustomExternalQuery extends CustomScoreQuery {
@Override
protected CustomScoreProvider getCustomScoreProvider(LeafReaderContext context) throws IOException {

View File

@ -461,7 +461,7 @@ public class TestPayloadSpans extends LuceneTestCase {
assertEquals("expected numSpans", numSpans, cnt);
}
final class PayloadAnalyzer extends Analyzer {
static final class PayloadAnalyzer extends Analyzer {
@Override
public TokenStreamComponents createComponents(String fieldName) {
@ -470,7 +470,7 @@ public class TestPayloadSpans extends LuceneTestCase {
}
}
final class PayloadFilter extends TokenFilter {
static final class PayloadFilter extends TokenFilter {
Set<String> entities = new HashSet<>();
Set<String> nopayload = new HashSet<>();
int pos;
@ -515,7 +515,7 @@ public class TestPayloadSpans extends LuceneTestCase {
}
}
public final class TestPayloadAnalyzer extends Analyzer {
public static final class TestPayloadAnalyzer extends Analyzer {
@Override
public TokenStreamComponents createComponents(String fieldName) {

View File

@ -120,7 +120,7 @@ public class TestMultiAnalyzer extends BaseTokenStreamTestCase {
* Expands "multi" to "multi" and "multi2", both at the same position,
* and expands "triplemulti" to "triplemulti", "multi3", and "multi2".
*/
private class MultiAnalyzer extends Analyzer {
private static class MultiAnalyzer extends Analyzer {
@Override
public TokenStreamComponents createComponents(String fieldName) {
@ -129,7 +129,7 @@ public class TestMultiAnalyzer extends BaseTokenStreamTestCase {
}
}
private final class TestFilter extends TokenFilter {
private static final class TestFilter extends TokenFilter {
private String prevType;
private int prevStartOffset;
@ -191,7 +191,7 @@ public class TestMultiAnalyzer extends BaseTokenStreamTestCase {
* Analyzes "the quick brown" as: quick(incr=2) brown(incr=1).
* Does not work correctly for input other than "the quick brown ...".
*/
private class PosIncrementAnalyzer extends Analyzer {
private static class PosIncrementAnalyzer extends Analyzer {
@Override
public TokenStreamComponents createComponents(String fieldName) {
@ -200,7 +200,7 @@ public class TestMultiAnalyzer extends BaseTokenStreamTestCase {
}
}
private final class TestPosIncrementFilter extends TokenFilter {
private static final class TestPosIncrementFilter extends TokenFilter {
CharTermAttribute termAtt;
PositionIncrementAttribute posIncrAtt;

View File

@ -336,7 +336,7 @@ public class TestMultiFieldQueryParser extends LuceneTestCase {
}
/** whitespace+lowercase analyzer with synonyms (dogs,dog) and (guinea pig,cavy) */
private class MockSynonymAnalyzer extends Analyzer {
private static class MockSynonymAnalyzer extends Analyzer {
@Override
public TokenStreamComponents createComponents(String fieldName) {
Tokenizer tokenizer = new MockTokenizer(MockTokenizer.WHITESPACE, true);

View File

@ -139,7 +139,7 @@ public class TestMultiAnalyzerQPHelper extends LuceneTestCase {
* Expands "multi" to "multi" and "multi2", both at the same position, and
* expands "triplemulti" to "triplemulti", "multi3", and "multi2".
*/
private class MultiAnalyzer extends Analyzer {
private static class MultiAnalyzer extends Analyzer {
@Override
public TokenStreamComponents createComponents(String fieldName) {
@ -148,7 +148,7 @@ public class TestMultiAnalyzerQPHelper extends LuceneTestCase {
}
}
private final class TestFilter extends TokenFilter {
private static final class TestFilter extends TokenFilter {
private String prevType;
private int prevStartOffset;
@ -206,7 +206,7 @@ public class TestMultiAnalyzerQPHelper extends LuceneTestCase {
* Analyzes "the quick brown" as: quick(incr=2) brown(incr=1). Does not work
* correctly for input other than "the quick brown ...".
*/
private class PosIncrementAnalyzer extends Analyzer {
private static class PosIncrementAnalyzer extends Analyzer {
@Override
public TokenStreamComponents createComponents(String fieldName) {
@ -215,7 +215,7 @@ public class TestMultiAnalyzerQPHelper extends LuceneTestCase {
}
}
private class TestPosIncrementFilter extends TokenFilter {
private static class TestPosIncrementFilter extends TokenFilter {
private final CharTermAttribute termAtt = addAttribute(CharTermAttribute.class);
private final PositionIncrementAttribute posIncrAtt = addAttribute(PositionIncrementAttribute.class);

View File

@ -350,7 +350,7 @@ public class TestQPHelper extends LuceneTestCase {
}
//individual CJK chars as terms, like StandardAnalyzer
private class SimpleCJKTokenizer extends Tokenizer {
private static class SimpleCJKTokenizer extends Tokenizer {
private CharTermAttribute termAtt = addAttribute(CharTermAttribute.class);
public SimpleCJKTokenizer() {
@ -368,7 +368,7 @@ public class TestQPHelper extends LuceneTestCase {
}
}
private class SimpleCJKAnalyzer extends Analyzer {
private static class SimpleCJKAnalyzer extends Analyzer {
@Override
public TokenStreamComponents createComponents(String fieldName) {
return new TokenStreamComponents(new SimpleCJKTokenizer());
@ -1252,7 +1252,7 @@ public class TestQPHelper extends LuceneTestCase {
super.tearDown();
}
private class CannedTokenizer extends Tokenizer {
private static class CannedTokenizer extends Tokenizer {
private int upto = 0;
private final PositionIncrementAttribute posIncr = addAttribute(PositionIncrementAttribute.class);
private final CharTermAttribute term = addAttribute(CharTermAttribute.class);
@ -1291,7 +1291,7 @@ public class TestQPHelper extends LuceneTestCase {
}
}
private class CannedAnalyzer extends Analyzer {
private static class CannedAnalyzer extends Analyzer {
@Override
public TokenStreamComponents createComponents(String ignored) {
return new TokenStreamComponents(new CannedTokenizer());

View File

@ -264,7 +264,7 @@ public abstract class QueryParserTestBase extends LuceneTestCase {
}
}
private class SimpleCJKAnalyzer extends Analyzer {
private static class SimpleCJKAnalyzer extends Analyzer {
@Override
public TokenStreamComponents createComponents(String fieldName) {
return new TokenStreamComponents(new SimpleCJKTokenizer());
@ -1095,7 +1095,7 @@ public abstract class QueryParserTestBase extends LuceneTestCase {
}
/** whitespace+lowercase analyzer with synonyms */
protected class Analyzer1 extends Analyzer {
protected static class Analyzer1 extends Analyzer {
public Analyzer1(){
super();
}
@ -1107,7 +1107,7 @@ public abstract class QueryParserTestBase extends LuceneTestCase {
}
/** whitespace+lowercase analyzer without synonyms */
protected class Analyzer2 extends Analyzer {
protected static class Analyzer2 extends Analyzer {
public Analyzer2(){
super();
}
@ -1122,7 +1122,7 @@ public abstract class QueryParserTestBase extends LuceneTestCase {
/**
* Mock collation analyzer: indexes terms as "collated" + term
*/
private class MockCollationFilter extends TokenFilter {
private static class MockCollationFilter extends TokenFilter {
private final CharTermAttribute termAtt = addAttribute(CharTermAttribute.class);
protected MockCollationFilter(TokenStream input) {
@ -1141,7 +1141,7 @@ public abstract class QueryParserTestBase extends LuceneTestCase {
}
}
private class MockCollationAnalyzer extends Analyzer {
private static class MockCollationAnalyzer extends Analyzer {
@Override
public TokenStreamComponents createComponents(String fieldName) {
Tokenizer tokenizer = new MockTokenizer(MockTokenizer.WHITESPACE, true);

View File

@ -112,7 +112,7 @@ public class FuzzyLikeThisQuery extends Query
this.maxNumTerms = maxNumTerms;
}
class FieldVals
static class FieldVals
{
String queryString;
String fieldName;

View File

@ -72,7 +72,7 @@ public class TestPayloadSpanUtil extends LuceneTestCase {
directory.close();
}
final class PayloadAnalyzer extends Analyzer {
final static class PayloadAnalyzer extends Analyzer {
@Override
public TokenStreamComponents createComponents(String fieldName) {
@ -81,7 +81,7 @@ public class TestPayloadSpanUtil extends LuceneTestCase {
}
}
final class PayloadFilter extends TokenFilter {
static final class PayloadFilter extends TokenFilter {
Set<String> entities = new HashSet<>();
Set<String> nopayload = new HashSet<>();
int pos;

View File

@ -441,7 +441,7 @@ public class WordBreakSpellChecker {
this.maxEvaluations = maxEvaluations;
}
private class LengthThenMaxFreqComparator implements
private static class LengthThenMaxFreqComparator implements
Comparator<SuggestWordArrayWrapper> {
@Override
public int compare(SuggestWordArrayWrapper o1, SuggestWordArrayWrapper o2) {
@ -455,7 +455,7 @@ public class WordBreakSpellChecker {
}
}
private class LengthThenSumFreqComparator implements
private static class LengthThenSumFreqComparator implements
Comparator<SuggestWordArrayWrapper> {
@Override
public int compare(SuggestWordArrayWrapper o1, SuggestWordArrayWrapper o2) {
@ -469,7 +469,7 @@ public class WordBreakSpellChecker {
}
}
private class CombinationsThenFreqComparator implements
private static class CombinationsThenFreqComparator implements
Comparator<CombineSuggestionWrapper> {
@Override
public int compare(CombineSuggestionWrapper o1, CombineSuggestionWrapper o2) {
@ -484,7 +484,7 @@ public class WordBreakSpellChecker {
}
}
private class SuggestWordArrayWrapper {
private static class SuggestWordArrayWrapper {
final SuggestWord[] suggestWords;
final int freqMax;
final int freqSum;
@ -502,7 +502,7 @@ public class WordBreakSpellChecker {
}
}
private class CombineSuggestionWrapper {
private static class CombineSuggestionWrapper {
final CombineSuggestion combineSuggestion;
final int numCombinations;

View File

@ -238,7 +238,7 @@ public class ContextQuery extends CompletionQuery {
}
}
private class ContextCompletionWeight extends CompletionWeight {
private static class ContextCompletionWeight extends CompletionWeight {
private final Map<IntsRef, Float> contextMap;
private final int[] contextLengths;

View File

@ -105,7 +105,7 @@ public class ExternalRefSorter implements BytesRefSorter, Closeable {
* Iterate over byte refs in a file.
*/
// TODO: this class is a bit silly ... sole purpose is to "remove" Closeable from what #iterator returns:
class ByteSequenceIterator implements BytesRefIterator {
static class ByteSequenceIterator implements BytesRefIterator {
private final OfflineSorter.ByteSequencesReader reader;
private BytesRef scratch;

View File

@ -267,7 +267,7 @@ public class WFSTCompletionLookup extends Lookup implements Accountable {
return Integer.MAX_VALUE - (int)value;
}
private final class WFSTInputIterator extends SortedInputIterator {
private static final class WFSTInputIterator extends SortedInputIterator {
WFSTInputIterator(Directory tempDir, String tempFileNamePrefix, InputIterator source) throws IOException {
super(tempDir, tempFileNamePrefix, source);

View File

@ -147,17 +147,17 @@ public class JaspellLookup extends Lookup implements Accountable {
node.data = Long.valueOf(in.readLong());
}
if ((mask & LO_KID) != 0) {
TSTNode kid = trie.new TSTNode('\0', node);
TSTNode kid = new TSTNode('\0', node);
node.relatives[TSTNode.LOKID] = kid;
readRecursively(in, kid);
}
if ((mask & EQ_KID) != 0) {
TSTNode kid = trie.new TSTNode('\0', node);
TSTNode kid = new TSTNode('\0', node);
node.relatives[TSTNode.EQKID] = kid;
readRecursively(in, kid);
}
if ((mask & HI_KID) != 0) {
TSTNode kid = trie.new TSTNode('\0', node);
TSTNode kid = new TSTNode('\0', node);
node.relatives[TSTNode.HIKID] = kid;
readRecursively(in, kid);
}
@ -196,7 +196,7 @@ public class JaspellLookup extends Lookup implements Accountable {
@Override
public boolean load(DataInput input) throws IOException {
count = input.readVLong();
TSTNode root = trie.new TSTNode('\0', null);
TSTNode root = new TSTNode('\0', null);
readRecursively(input, root);
trie.setRoot(root);
return true;

View File

@ -71,7 +71,7 @@ public class JaspellTernarySearchTrie implements Accountable {
/**
* An inner class of Ternary Search Trie that represents a node in the trie.
*/
protected final class TSTNode implements Accountable {
protected static final class TSTNode implements Accountable {
/** Index values for accessing relatives array. */
protected final static int PARENT = 0, LOKID = 1, EQKID = 2, HIKID = 3;

View File

@ -411,7 +411,7 @@ public class DocumentDictionaryTest extends LuceneTestCase {
return suggestionList;
}
private class Suggestion {
private static class Suggestion {
private long weight;
private BytesRef payload;
private Set<BytesRef> contexts;

View File

@ -910,7 +910,7 @@ public class AnalyzingInfixSuggesterTest extends LuceneTestCase {
a.close();
}
private class IndexDocument implements Runnable {
private static class IndexDocument implements Runnable {
AnalyzingInfixSuggester suggester;
String key;

View File

@ -1118,7 +1118,7 @@ public class FuzzySuggesterTest extends LuceneTestCase {
private static final Comparator<CharSequence> CHARSEQUENCE_COMPARATOR = new CharSequenceComparator();
public class CompareByCostThenAlpha implements Comparator<LookupResult> {
public static class CompareByCostThenAlpha implements Comparator<LookupResult> {
@Override
public int compare(LookupResult a, LookupResult b) {
if (a.value > b.value) {

View File

@ -274,7 +274,7 @@ public abstract class BaseLockFactoryTestCase extends LuceneTestCase {
}
}
private class SearcherThread extends Thread {
private static class SearcherThread extends Thread {
private Directory dir;
private int numIteration;
public boolean hitException = false;

View File

@ -786,7 +786,7 @@ public class GetMavenDependenciesTask extends Task {
/**
* Stores information about an external dependency
*/
private class ExternalDependency implements Comparable<ExternalDependency> {
private static class ExternalDependency implements Comparable<ExternalDependency> {
String groupId;
String artifactId;
boolean isTestDependency;

View File

@ -150,7 +150,7 @@ public class LibVersionsCheckTask extends Task {
*/
private Map<String,HashSet<String>> ignoreConflictVersions = new HashMap<>();
private class Dependency {
private static class Dependency {
String org;
String name;
String directVersion;

View File

@ -728,8 +728,8 @@ public class MailEntityProcessor extends EntityProcessorBase {
return true;
}
}
class MailsSinceLastCheckFilter implements CustomFilter {
static class MailsSinceLastCheckFilter implements CustomFilter {
private Date since;

View File

@ -167,7 +167,7 @@ public class VariableResolver {
return TemplateUpdateProcessorFactory.getVariables(expr, cache);
}
class CurrentLevel {
static class CurrentLevel {
final Map<String,Object> map;
final int level;
CurrentLevel(int level, Map<String,Object> map) {

View File

@ -73,7 +73,7 @@ public class AbstractDIHCacheTestCase {
//A limitation of this test class is that the primary key needs to be the first one in the list.
//DIHCaches, however, can handle any field being the primary key.
class ControlData implements Comparable<ControlData>, Iterable<Object> {
static class ControlData implements Comparable<ControlData>, Iterable<Object> {
Object[] data;
ControlData(Object[] data) {

View File

@ -568,8 +568,8 @@ public abstract class AbstractSqlEntityProcessorTestCase extends
}
return changeSet.toArray(new String[changeSet.size()]);
}
class IntChanges {
static class IntChanges {
public Integer[] changedKeys;
public Integer[] deletedKeys;
public Integer[] addedKeys;

View File

@ -109,7 +109,7 @@ public class TestContentStreamDataSource extends AbstractDataImportHandlerTestCa
fail("Commit should have occured but it did not");
}
private class SolrInstance {
private static class SolrInstance {
String name;
Integer port;
File homeDir;

View File

@ -98,7 +98,7 @@ class XLSXWriter extends TextResponseWriter {
SolrQueryRequest req;
SolrQueryResponse rsp;
class SerialWriteWorkbook {
static class SerialWriteWorkbook {
SXSSFWorkbook swb;
Sheet sh;

View File

@ -300,7 +300,7 @@ public class LTRScoringQuery extends Query {
return field;
}
public class FeatureInfo {
public static class FeatureInfo {
final private String name;
private float value;
private boolean used;

View File

@ -80,7 +80,7 @@ public class TestLTRReRankingPipeline extends LuceneTestCase {
return features;
}
private class MockModel extends LTRScoringModel {
private static class MockModel extends LTRScoringModel {
public MockModel(String name, List<Feature> features,
List<Normalizer> norms,

View File

@ -376,7 +376,7 @@ public class VelocityResponseWriter implements QueryResponseWriter, SolrCoreAwar
}
// see: http://svn.apache.org/repos/asf/velocity/tools/branches/2.0.x/src/main/java/org/apache/velocity/tools/generic/ResourceTool.java
private class SolrVelocityResourceTool extends ResourceTool {
private static class SolrVelocityResourceTool extends ResourceTool {
private ClassLoader solrClassLoader;

View File

@ -431,7 +431,7 @@ public class Overseer implements Closeable {
}
class OverseerThread extends Thread implements Closeable {
static class OverseerThread extends Thread implements Closeable {
protected volatile boolean isClosed;
private Closeable thread;

View File

@ -108,7 +108,7 @@ public class OverseerTaskQueue extends DistributedQueue {
/**
* Watcher that blocks until a WatchedEvent occurs for a znode.
*/
private final class LatchWatcher implements Watcher {
private static final class LatchWatcher implements Watcher {
private final Object lock;
private WatchedEvent event;

View File

@ -239,7 +239,7 @@ public class ZkController {
}
// notifies registered listeners after the ZK reconnect in the background
private class OnReconnectNotifyAsync implements Callable {
private static class OnReconnectNotifyAsync implements Callable {
private final OnReconnect listener;

View File

@ -48,7 +48,7 @@ import org.slf4j.LoggerFactory;
*
*/
public abstract class CachingDirectoryFactory extends DirectoryFactory {
protected class CacheValue {
protected static class CacheValue {
final public String path;
final public Directory directory;
// for debug

View File

@ -207,7 +207,7 @@ public class CdcrReplicator implements Runnable {
/**
* Exception to catch update request issues with the target cluster.
*/
public class CdcrReplicatorException extends Exception {
public static class CdcrReplicatorException extends Exception {
private final UpdateRequest req;
private final UpdateResponse rsp;

View File

@ -197,7 +197,7 @@ class CdcrReplicatorState {
}
class BenchmarkTimer {
static class BenchmarkTimer {
private long startTime;
private long runTime = 0;
@ -266,7 +266,7 @@ class CdcrReplicatorState {
}
private class ErrorQueueEntry {
private static class ErrorQueueEntry {
private ErrorType type;
private Date timestamp;
@ -277,7 +277,7 @@ class CdcrReplicatorState {
}
}
private class FixedQueue<E> extends LinkedList<E> {
private static class FixedQueue<E> extends LinkedList<E> {
private int maxSize;

View File

@ -939,7 +939,7 @@ public class ExportWriter implements SolrCore.RawWriter, Closeable {
public int resetValue();
}
class IntDesc implements IntComp {
static class IntDesc implements IntComp {
public int resetValue() {
return Integer.MIN_VALUE;
@ -956,7 +956,7 @@ public class ExportWriter implements SolrCore.RawWriter, Closeable {
}
}
class IntAsc implements IntComp {
static class IntAsc implements IntComp {
public int resetValue() {
return Integer.MAX_VALUE;
@ -1032,7 +1032,7 @@ public class ExportWriter implements SolrCore.RawWriter, Closeable {
public long resetValue();
}
class LongDesc implements LongComp {
static class LongDesc implements LongComp {
public long resetValue() {
return Long.MIN_VALUE;
@ -1049,7 +1049,7 @@ public class ExportWriter implements SolrCore.RawWriter, Closeable {
}
}
class LongAsc implements LongComp {
static class LongAsc implements LongComp {
public long resetValue() {
return Long.MAX_VALUE;
@ -1125,7 +1125,7 @@ public class ExportWriter implements SolrCore.RawWriter, Closeable {
public float resetValue();
}
public class FloatDesc implements FloatComp {
public static class FloatDesc implements FloatComp {
public float resetValue() {
return -Float.MAX_VALUE;
}
@ -1141,7 +1141,7 @@ public class ExportWriter implements SolrCore.RawWriter, Closeable {
}
}
public class FloatAsc implements FloatComp {
public static class FloatAsc implements FloatComp {
public float resetValue() {
return Float.MAX_VALUE;
}
@ -1219,7 +1219,7 @@ public class ExportWriter implements SolrCore.RawWriter, Closeable {
public double resetValue();
}
public class DoubleDesc implements DoubleComp {
public static class DoubleDesc implements DoubleComp {
public double resetValue() {
return -Double.MAX_VALUE;
}
@ -1235,7 +1235,7 @@ public class ExportWriter implements SolrCore.RawWriter, Closeable {
}
}
public class DoubleAsc implements DoubleComp {
public static class DoubleAsc implements DoubleComp {
public double resetValue() {
return Double.MAX_VALUE;
}
@ -1712,7 +1712,7 @@ public class ExportWriter implements SolrCore.RawWriter, Closeable {
}
}
public class IgnoreException extends IOException {
public static class IgnoreException extends IOException {
public void printStackTrace(PrintWriter pw) {
pw.print("Early Client Disconnect");
}

View File

@ -1690,7 +1690,7 @@ public class IndexFetcher {
}
}
private class DirectoryFile implements FileInterface {
private static class DirectoryFile implements FileInterface {
private final String saveAs;
private Directory copy2Dir;
private IndexOutput outStream;
@ -1725,7 +1725,7 @@ public class IndexFetcher {
}
}
private class LocalFsFile implements FileInterface {
private static class LocalFsFile implements FileInterface {
private File copy2Dir;
FileChannel fileChannel;

View File

@ -141,7 +141,7 @@ public class SQLHandler extends RequestHandlerBase implements SolrCoreAware, Per
/*
* Only necessary for SolrJ JDBC driver since metadata has to be passed back
*/
private class SqlHandlerStream extends JDBCStream {
private static class SqlHandlerStream extends JDBCStream {
private final boolean includeMetadata;
private boolean firstTuple = true;
List<String> metadataFields = new ArrayList<>();

View File

@ -521,7 +521,7 @@ public class ExpandComponent extends SearchComponent implements PluginInfoInitia
rb.rsp.add("expanded", expanded);
}
private class GroupExpandCollector implements Collector, GroupCollector {
private static class GroupExpandCollector implements Collector, GroupCollector {
private SortedDocValues docValues;
private MultiDocValues.OrdinalMap ordinalMap;
private SortedDocValues segmentValues;
@ -614,7 +614,7 @@ public class ExpandComponent extends SearchComponent implements PluginInfoInitia
}
}
private class NumericGroupExpandCollector implements Collector, GroupCollector {
private static class NumericGroupExpandCollector implements Collector, GroupCollector {
private NumericDocValues docValues;
private String field;
@ -763,7 +763,7 @@ public class ExpandComponent extends SearchComponent implements PluginInfoInitia
// this reader alters the content of the given reader so it should not
// delegate the caching stuff
private class ReaderWrapper extends FilterLeafReader {
private static class ReaderWrapper extends FilterLeafReader {
private String field;

View File

@ -283,7 +283,7 @@ public class HttpShardHandlerFactory extends ShardHandlerFactory implements org.
* If all nodes prefer local-cores then a bad/heavily-loaded node will receive less requests from healthy nodes.
* This will help prevent a distributed deadlock or timeouts in all the healthy nodes due to one bad node.
*/
private class IsOnPreferredHostComparator implements Comparator<Object> {
private static class IsOnPreferredHostComparator implements Comparator<Object> {
final private String preferredHostAddress;
public IsOnPreferredHostComparator(String preferredHostAddress) {
this.preferredHostAddress = preferredHostAddress;

View File

@ -301,7 +301,7 @@ public class PivotFacetFieldValueCollection implements Iterable<PivotFacetValue>
}
/** Sorts {@link PivotFacetValue} instances by their count */
public class PivotFacetCountComparator implements Comparator<PivotFacetValue> {
public static class PivotFacetCountComparator implements Comparator<PivotFacetValue> {
public int compare(PivotFacetValue left, PivotFacetValue right) {
int countCmp = right.getCount() - left.getCount();
return (0 != countCmp) ? countCmp :
@ -310,7 +310,7 @@ public class PivotFacetFieldValueCollection implements Iterable<PivotFacetValue>
}
/** Sorts {@link PivotFacetValue} instances by their value */
public class PivotFacetValueComparator implements Comparator<PivotFacetValue> {
public static class PivotFacetValueComparator implements Comparator<PivotFacetValue> {
public int compare(PivotFacetValue left, PivotFacetValue right) {
return compareWithNullLast(left.getValue(), right.getValue());
}

View File

@ -353,7 +353,7 @@ public class TermsComponent extends SearchComponent {
return sreq;
}
public class TermsHelper {
public static class TermsHelper {
// map to store returned terms
private HashMap<String, HashMap<String, TermsResponse.Term>> fieldmap;
private SolrParams params;

View File

@ -750,7 +750,7 @@ public class DefaultSolrHighlighter extends SolrHighlighter implements PluginInf
}
// Wraps FVH to allow pass-by-reference. Public access to allow use in 3rd party subclasses
public class FvhContainer {
public static class FvhContainer {
FastVectorHighlighter fvh;
FieldQuery fieldQuery;

View File

@ -368,13 +368,13 @@ public final class ManagedIndexSchema extends IndexSchema {
}
public class FieldExistsException extends SolrException {
public static class FieldExistsException extends SolrException {
public FieldExistsException(ErrorCode code, String msg) {
super(code, msg);
}
}
public class SchemaChangedInZkException extends SolrException {
public static class SchemaChangedInZkException extends SolrException {
public SchemaChangedInZkException(ErrorCode code, String msg) {
super(code, msg);
}

View File

@ -160,7 +160,7 @@ public class RandomSortField extends FieldType {
public class RandomValueSource extends ValueSource {
public static class RandomValueSource extends ValueSource {
private final String field;
public RandomValueSource(String field) {

View File

@ -59,9 +59,9 @@ public class ComplexPhraseQParserPlugin extends QParserPlugin {
/**
* Modified from {@link org.apache.solr.search.LuceneQParser} and {@link org.apache.solr.search.SurroundQParserPlugin.SurroundQParser}
*/
class ComplexPhraseQParser extends QParser {
static class ComplexPhraseQParser extends QParser {
final class SolrQueryParserDelegate extends SolrQueryParser {
static final class SolrQueryParserDelegate extends SolrQueryParser {
private SolrQueryParserDelegate(QParser parser, String defaultField) {
super(parser, defaultField);
}

View File

@ -121,7 +121,7 @@ public class ExportQParserPlugin extends QParserPlugin {
}
}
private class ExportCollector extends TopDocsCollector {
private static class ExportCollector extends TopDocsCollector {
private FixedBitSet[] sets;

View File

@ -956,7 +956,7 @@ public class ExtendedDismaxQParser extends QParser {
/** A simple container for storing alias info
*/
protected class Alias {
protected static class Alias {
public float tie;
public Map<String,Float> fields;
}
@ -1597,7 +1597,7 @@ public class ExtendedDismaxQParser extends QParser {
/**
* Simple container for configuration information used when parsing queries
*/
public class ExtendedDismaxConfiguration {
public static class ExtendedDismaxConfiguration {
/**
* The field names specified by 'qf' that (most) clauses will

View File

@ -59,7 +59,7 @@ public class HashQParserPlugin extends QParserPlugin {
return new HashQParser(query, localParams, params, request);
}
private class HashQParser extends QParser {
private static class HashQParser extends QParser {
public HashQParser(String query, SolrParams localParams, SolrParams params, SolrQueryRequest request) {
super(query, localParams, params, request);
@ -74,7 +74,7 @@ public class HashQParserPlugin extends QParserPlugin {
}
}
private class HashQuery extends ExtendedQueryBase implements PostFilter {
private static class HashQuery extends ExtendedQueryBase implements PostFilter {
private String keysParam;
private int workers;
@ -135,7 +135,7 @@ public class HashQParserPlugin extends QParserPlugin {
return searcher.rewrite(constantScoreQuery).createWeight(searcher, false, boost);
}
public class BitsFilter extends Filter {
public static class BitsFilter extends Filter {
private FixedBitSet[] bitSets;
public BitsFilter(FixedBitSet[] bitSets) {
this.bitSets = bitSets;
@ -166,7 +166,7 @@ public class HashQParserPlugin extends QParserPlugin {
}
class SegmentPartitioner implements Runnable {
static class SegmentPartitioner implements Runnable {
public LeafReaderContext context;
private int worker;
@ -238,7 +238,7 @@ public class HashQParserPlugin extends QParserPlugin {
}
}
private class HashCollector extends DelegatingCollector {
private static class HashCollector extends DelegatingCollector {
private int worker;
private int workers;
private HashKey hashKey;
@ -271,7 +271,7 @@ public class HashQParserPlugin extends QParserPlugin {
public long hashCode(int doc) throws IOException;
}
private class BytesHash implements HashKey {
private static class BytesHash implements HashKey {
private SortedDocValues values;
private String field;
@ -303,7 +303,7 @@ public class HashQParserPlugin extends QParserPlugin {
}
}
private class NumericHash implements HashKey {
private static class NumericHash implements HashKey {
private NumericDocValues values;
private String field;
@ -331,7 +331,7 @@ public class HashQParserPlugin extends QParserPlugin {
}
}
private class ZeroHash implements HashKey {
private static class ZeroHash implements HashKey {
public long hashCode(int doc) {
return 0;
@ -342,7 +342,7 @@ public class HashQParserPlugin extends QParserPlugin {
}
}
private class CompositeHash implements HashKey {
private static class CompositeHash implements HashKey {
private HashKey key1;
private HashKey key2;

View File

@ -139,7 +139,7 @@ public class ReRankCollector extends TopDocsCollector {
}
}
public class BoostedComp implements Comparator {
public static class BoostedComp implements Comparator {
IntFloatHashMap boostedMap;
public BoostedComp(IntIntHashMap boostedDocs, ScoreDoc[] scoreDocs, float maxScore) {

Some files were not shown because too many files have changed in this diff Show More