You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ro...@apache.org on 2012/11/23 13:01:26 UTC
svn commit: r1412849 [2/13] - in /lucene/dev/branches/LUCENE-2878: ./
dev-tools/ dev-tools/eclipse/ dev-tools/idea/.idea/libraries/
dev-tools/idea/solr/contrib/dataimporthandler/ dev-tools/maven/
dev-tools/maven/solr/contrib/dataimporthandler/ dev-tool...
Modified: lucene/dev/branches/LUCENE-2878/lucene/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/CHANGES.txt?rev=1412849&r1=1412848&r2=1412849&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/CHANGES.txt (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/CHANGES.txt Fri Nov 23 12:00:32 2012
@@ -8,9 +8,12 @@ http://s.apache.org/luceneversions
Changes in backwards compatibility policy
+* LUCENE-4535: oal.util.FilterIterator is now an internal API.
+ (Adrien Grand)
+
* LUCENE-3312: The API of oal.document was restructured to
differentiate between stored documents and indexed documents.
- IndexReader.document(int) now returns StorableDocument
+ IndexReader.document(int) now returns StoredDocument
instead of Document. In most cases a simple replacement
of the return type is enough to upgrade (see MIGRATE.txt).
(Nikola Tanković, Uwe Schindler, Chris Male, Mike McCandless,
@@ -18,11 +21,43 @@ Changes in backwards compatibility polic
======================= Lucene 4.1.0 =======================
+Changes in backwards compatibility policy
+
+* LUCENE-4514: Scorer's freq() method returns an integer value indicating
+ the number of times the scorer matches the current document. Previously
+ this was only sometimes the case, in some cases it returned a (meaningless)
+ floating point value. Scorer now extends DocsEnum so it has attributes().
+ (Robert Muir)
+
+* LUCENE-4543: TFIDFSimilarity's index-time computeNorm is now final to
+ match the fact that its query-time norm usage requires a FIXED_8 encoding.
+ Override lengthNorm and/or encode/decodeNormValue to change the specifics,
+ like Lucene 3.x. (Robert Muir)
+
+* LUCENE-3441: The facet module now supports NRT. As a result, the following
+ changes were made:
+ - DirectoryTaxonomyReader has a new constructor which takes a
+ DirectoryTaxonomyWriter. You should use that constructor in order to get
+ the NRT support (or the old one for non-NRT).
+ - TaxonomyReader.refresh() removed in exchange for TaxonomyReader.openIfChanged
+ static method. Similar to DirectoryReader, the method either returns null
+ if no changes were made to the taxonomy, or a new TR instance otherwise.
+ Instead of calling refresh(), you should write similar code to how you reopen
+ a regular DirectoryReader.
+ - TaxonomyReader.openIfChanged (previously refresh()) no longer throws
+ IncosistentTaxonomyException, and supports recreate. InconsistentTaxoEx
+ was removed.
+ - ChildrenArrays was pulled out of TaxonomyReader into a top-level class.
+ - TaxonomyReader was made an abstract class (instead of an interface), with
+ methods such as close() and reference counting management pulled from
+ DirectoryTaxonomyReader, and made final. The rest of the methods, remained
+ abstract.
+ (Shai Erera, Gilad Barkai)
+
New Features
-* LUCENE-4226: New experimental StoredFieldsFormat (in lucene/codecs) that
- compresses chunks of documents together in order to improve the compression
- ratio. (Adrien Grand)
+* LUCENE-4226: New experimental StoredFieldsFormat that compresses chunks of
+ documents together in order to improve the compression ratio. (Adrien Grand)
* LUCENE-4426: New ValueSource implementations (in lucene/queries) for
DocValues fields. (Adrien Grand)
@@ -49,6 +84,21 @@ New Features
for better search performance.
(Han Jiang, Adrien Grand, Robert Muir, Mike McCandless)
+* LUCENE-3846: New FuzzySuggester, like AnalyzingSuggester except it
+ also finds completions allowing for fuzzy edits in the input string.
+ (Robert Muir, Simon Willnauer, Mike McCandless)
+
+* LUCENE-4515: MemoryIndex now supports adding the same field multiple
+ times. (Simon Willnauer)
+
+* LUCENE-4540: Added an experimental Norm.setPackedLong, which allows
+ the use of VAR_INTS-encoded norms. This can be useful for cases where
+ you only need a few bits per-document, or where you might want exact
+ document length, and so on. (Robert Muir)
+
+* LUCENE-4489: Added consumeAllTokens option to LimitTokenCountFilter
+ (hossman, Robert Muir)
+
API Changes
* LUCENE-4399: Deprecated AppendingCodec. Lucene's term dictionaries
@@ -62,6 +112,16 @@ API Changes
information about the trigger of the merge ie. merge triggered due
to a segment merge or a full flush etc. (Simon Willnauer)
+* Lucene-4415: TermsFilter is now immutable. All terms need to be provided
+ as constructor argument. (Simon Willnauer)
+
+* LUCENE-4520: ValueSource.getSortField no longer throws IOExceptions
+ (Alan Woodward)
+
+* LUCENE-4537: RateLimiter is now separated from FSDirectory and exposed via
+ RateLimitingDirectoryWrapper. Any Directory can now be rate-limited.
+ (Simon Willnauer)
+
Bug Fixes
* LUCENE-1822: BaseFragListBuilder hard-coded 6 char margin is too naive.
@@ -86,8 +146,54 @@ Bug Fixes
* LUCENE-4504: Fix broken sort comparator in ValueSource.getSortField,
used when sorting by a function query. (Tom Shally via Robert Muir)
+* LUCENE-4511: TermsFilter might return wrong results if a field is not
+ indexed or doesn't exist in the index. (Simon Willnauer)
+
+* LUCENE-4521: IndexWriter.tryDeleteDocument could return true
+ (successfully deleting the document) but then on IndexWriter
+ close/commit fail to write the new deletions, if no other changes
+ happened in the IndexWriter instance. (Ivan Vasilev via Mike
+ McCandless)
+
+* LUCENE-4513: Fixed that deleted nested docs are scored into the
+ parent doc when using ToParentBlockJoinQuery. (Martijn van Groningen)
+
+* LUCENE-4534: Fixed WFSTCompletionLookup and Analyzing/FuzzySuggester
+ to allow 0 byte values in the lookup keys. (Mike McCandless)
+
+* LUCENE-4532: DirectoryTaxonomyWriter use a timestamp to denote taxonomy
+ index re-creation, which could cause a bug in case machine clocks were
+ not synced. Instead, it now tracks an 'epoch' version, which is incremented
+ whenever the taxonomy is re-created, or replaced. (Shai Erera)
+
+* LUCENE-4544: Fixed off-by-1 in ConcurrentMergeScheduler that would
+ allow 1+maxMergeCount merges threads to be created, instead of just
+ maxMergeCount (Radim Kolar, Mike McCandless)
+
+* LUCENE-4567: Fixed NullPointerException in analzying, fuzzy, and
+ WFST suggesters when no suggestions were added (selckin via Mike
+ McCandless)
+
+* LUCENE-4568: Fixed integer overflow in
+ PagedBytes.PagedBytesData{In,Out}put.getPosition. (Adrien Grand)
+
Optimizations
+* LUCENE-2221: oal.util.BitUtil was modified to use Long.bitCount and
+ Long.numberOfTrailingZeros (which are intrinsics since Java 6u18) instead of
+ pure java bit twiddling routines in order to improve performance on modern
+ JVMs/hardware. (Dawid Weiss, Adrien Grand)
+
+* LUCENE-4509: Enable stored fields compression by default in the Lucene 4.1
+ default codec. (Adrien Grand)
+
+* LUCENE-4536: PackedInts on-disk format is now byte-aligned (it used to be
+ long-aligned), saving up to 7 bytes per array of values.
+ (Adrien Grand, Mike McCandless)
+
+* LUCENE-4512: Additional memory savings for CompressingStoredFieldsFormat.
+ (Adrien Grand, Robert Muir)
+
* LUCENE-4443: Lucene41PostingsFormat no longer writes unnecessary offsets
into the skipdata. (Robert Muir)
@@ -115,6 +221,19 @@ Optimizations
Instead of writing a file pointer to a VIntBlock containing the doc id, just
write the doc id. (Mike McCandless, Robert Muir)
+* LUCENE-4515: MemoryIndex now uses Byte/IntBlockPool internally to hold terms and
+ posting lists. All index data is represented as consecutive byte/int arrays to
+ reduce GC cost and memory overhead. (Simon Willnauer)
+
+* LUCENE-4538: DocValues now caches direct sources in a ThreadLocal exposed via SourceCache.
+ Users of this API can now simply obtain an instance via DocValues#getDirectSource per thread.
+ (Simon Willnauer)
+
+Documentation
+
+* LUCENE-4483: Refer to BytesRef.deepCopyOf in Term's constructor that takes BytesRef.
+ (Paul Elschot via Robert Muir)
+
Build
* Upgrade randomized testing to version 2.0.4: avoid hangs on shutdown
@@ -309,7 +428,7 @@ Bug Fixes
many cases where IndexWriter could leave leftover files (on
exception in various places, on reuse of a segment name after crash
and recovery. (Uwe Schindler, Robert Muir, Mike McCandless)
-
+
Optimizations
* LUCENE-4322: Decrease lucene-core JAR size. The core JAR size had increased a
Modified: lucene/dev/branches/LUCENE-2878/lucene/MIGRATE.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/MIGRATE.txt?rev=1412849&r1=1412848&r2=1412849&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/MIGRATE.txt (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/MIGRATE.txt Fri Nov 23 12:00:32 2012
@@ -4,5 +4,5 @@
The API of oal.document was restructured to differentiate between stored
documents and indexed documents. IndexReader.document(int) now returns
-StorableDocument instead of Document. In most cases a simple replacement
+StoredDocument instead of Document. In most cases a simple replacement
of the return type is enough to upgrade.
Modified: lucene/dev/branches/LUCENE-2878/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/LimitTokenCountAnalyzer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/LimitTokenCountAnalyzer.java?rev=1412849&r1=1412848&r2=1412849&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/LimitTokenCountAnalyzer.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/LimitTokenCountAnalyzer.java Fri Nov 23 12:00:32 2012
@@ -23,17 +23,32 @@ import org.apache.lucene.analysis.Analyz
/**
* This Analyzer limits the number of tokens while indexing. It is
* a replacement for the maximum field length setting inside {@link org.apache.lucene.index.IndexWriter}.
+ * @see LimitTokenCountFilter
*/
public final class LimitTokenCountAnalyzer extends AnalyzerWrapper {
private final Analyzer delegate;
private final int maxTokenCount;
+ private final boolean consumeAllTokens;
/**
* Build an analyzer that limits the maximum number of tokens per field.
+ * This analyzer will not consume any tokens beyond the maxTokenCount limit
+ *
+ * @see #LimitTokenCountAnalyzer(Analyzer,int,boolean)
*/
public LimitTokenCountAnalyzer(Analyzer delegate, int maxTokenCount) {
+ this(delegate, maxTokenCount, false);
+ }
+ /**
+ * Build an analyzer that limits the maximum number of tokens per field.
+ * @param delegate the analyzer to wrap
+ * @param maxTokenCount max number of tokens to produce
+ * @param consumeAllTokens whether all tokens from the delegate should be consumed even if maxTokenCount is reached.
+ */
+ public LimitTokenCountAnalyzer(Analyzer delegate, int maxTokenCount, boolean consumeAllTokens) {
this.delegate = delegate;
this.maxTokenCount = maxTokenCount;
+ this.consumeAllTokens = consumeAllTokens;
}
@Override
@@ -44,11 +59,11 @@ public final class LimitTokenCountAnalyz
@Override
protected TokenStreamComponents wrapComponents(String fieldName, TokenStreamComponents components) {
return new TokenStreamComponents(components.getTokenizer(),
- new LimitTokenCountFilter(components.getTokenStream(), maxTokenCount));
+ new LimitTokenCountFilter(components.getTokenStream(), maxTokenCount, consumeAllTokens));
}
@Override
public String toString() {
- return "LimitTokenCountAnalyzer(" + delegate.toString() + ", maxTokenCount=" + maxTokenCount + ")";
+ return "LimitTokenCountAnalyzer(" + delegate.toString() + ", maxTokenCount=" + maxTokenCount + ", consumeAllTokens=" + consumeAllTokens + ")";
}
}
Modified: lucene/dev/branches/LUCENE-2878/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/LimitTokenCountFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/LimitTokenCountFilter.java?rev=1412849&r1=1412848&r2=1412849&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/LimitTokenCountFilter.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/LimitTokenCountFilter.java Fri Nov 23 12:00:32 2012
@@ -25,32 +25,68 @@ import java.io.IOException;
/**
* This TokenFilter limits the number of tokens while indexing. It is
* a replacement for the maximum field length setting inside {@link org.apache.lucene.index.IndexWriter}.
+ * <p>
+ * By default, this filter ignores any tokens in the wrapped {@code TokenStream}
+ * once the limit has been reached, which can result in {@code reset()} being
+ * called prior to {@code incrementToken()} returning {@code false}. For most
+ * {@code TokenStream} implementations this should be acceptable, and faster
+ * then consuming the full stream. If you are wrapping a {@code TokenStream}
+ * which requires that the full stream of tokens be exhausted in order to
+ * function properly, use the
+ * {@link #LimitTokenCountFilter(TokenStream,int,boolean) consumeAllTokens}
+ * option.
*/
public final class LimitTokenCountFilter extends TokenFilter {
private final int maxTokenCount;
+ private final boolean consumeAllTokens;
private int tokenCount = 0;
+ private boolean exhausted = false;
/**
* Build a filter that only accepts tokens up to a maximum number.
+ * This filter will not consume any tokens beyond the maxTokenCount limit
+ *
+ * @see #LimitTokenCountFilter(TokenStream,int,boolean)
*/
public LimitTokenCountFilter(TokenStream in, int maxTokenCount) {
+ this(in, maxTokenCount, false);
+ }
+
+ /**
+ * Build an filter that limits the maximum number of tokens per field.
+ * @param in the stream to wrap
+ * @param maxTokenCount max number of tokens to produce
+ * @param consumeAllTokens whether all tokens from the input must be consumed even if maxTokenCount is reached.
+ */
+ public LimitTokenCountFilter(TokenStream in, int maxTokenCount, boolean consumeAllTokens) {
super(in);
this.maxTokenCount = maxTokenCount;
+ this.consumeAllTokens = consumeAllTokens;
}
@Override
public boolean incrementToken() throws IOException {
- if (tokenCount < maxTokenCount && input.incrementToken()) {
- tokenCount++;
- return true;
+ if (exhausted) {
+ return false;
+ } else if (tokenCount < maxTokenCount) {
+ if (input.incrementToken()) {
+ tokenCount++;
+ return true;
+ } else {
+ exhausted = true;
+ return false;
+ }
+ } else {
+ while (consumeAllTokens && input.incrementToken()) { /* NOOP */ }
+ return false;
}
- return false;
}
@Override
public void reset() throws IOException {
super.reset();
tokenCount = 0;
+ exhausted = false;
}
}
Modified: lucene/dev/branches/LUCENE-2878/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/LimitTokenCountFilterFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/LimitTokenCountFilterFactory.java?rev=1412849&r1=1412848&r2=1412849&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/LimitTokenCountFilterFactory.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/LimitTokenCountFilterFactory.java Fri Nov 23 12:00:32 2012
@@ -29,25 +29,29 @@ import org.apache.lucene.analysis.util.T
* <fieldType name="text_lngthcnt" class="solr.TextField" positionIncrementGap="100">
* <analyzer>
* <tokenizer class="solr.WhitespaceTokenizerFactory"/>
- * <filter class="solr.LimitTokenCountFilterFactory" maxTokenCount="10"/>
+ * <filter class="solr.LimitTokenCountFilterFactory" maxTokenCount="10" consumeAllTokens="false" />
* </analyzer>
* </fieldType></pre>
- *
+ * <p>
+ * The {@code consumeAllTokens} property is optional and defaults to {@code false}. See {@link LimitTokenCountFilter} for an explanation of it's use.
*/
public class LimitTokenCountFilterFactory extends TokenFilterFactory {
public static final String MAX_TOKEN_COUNT_KEY = "maxTokenCount";
+ public static final String CONSUME_ALL_TOKENS_KEY = "consumeAllTokens";
int maxTokenCount;
+ boolean consumeAllTokens;
@Override
public void init(Map<String, String> args) {
super.init( args );
maxTokenCount = getInt(MAX_TOKEN_COUNT_KEY);
+ consumeAllTokens = getBoolean(CONSUME_ALL_TOKENS_KEY, false);
}
@Override
public TokenStream create(TokenStream input) {
- return new LimitTokenCountFilter(input, maxTokenCount);
+ return new LimitTokenCountFilter(input, maxTokenCount, consumeAllTokens);
}
}
Modified: lucene/dev/branches/LUCENE-2878/lucene/analysis/common/src/test/org/apache/lucene/analysis/core/TestRandomChains.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/analysis/common/src/test/org/apache/lucene/analysis/core/TestRandomChains.java?rev=1412849&r1=1412848&r2=1412849&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/analysis/common/src/test/org/apache/lucene/analysis/core/TestRandomChains.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/analysis/common/src/test/org/apache/lucene/analysis/core/TestRandomChains.java Fri Nov 23 12:00:32 2012
@@ -59,6 +59,7 @@ import org.apache.lucene.analysis.Valida
import org.apache.lucene.analysis.charfilter.NormalizeCharMap;
import org.apache.lucene.analysis.cjk.CJKBigramFilter;
import org.apache.lucene.analysis.commongrams.CommonGramsFilter;
+import org.apache.lucene.analysis.commongrams.CommonGramsQueryFilter;
import org.apache.lucene.analysis.compound.DictionaryCompoundWordTokenFilter;
import org.apache.lucene.analysis.compound.HyphenationCompoundWordTokenFilter;
import org.apache.lucene.analysis.compound.TestCompoundWordTokenFilter;
@@ -160,7 +161,9 @@ public class TestRandomChains extends Ba
// TODO: doesn't handle graph inputs (or even look at positionIncrement)
HyphenatedWordsFilter.class,
// LUCENE-4065: only if you pass 'false' to enablePositionIncrements!
- TypeTokenFilter.class
+ TypeTokenFilter.class,
+ // TODO: doesn't handle graph inputs
+ CommonGramsQueryFilter.class
);
}
Modified: lucene/dev/branches/LUCENE-2878/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestLimitTokenCountAnalyzer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestLimitTokenCountAnalyzer.java?rev=1412849&r1=1412848&r2=1412849&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestLimitTokenCountAnalyzer.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestLimitTokenCountAnalyzer.java Fri Nov 23 12:00:32 2012
@@ -33,40 +33,64 @@ import org.apache.lucene.index.IndexWrit
import org.apache.lucene.index.IndexWriterConfig;
import org.apache.lucene.index.Term;
import org.apache.lucene.store.Directory;
+import org.apache.lucene.util._TestUtil;
public class TestLimitTokenCountAnalyzer extends BaseTokenStreamTestCase {
public void testLimitTokenCountAnalyzer() throws IOException {
- Analyzer a = new LimitTokenCountAnalyzer(new WhitespaceAnalyzer(TEST_VERSION_CURRENT), 2);
- // dont use assertAnalyzesTo here, as the end offset is not the end of the string!
- assertTokenStreamContents(a.tokenStream("dummy", new StringReader("1 2 3 4 5")), new String[] { "1", "2" }, new int[] { 0, 3 }, new int[] { 1, 4 }, 4);
- assertTokenStreamContents(a.tokenStream("dummy", new StringReader("1 2 3 4 5")), new String[] { "1", "2" }, new int[] { 0, 2 }, new int[] { 1, 3 }, 3);
+ for (boolean consumeAll : new boolean[] { true, false }) {
+ MockAnalyzer mock = new MockAnalyzer(random());
+
+ // if we are consuming all tokens, we can use the checks,
+ // otherwise we can't
+ mock.setEnableChecks(consumeAll);
+ Analyzer a = new LimitTokenCountAnalyzer(mock, 2, consumeAll);
+
+ // dont use assertAnalyzesTo here, as the end offset is not the end of the string (unless consumeAll is true, in which case its correct)!
+ assertTokenStreamContents(a.tokenStream("dummy", new StringReader("1 2 3 4 5")), new String[] { "1", "2" }, new int[] { 0, 3 }, new int[] { 1, 4 }, consumeAll ? 16 : null);
+ assertTokenStreamContents(a.tokenStream("dummy", new StringReader("1 2 3 4 5")), new String[] { "1", "2" }, new int[] { 0, 2 }, new int[] { 1, 3 }, consumeAll ? 9 : null);
+
+ // less than the limit, ensure we behave correctly
+ assertTokenStreamContents(a.tokenStream("dummy", new StringReader("1 ")), new String[] { "1" }, new int[] { 0 }, new int[] { 1 }, consumeAll ? 3 : null);
- a = new LimitTokenCountAnalyzer(new StandardAnalyzer(TEST_VERSION_CURRENT), 2);
- // dont use assertAnalyzesTo here, as the end offset is not the end of the string!
- assertTokenStreamContents(a.tokenStream("dummy", new StringReader("1 2 3 4 5")), new String[] { "1", "2" }, new int[] { 0, 2 }, new int[] { 1, 3 }, 3);
+ // equal to limit
+ assertTokenStreamContents(a.tokenStream("dummy", new StringReader("1 2 ")), new String[] { "1", "2" }, new int[] { 0, 3 }, new int[] { 1, 4 }, consumeAll ? 6 : null);
+ }
}
public void testLimitTokenCountIndexWriter() throws IOException {
- Directory dir = newDirectory();
-
- IndexWriter writer = new IndexWriter(dir, new IndexWriterConfig(
- TEST_VERSION_CURRENT, new LimitTokenCountAnalyzer(new MockAnalyzer(random()), 100000)));
-
- Document doc = new Document();
- StringBuilder b = new StringBuilder();
- for(int i=0;i<10000;i++)
- b.append(" a");
- b.append(" x");
- doc.add(newTextField("field", b.toString(), Field.Store.NO));
- writer.addDocument(doc);
- writer.close();
-
- IndexReader reader = DirectoryReader.open(dir);
- Term t = new Term("field", "x");
- assertEquals(1, reader.docFreq(t));
- reader.close();
- dir.close();
+
+ for (boolean consumeAll : new boolean[] { true, false }) {
+ Directory dir = newDirectory();
+ int limit = _TestUtil.nextInt(random(), 50, 101000);
+ MockAnalyzer mock = new MockAnalyzer(random());
+
+ // if we are consuming all tokens, we can use the checks,
+ // otherwise we can't
+ mock.setEnableChecks(consumeAll);
+ Analyzer a = new LimitTokenCountAnalyzer(mock, limit, consumeAll);
+
+ IndexWriter writer = new IndexWriter(dir, new IndexWriterConfig
+ (TEST_VERSION_CURRENT, a));
+
+ Document doc = new Document();
+ StringBuilder b = new StringBuilder();
+ for(int i=1;i<limit;i++)
+ b.append(" a");
+ b.append(" x");
+ b.append(" z");
+ doc.add(newTextField("field", b.toString(), Field.Store.NO));
+ writer.addDocument(doc);
+ writer.close();
+
+ IndexReader reader = DirectoryReader.open(dir);
+ Term t = new Term("field", "x");
+ assertEquals(1, reader.docFreq(t));
+ t = new Term("field", "z");
+ assertEquals(0, reader.docFreq(t));
+ reader.close();
+ dir.close();
+ }
}
-}
\ No newline at end of file
+}
Modified: lucene/dev/branches/LUCENE-2878/lucene/analysis/morfologik/src/java/org/apache/lucene/analysis/morfologik/MorfologikFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/analysis/morfologik/src/java/org/apache/lucene/analysis/morfologik/MorfologikFilter.java?rev=1412849&r1=1412848&r2=1412849&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/analysis/morfologik/src/java/org/apache/lucene/analysis/morfologik/MorfologikFilter.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/analysis/morfologik/src/java/org/apache/lucene/analysis/morfologik/MorfologikFilter.java Fri Nov 23 12:00:32 2012
@@ -67,9 +67,18 @@ public class MorfologikFilter extends To
public MorfologikFilter(final TokenStream in, final DICTIONARY dict, final Version version) {
super(in);
this.input = in;
- this.stemmer = new PolishStemmer(dict);
- this.charUtils = CharacterUtils.getInstance(version);
- this.lemmaList = Collections.emptyList();
+
+ // SOLR-4007: temporarily substitute context class loader to allow finding dictionary resources.
+ Thread me = Thread.currentThread();
+ ClassLoader cl = me.getContextClassLoader();
+ try {
+ me.setContextClassLoader(PolishStemmer.class.getClassLoader());
+ this.stemmer = new PolishStemmer(dict);
+ this.charUtils = CharacterUtils.getInstance(version);
+ this.lemmaList = Collections.emptyList();
+ } finally {
+ me.setContextClassLoader(cl);
+ }
}
private void popNextLemma() {
Modified: lucene/dev/branches/LUCENE-2878/lucene/classification/src/java/org/apache/lucene/classification/KNearestNeighborClassifier.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/classification/src/java/org/apache/lucene/classification/KNearestNeighborClassifier.java?rev=1412849&r1=1412848&r2=1412849&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/classification/src/java/org/apache/lucene/classification/KNearestNeighborClassifier.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/classification/src/java/org/apache/lucene/classification/KNearestNeighborClassifier.java Fri Nov 23 12:00:32 2012
@@ -32,6 +32,7 @@ import java.util.Map;
/**
* A k-Nearest Neighbor classifier (see <code>http://en.wikipedia.org/wiki/K-nearest_neighbors</code>) based
* on {@link MoreLikeThis}
+ *
* @lucene.experimental
*/
public class KNearestNeighborClassifier implements Classifier {
@@ -44,6 +45,7 @@ public class KNearestNeighborClassifier
/**
* Create a {@link Classifier} using kNN algorithm
+ *
* @param k the number of neighbors to analyze as an <code>int</code>
*/
public KNearestNeighborClassifier(int k) {
@@ -62,8 +64,7 @@ public class KNearestNeighborClassifier
Integer count = classCounts.get(cl);
if (count != null) {
classCounts.put(cl, count + 1);
- }
- else {
+ } else {
classCounts.put(cl, 1);
}
}
@@ -76,7 +77,7 @@ public class KNearestNeighborClassifier
assignedClass = cl;
}
}
- double score = 1; // TODO : derive score from query
+ double score = max / k;
return new ClassificationResult(assignedClass, score);
}
Modified: lucene/dev/branches/LUCENE-2878/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextPerDocProducer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextPerDocProducer.java?rev=1412849&r1=1412848&r2=1412849&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextPerDocProducer.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextPerDocProducer.java Fri Nov 23 12:00:32 2012
@@ -136,7 +136,7 @@ public class SimpleTextPerDocProducer ex
}
@Override
- public Source load() throws IOException {
+ protected Source loadSource() throws IOException {
boolean success = false;
IndexInput in = input.clone();
try {
@@ -198,9 +198,14 @@ public class SimpleTextPerDocProducer ex
assert scratch.equals(END);
return reader.getSource();
}
-
+
@Override
public Source getDirectSource() throws IOException {
+ return this.getSource(); // don't cache twice
+ }
+
+ @Override
+ protected Source loadDirectSource() throws IOException {
return this.getSource();
}
Modified: lucene/dev/branches/LUCENE-2878/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextTermVectorsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextTermVectorsReader.java?rev=1412849&r1=1412848&r2=1412849&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextTermVectorsReader.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextTermVectorsReader.java Fri Nov 23 12:00:32 2012
@@ -97,12 +97,6 @@ public class SimpleTextTermVectorsReader
@Override
public Fields get(int doc) throws IOException {
- // TestTV tests for this in testBadParams... but is this
- // really guaranteed by the API?
- if (doc < 0 || doc >= offsets.length) {
- throw new IllegalArgumentException("doc id out of range");
- }
-
SortedMap<String,SimpleTVTerms> fields = new TreeMap<String,SimpleTVTerms>();
in.seek(offsets[doc]);
readLine();
Modified: lucene/dev/branches/LUCENE-2878/lucene/codecs/src/test/org/apache/lucene/codecs/compressing/AbstractTestCompressionMode.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/codecs/src/test/org/apache/lucene/codecs/compressing/AbstractTestCompressionMode.java?rev=1412849&r1=1412848&r2=1412849&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/codecs/src/test/org/apache/lucene/codecs/compressing/AbstractTestCompressionMode.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/codecs/src/test/org/apache/lucene/codecs/compressing/AbstractTestCompressionMode.java Fri Nov 23 12:00:32 2012
@@ -38,6 +38,10 @@ public abstract class AbstractTestCompre
final int length = random().nextBoolean()
? random().nextInt(20)
: random().nextInt(192 * 1024);
+ return randomArray(length, max);
+ }
+
+ static byte[] randomArray(int length, int max) {
final byte[] arr = new byte[length];
for (int i = 0; i < arr.length; ++i) {
arr[i] = (byte) RandomInts.randomIntBetween(random(), 0, max);
@@ -45,67 +49,125 @@ public abstract class AbstractTestCompre
return arr;
}
- byte[] compress(byte[] uncompressed) throws IOException {
+ byte[] compress(byte[] decompressed) throws IOException {
Compressor compressor = mode.newCompressor();
- return compress(compressor, uncompressed);
+ return compress(compressor, decompressed);
}
- static byte[] compress(Compressor compressor, byte[] uncompressed) throws IOException {
- byte[] compressed = new byte[uncompressed.length * 2 + 16]; // should be enough
+ static byte[] compress(Compressor compressor, byte[] decompressed) throws IOException {
+ byte[] compressed = new byte[decompressed.length * 2 + 16]; // should be enough
ByteArrayDataOutput out = new ByteArrayDataOutput(compressed);
- compressor.compress(uncompressed, 0, uncompressed.length, out);
+ compressor.compress(decompressed, 0, decompressed.length, out);
final int compressedLen = out.getPosition();
return Arrays.copyOf(compressed, compressedLen);
}
- byte[] uncompress(byte[] compressed) throws IOException {
- Uncompressor uncompressor = mode.newUncompressor();
- return uncompress(uncompressor, compressed);
+ byte[] decompress(byte[] compressed, int originalLength) throws IOException {
+ Decompressor decompressor = mode.newDecompressor();
+ return decompress(decompressor, compressed, originalLength);
}
- static byte[] uncompress(Uncompressor uncompressor, byte[] compressed) throws IOException {
+ static byte[] decompress(Decompressor decompressor, byte[] compressed, int originalLength) throws IOException {
final BytesRef bytes = new BytesRef();
- uncompressor.uncompress(new ByteArrayDataInput(compressed), bytes);
+ decompressor.decompress(new ByteArrayDataInput(compressed), originalLength, 0, originalLength, bytes);
return Arrays.copyOfRange(bytes.bytes, bytes.offset, bytes.offset + bytes.length);
}
- byte[] uncompress(byte[] compressed, int offset, int length) throws IOException {
- Uncompressor uncompressor = mode.newUncompressor();
+ byte[] decompress(byte[] compressed, int originalLength, int offset, int length) throws IOException {
+ Decompressor decompressor = mode.newDecompressor();
final BytesRef bytes = new BytesRef();
- uncompressor.uncompress(new ByteArrayDataInput(compressed), offset, length, bytes);
+ decompressor.decompress(new ByteArrayDataInput(compressed), originalLength, offset, length, bytes);
return Arrays.copyOfRange(bytes.bytes, bytes.offset, bytes.offset + bytes.length);
}
- public void testUncompress() throws IOException {
- final byte[] uncompressed = randomArray();
- final byte[] compressed = compress(uncompressed);
- final byte[] restored = uncompress(compressed);
- assertArrayEquals(uncompressed, restored);
+ static byte[] copyCompressedData(Decompressor decompressor, byte[] compressed, int originalLength) throws IOException {
+ GrowableByteArrayDataOutput out = new GrowableByteArrayDataOutput(compressed.length);
+ decompressor.copyCompressedData(new ByteArrayDataInput(compressed), originalLength, out);
+ return Arrays.copyOf(out.bytes, out.length);
+ }
+
+ byte[] copyCompressedData(byte[] compressed, int originalLength) throws IOException {
+ return copyCompressedData(mode.newDecompressor(), compressed, originalLength);
+ }
+
+ public void testDecompress() throws IOException {
+ final byte[] decompressed = randomArray();
+ final byte[] compressed = compress(decompressed);
+ final byte[] restored = decompress(compressed, decompressed.length);
+ assertArrayEquals(decompressed, restored);
}
- public void testPartialUncompress() throws IOException {
+ public void testPartialDecompress() throws IOException {
final int iterations = atLeast(10);
for (int i = 0; i < iterations; ++i) {
- final byte[] uncompressed = randomArray();
- final byte[] compressed = compress(uncompressed);
+ final byte[] decompressed = randomArray();
+ final byte[] compressed = compress(decompressed);
final int offset, length;
- if (uncompressed.length == 0) {
+ if (decompressed.length == 0) {
offset = length = 0;
} else {
- offset = random().nextInt(uncompressed.length);
- length = random().nextInt(uncompressed.length - offset);
+ offset = random().nextInt(decompressed.length);
+ length = random().nextInt(decompressed.length - offset);
}
- final byte[] restored = uncompress(compressed, offset, length);
- assertArrayEquals(Arrays.copyOfRange(uncompressed, offset, offset + length), restored);
+ final byte[] restored = decompress(compressed, decompressed.length, offset, length);
+ assertArrayEquals(Arrays.copyOfRange(decompressed, offset, offset + length), restored);
}
}
public void testCopyCompressedData() throws IOException {
- final byte[] uncompressed = randomArray();
- final byte[] compressed = compress(uncompressed);
- GrowableByteArrayDataOutput out = new GrowableByteArrayDataOutput(uncompressed.length);
- mode.newUncompressor().copyCompressedData(new ByteArrayDataInput(compressed), out);
- assertArrayEquals(compressed, Arrays.copyOf(out.bytes, out.length));
+ final byte[] decompressed = randomArray();
+ final byte[] compressed = compress(decompressed);
+ assertArrayEquals(compressed, copyCompressedData(compressed, decompressed.length));
+ }
+
+ public void test(byte[] decompressed) throws IOException {
+ final byte[] compressed = compress(decompressed);
+ final byte[] restored = decompress(compressed, decompressed.length);
+ assertEquals(decompressed.length, restored.length);
+ assertArrayEquals(compressed, copyCompressedData(compressed, decompressed.length));
+ }
+
+ public void testEmptySequence() throws IOException {
+ test(new byte[0]);
+ }
+
+ public void testShortSequence() throws IOException {
+ test(new byte[] { (byte) random().nextInt(256) });
+ }
+
+ public void testIncompressible() throws IOException {
+ final byte[] decompressed = new byte[RandomInts.randomIntBetween(random(), 20, 256)];
+ for (int i = 0; i < decompressed.length; ++i) {
+ decompressed[i] = (byte) i;
+ }
+ test(decompressed);
+ }
+
+ // for LZ compression
+
+ public void testShortLiteralsAndMatchs() throws IOException {
+ // literals and matchs lengths <= 15
+ final byte[] decompressed = "1234562345673456745678910123".getBytes("UTF-8");
+ test(decompressed);
+ }
+
+ public void testLongMatchs() throws IOException {
+ // match length > 16
+ final byte[] decompressed = new byte[RandomInts.randomIntBetween(random(), 300, 1024)];
+ for (int i = 0; i < decompressed.length; ++i) {
+ decompressed[i] = (byte) i;
+ }
+ test(decompressed);
+ }
+
+ public void testLongLiterals() throws IOException {
+ // long literals (length > 16) which are not the last literals
+ final byte[] decompressed = randomArray(RandomInts.randomIntBetween(random(), 400, 1024), 256);
+ final int matchRef = random().nextInt(30);
+ final int matchOff = RandomInts.randomIntBetween(random(), decompressed.length - 40, decompressed.length - 20);
+ final int matchLength = RandomInts.randomIntBetween(random(), 4, 10);
+ System.arraycopy(decompressed, matchRef, decompressed, matchOff, matchLength);
+ test(decompressed);
}
-}
+}
\ No newline at end of file
Modified: lucene/dev/branches/LUCENE-2878/lucene/codecs/src/test/org/apache/lucene/codecs/compressing/TestCompressingStoredFieldsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/codecs/src/test/org/apache/lucene/codecs/compressing/TestCompressingStoredFieldsFormat.java?rev=1412849&r1=1412848&r2=1412849&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/codecs/src/test/org/apache/lucene/codecs/compressing/TestCompressingStoredFieldsFormat.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/codecs/src/test/org/apache/lucene/codecs/compressing/TestCompressingStoredFieldsFormat.java Fri Nov 23 12:00:32 2012
@@ -18,9 +18,11 @@ package org.apache.lucene.codecs.compres
*/
import java.io.IOException;
+import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
+import java.util.concurrent.atomic.AtomicReference;
import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.codecs.lucene41.Lucene41Codec;
@@ -38,22 +40,54 @@ import org.apache.lucene.index.IndexWrit
import org.apache.lucene.index.RandomIndexWriter;
import org.apache.lucene.index.StorableField;
import org.apache.lucene.index.StoredDocument;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.NumericRangeQuery;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.TermQuery;
+import org.apache.lucene.search.TopDocs;
import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.MockDirectoryWrapper;
+import org.apache.lucene.store.MockDirectoryWrapper.Throttling;
import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util._TestUtil;
import com.carrotsearch.randomizedtesting.generators.RandomInts;
+import com.carrotsearch.randomizedtesting.generators.RandomPicks;
public class TestCompressingStoredFieldsFormat extends LuceneTestCase {
- public void testWriteReadMerge() throws IOException {
- Directory dir = newDirectory();
- IndexWriterConfig iwConf = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
+ private Directory dir;
+ IndexWriterConfig iwConf;
+ private RandomIndexWriter iw;
+
+ public void setUp() throws Exception {
+ super.setUp();
+ dir = newDirectory();
+ iwConf = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
iwConf.setMaxBufferedDocs(RandomInts.randomIntBetween(random(), 2, 30));
iwConf.setCodec(CompressingCodec.randomInstance(random()));
- RandomIndexWriter iw = new RandomIndexWriter(random(), dir, iwConf);
+ iw = new RandomIndexWriter(random(), dir, iwConf);
+ }
+
+ public void tearDown() throws Exception {
+ super.tearDown();
+ IOUtils.close(iw, dir);
+ iw = null;
+ dir = null;
+ }
+
+ private byte[] randomByteArray(int length, int max) {
+ final byte[] result = new byte[length];
+ for (int i = 0; i < length; ++i) {
+ result[i] = (byte) random().nextInt(max);
+ }
+ return result;
+ }
+
+ public void testWriteReadMerge() throws IOException {
final int docCount = atLeast(200);
final byte[][][] data = new byte [docCount][][];
for (int i = 0; i < docCount; ++i) {
@@ -65,12 +99,8 @@ public class TestCompressingStoredFields
final int length = rarely()
? random().nextInt(1000)
: random().nextInt(10);
- final byte[] arr = new byte[length];
final int max = rarely() ? 256 : 2;
- for (int k = 0; k < length; ++k) {
- arr[k] = (byte) random().nextInt(max);
- }
- data[i][j] = arr;
+ data[i][j] = randomByteArray(length, max);
}
}
@@ -133,21 +163,13 @@ public class TestCompressingStoredFields
iw.deleteAll();
iw.commit();
iw.forceMerge(1);
- iw.close();
- dir.close();
}
public void testReadSkip() throws IOException {
- Directory dir = newDirectory();
- IndexWriterConfig iwConf = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
- iwConf.setMaxBufferedDocs(RandomInts.randomIntBetween(random(), 2, 30));
- iwConf.setCodec(CompressingCodec.randomInstance(random()));
- RandomIndexWriter iw = new RandomIndexWriter(random(), dir, iwConf);
-
FieldType ft = new FieldType();
ft.setStored(true);
ft.freeze();
-
+
final String string = _TestUtil.randomSimpleString(random(), 50);
final byte[] bytes = string.getBytes("UTF-8");
final long l = random().nextBoolean() ? random().nextInt(42) : random().nextLong();
@@ -171,7 +193,7 @@ public class TestCompressingStoredFields
}
iw.w.addDocument(doc);
}
- iw.close();
+ iw.commit();
final DirectoryReader reader = DirectoryReader.open(dir);
final int docID = random().nextInt(100);
@@ -187,7 +209,153 @@ public class TestCompressingStoredFields
}
}
reader.close();
+ }
+
+ public void testEmptyDocs() throws IOException {
+ // make sure that the fact that documents might be empty is not a problem
+ final Document emptyDoc = new Document();
+ final int numDocs = random().nextBoolean() ? 1 : atLeast(1000);
+ for (int i = 0; i < numDocs; ++i) {
+ iw.addDocument(emptyDoc);
+ }
+ iw.commit();
+ final DirectoryReader rd = DirectoryReader.open(dir);
+ for (int i = 0; i < numDocs; ++i) {
+ final StoredDocument doc = rd.document(i);
+ assertNotNull(doc);
+ assertTrue(doc.getFields().isEmpty());
+ }
+ rd.close();
+ }
+
+ public void testConcurrentReads() throws Exception {
+ // make sure the readers are properly cloned
+ final Document doc = new Document();
+ final Field field = new StringField("fld", "", Store.YES);
+ doc.add(field);
+ final int numDocs = atLeast(1000);
+ for (int i = 0; i < numDocs; ++i) {
+ field.setStringValue("" + i);
+ iw.addDocument(doc);
+ }
+ iw.commit();
+
+ final DirectoryReader rd = DirectoryReader.open(dir);
+ final IndexSearcher searcher = new IndexSearcher(rd);
+ final int concurrentReads = atLeast(5);
+ final int readsPerThread = atLeast(50);
+ final List<Thread> readThreads = new ArrayList<Thread>();
+ final AtomicReference<Exception> ex = new AtomicReference<Exception>();
+ for (int i = 0; i < concurrentReads; ++i) {
+ readThreads.add(new Thread() {
+
+ int[] queries;
+
+ {
+ queries = new int[readsPerThread];
+ for (int i = 0; i < queries.length; ++i) {
+ queries[i] = random().nextInt(numDocs);
+ }
+ }
+
+ @Override
+ public void run() {
+ for (int q : queries) {
+ final Query query = new TermQuery(new Term("fld", "" + q));
+ try {
+ final TopDocs topDocs = searcher.search(query, 1);
+ if (topDocs.totalHits != 1) {
+ throw new IllegalStateException("Expected 1 hit, got " + topDocs.totalHits);
+ }
+ final StoredDocument sdoc = rd.document(topDocs.scoreDocs[0].doc);
+ if (sdoc == null || sdoc.get("fld") == null) {
+ throw new IllegalStateException("Could not find document " + q);
+ }
+ if (!Integer.toString(q).equals(sdoc.get("fld"))) {
+ throw new IllegalStateException("Expected " + q + ", but got " + sdoc.get("fld"));
+ }
+ } catch (Exception e) {
+ ex.compareAndSet(null, e);
+ }
+ }
+ }
+ });
+ }
+ for (Thread thread : readThreads) {
+ thread.start();
+ }
+ for (Thread thread : readThreads) {
+ thread.join();
+ }
+ rd.close();
+ if (ex.get() != null) {
+ throw ex.get();
+ }
+ }
+
+ @Nightly
+ public void testBigDocuments() throws IOException {
+ // "big" as "much bigger than the chunk size"
+ // for this test we force a FS dir
+ iw.close();
dir.close();
+ dir = newFSDirectory(_TestUtil.getTempDir(getClass().getSimpleName()));
+ iw = new RandomIndexWriter(random(), dir, iwConf);
+
+ if (dir instanceof MockDirectoryWrapper) {
+ ((MockDirectoryWrapper) dir).setThrottling(Throttling.NEVER);
+ }
+
+ final Document emptyDoc = new Document(); // emptyDoc
+ final Document bigDoc1 = new Document(); // lot of small fields
+ final Document bigDoc2 = new Document(); // 1 very big field
+
+ final Field idField = new StringField("id", "", Store.NO);
+ emptyDoc.add(idField);
+ bigDoc1.add(idField);
+ bigDoc2.add(idField);
+
+ final FieldType onlyStored = new FieldType(StringField.TYPE_STORED);
+ onlyStored.setIndexed(false);
+
+ final Field smallField = new Field("fld", randomByteArray(random().nextInt(10), 256), onlyStored);
+ final int numFields = RandomInts.randomIntBetween(random(), 500000, 1000000);
+ for (int i = 0; i < numFields; ++i) {
+ bigDoc1.add(smallField);
+ }
+
+ final Field bigField = new Field("fld", randomByteArray(RandomInts.randomIntBetween(random(), 1000000, 5000000), 2), onlyStored);
+ bigDoc2.add(bigField);
+
+ final int numDocs = atLeast(5);
+ final Document[] docs = new Document[numDocs];
+ for (int i = 0; i < numDocs; ++i) {
+ docs[i] = RandomPicks.randomFrom(random(), Arrays.asList(emptyDoc, bigDoc1, bigDoc2));
+ }
+ for (int i = 0; i < numDocs; ++i) {
+ idField.setStringValue("" + i);
+ iw.addDocument(docs[i]);
+ if (random().nextInt(numDocs) == 0) {
+ iw.commit();
+ }
+ }
+ iw.commit();
+ iw.forceMerge(1); // look at what happens when big docs are merged
+ final DirectoryReader rd = DirectoryReader.open(dir);
+ final IndexSearcher searcher = new IndexSearcher(rd);
+ for (int i = 0; i < numDocs; ++i) {
+ final Query query = new TermQuery(new Term("id", "" + i));
+ final TopDocs topDocs = searcher.search(query, 1);
+ assertEquals("" + i, 1, topDocs.totalHits);
+ final StoredDocument doc = rd.document(topDocs.scoreDocs[0].doc);
+ assertNotNull(doc);
+ final StorableField[] fieldValues = doc.getFields("fld");
+ assertEquals(docs[i].getFields("fld").length, fieldValues.length);
+ if (fieldValues.length > 0) {
+ assertEquals(docs[i].getFields("fld")[0].binaryValue(), fieldValues[0].binaryValue());
+ }
+ }
+ rd.close();
}
}
Modified: lucene/dev/branches/LUCENE-2878/lucene/common-build.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/common-build.xml?rev=1412849&r1=1412848&r2=1412849&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/common-build.xml (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/common-build.xml Fri Nov 23 12:00:32 2012
@@ -833,7 +833,7 @@
<assertions>
<enable package="org.apache.lucene"/>
<enable package="org.apache.solr"/>
- </assertions>
+ </assertions>
<!-- JVM arguments and system properties. -->
<jvmarg line="${args}"/>
@@ -903,6 +903,7 @@
<propertyref prefix="tests.maxfailures" />
<propertyref prefix="tests.failfast" />
<propertyref prefix="tests.badapples" />
+ <propertyref prefix="tests.timeoutSuite" />
</syspropertyset>
<!-- Pass randomized settings to the forked JVM. -->
@@ -1083,7 +1084,7 @@ ant test -Dtests.dups=N ...
ant -Dtests.nightly=[false] - nightly test group (@Nightly)
ant -Dtests.weekly=[false] - weekly tests (@Weekly)
ant -Dtests.awaitsfix=[false] - known issue (@AwaitsFix)
-ant -Dtests.slow=[false] - slow tests (@Slow)
+ant -Dtests.slow=[true] - slow tests (@Slow)
#
# Load balancing and caches. --------------------------------------
@@ -1119,6 +1120,9 @@ ant -verbose test
# Enabled automatically when running for a single test case.
ant -Dtests.showSuccess=true test
+# Change the default suite timeout to 5 seconds.
+ant -Dtests.timeoutSuite=5000! ...
+
# Display local averaged stats, if any (30 slowest tests).
ant test-times -Dmax=30
Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/analysis/TokenStreamToAutomaton.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/analysis/TokenStreamToAutomaton.java?rev=1412849&r1=1412848&r2=1412849&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/analysis/TokenStreamToAutomaton.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/analysis/TokenStreamToAutomaton.java Fri Nov 23 12:00:32 2012
@@ -22,6 +22,7 @@ import java.io.IOException;
import java.io.OutputStreamWriter;
import java.io.Writer;
+import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
import org.apache.lucene.analysis.tokenattributes.PositionLengthAttribute;
import org.apache.lucene.analysis.tokenattributes.TermToBytesRefAttribute;
@@ -88,6 +89,7 @@ public class TokenStreamToAutomaton {
final TermToBytesRefAttribute termBytesAtt = in.addAttribute(TermToBytesRefAttribute.class);
final PositionIncrementAttribute posIncAtt = in.addAttribute(PositionIncrementAttribute.class);
final PositionLengthAttribute posLengthAtt = in.addAttribute(PositionLengthAttribute.class);
+
final BytesRef term = termBytesAtt.getBytesRef();
in.reset();
Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40TermVectorsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40TermVectorsReader.java?rev=1412849&r1=1412848&r2=1412849&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40TermVectorsReader.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40TermVectorsReader.java Fri Nov 23 12:00:32 2012
@@ -730,9 +730,6 @@ public class Lucene40TermVectorsReader e
@Override
public Fields get(int docID) throws IOException {
- if (docID < 0 || docID >= numTotalDocs) {
- throw new IllegalArgumentException("doID=" + docID + " is out of bounds [0.." + (numTotalDocs-1) + "]");
- }
if (tvx != null) {
Fields fields = new TVFields(docID);
if (fields.size() == 0) {
Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/Bytes.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/Bytes.java?rev=1412849&r1=1412848&r2=1412849&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/Bytes.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/Bytes.java Fri Nov 23 12:00:32 2012
@@ -308,7 +308,7 @@ public final class Bytes {
/**
* Opens all necessary files, but does not read any data in until you call
- * {@link #load}.
+ * {@link #loadSource}.
*/
static abstract class BytesReaderBase extends DocValues {
protected final IndexInput idxIn;
Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/FixedDerefBytesImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/FixedDerefBytesImpl.java?rev=1412849&r1=1412848&r2=1412849&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/FixedDerefBytesImpl.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/FixedDerefBytesImpl.java Fri Nov 23 12:00:32 2012
@@ -79,12 +79,12 @@ class FixedDerefBytesImpl {
}
@Override
- public Source load() throws IOException {
+ protected Source loadSource() throws IOException {
return new FixedDerefSource(cloneData(), cloneIndex(), size, numValuesStored);
}
@Override
- public Source getDirectSource()
+ protected Source loadDirectSource()
throws IOException {
return new DirectFixedDerefSource(cloneData(), cloneIndex(), size, getType());
}
Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/FixedSortedBytesImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/FixedSortedBytesImpl.java?rev=1412849&r1=1412848&r2=1412849&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/FixedSortedBytesImpl.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/FixedSortedBytesImpl.java Fri Nov 23 12:00:32 2012
@@ -135,13 +135,13 @@ class FixedSortedBytesImpl {
}
@Override
- public Source load() throws IOException {
+ protected Source loadSource() throws IOException {
return new FixedSortedSource(cloneData(), cloneIndex(), size, valueCount,
comparator);
}
@Override
- public Source getDirectSource() throws IOException {
+ protected Source loadDirectSource() throws IOException {
return new DirectFixedSortedSource(cloneData(), cloneIndex(), size,
valueCount, comparator, type);
}
Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/FixedStraightBytesImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/FixedStraightBytesImpl.java?rev=1412849&r1=1412848&r2=1412849&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/FixedStraightBytesImpl.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/FixedStraightBytesImpl.java Fri Nov 23 12:00:32 2012
@@ -116,7 +116,7 @@ class FixedStraightBytesImpl {
}
protected void resetPool() {
- pool.dropBuffersAndReset();
+ pool.reset(false, false);
}
protected void writeData(IndexOutput out) throws IOException {
@@ -280,7 +280,7 @@ class FixedStraightBytesImpl {
}
@Override
- public Source load() throws IOException {
+ protected Source loadSource() throws IOException {
return size == 1 ? new SingleByteSource(cloneData(), maxDoc) :
new FixedStraightSource(cloneData(), size, maxDoc, type);
}
@@ -291,7 +291,7 @@ class FixedStraightBytesImpl {
}
@Override
- public Source getDirectSource() throws IOException {
+ protected Source loadDirectSource() throws IOException {
return new DirectFixedStraightSource(cloneData(), size, getType());
}
Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/Floats.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/Floats.java?rev=1412849&r1=1412848&r2=1412849&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/Floats.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/Floats.java Fri Nov 23 12:00:32 2012
@@ -125,7 +125,7 @@ public class Floats {
}
@Override
- public Source load() throws IOException {
+ protected Source loadSource() throws IOException {
final IndexInput indexInput = cloneData();
try {
return arrayTemplate.newFromInput(indexInput, maxDoc);
Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/Ints.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/Ints.java?rev=1412849&r1=1412848&r2=1412849&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/Ints.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/Ints.java Fri Nov 23 12:00:32 2012
@@ -149,7 +149,7 @@ public final class Ints {
}
@Override
- public Source load() throws IOException {
+ protected Source loadSource() throws IOException {
final IndexInput indexInput = cloneData();
try {
return arrayTemplate.newFromInput(indexInput, maxDoc);
Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/PackedIntValues.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/PackedIntValues.java?rev=1412849&r1=1412848&r2=1412849&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/PackedIntValues.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/PackedIntValues.java Fri Nov 23 12:00:32 2012
@@ -149,7 +149,7 @@ class PackedIntValues {
/**
* Opens all necessary files, but does not read any data in until you call
- * {@link #load}.
+ * {@link #loadSource}.
*/
static class PackedIntsReader extends DocValues {
private final IndexInput datIn;
@@ -182,7 +182,7 @@ class PackedIntValues {
* already previously loaded but then discarded the Source.
*/
@Override
- public Source load() throws IOException {
+ protected Source loadSource() throws IOException {
boolean success = false;
final Source source;
IndexInput input = null;
@@ -217,7 +217,7 @@ class PackedIntValues {
@Override
- public Source getDirectSource() throws IOException {
+ protected Source loadDirectSource() throws IOException {
return values != null ? new FixedStraightBytesImpl.DirectFixedStraightSource(datIn.clone(), 8, Type.FIXED_INTS_64) : new PackedIntsSource(datIn.clone(), true);
}
}
Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/VarDerefBytesImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/VarDerefBytesImpl.java?rev=1412849&r1=1412848&r2=1412849&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/VarDerefBytesImpl.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/VarDerefBytesImpl.java Fri Nov 23 12:00:32 2012
@@ -99,12 +99,12 @@ class VarDerefBytesImpl {
}
@Override
- public Source load() throws IOException {
+ protected Source loadSource() throws IOException {
return new VarDerefSource(cloneData(), cloneIndex(), totalBytes);
}
@Override
- public Source getDirectSource()
+ protected Source loadDirectSource()
throws IOException {
return new DirectVarDerefSource(cloneData(), cloneIndex(), getType());
}
Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/VarSortedBytesImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/VarSortedBytesImpl.java?rev=1412849&r1=1412848&r2=1412849&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/VarSortedBytesImpl.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/VarSortedBytesImpl.java Fri Nov 23 12:00:32 2012
@@ -161,13 +161,13 @@ final class VarSortedBytesImpl {
}
@Override
- public org.apache.lucene.index.DocValues.Source load()
+ public org.apache.lucene.index.DocValues.Source loadSource()
throws IOException {
return new VarSortedSource(cloneData(), cloneIndex(), comparator);
}
@Override
- public Source getDirectSource() throws IOException {
+ protected Source loadDirectSource() throws IOException {
return new DirectSortedSource(cloneData(), cloneIndex(), comparator, getType());
}
Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/VarStraightBytesImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/VarStraightBytesImpl.java?rev=1412849&r1=1412848&r2=1412849&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/VarStraightBytesImpl.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/VarStraightBytesImpl.java Fri Nov 23 12:00:32 2012
@@ -134,7 +134,6 @@ class VarStraightBytesImpl {
docToAddress[lastDocID] = address + offset;
}
address += numDataBytes; // this is the address after all addr pointers are updated
- iter.close();
} finally {
IOUtils.close(cloneIdx);
}
@@ -188,7 +187,7 @@ class VarStraightBytesImpl {
} else {
IOUtils.closeWhileHandlingException(datOut);
}
- pool.dropBuffersAndReset();
+ pool.reset(false, false);
}
success = false;
@@ -247,12 +246,12 @@ class VarStraightBytesImpl {
}
@Override
- public Source load() throws IOException {
+ protected Source loadSource() throws IOException {
return new VarStraightSource(cloneData(), cloneIndex());
}
@Override
- public Source getDirectSource()
+ protected Source loadDirectSource()
throws IOException {
return new DirectVarStraightSource(cloneData(), cloneIndex(), getType());
}
Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene41/ForUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene41/ForUtil.java?rev=1412849&r1=1412848&r2=1412849&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene41/ForUtil.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene41/ForUtil.java Fri Nov 23 12:00:32 2012
@@ -19,7 +19,6 @@ package org.apache.lucene.codecs.lucene4
import java.io.IOException;
import java.util.Arrays;
-import org.apache.lucene.index.CorruptIndexException;
import org.apache.lucene.store.DataInput;
import org.apache.lucene.store.DataOutput;
import org.apache.lucene.store.IndexInput;
@@ -83,8 +82,10 @@ final class ForUtil {
* Compute the number of bytes required to encode a block of values that require
* <code>bitsPerValue</code> bits per value with format <code>format</code>.
*/
- private static int encodedSize(PackedInts.Format format, int bitsPerValue) {
- return format.nblocks(bitsPerValue, BLOCK_SIZE) << 3;
+ private static int encodedSize(PackedInts.Format format, int packedIntsVersion, int bitsPerValue) {
+ final long byteCount = format.byteCount(packedIntsVersion, BLOCK_SIZE, bitsPerValue);
+ assert byteCount >= 0 && byteCount <= Integer.MAX_VALUE : byteCount;
+ return (int) byteCount;
}
private final int[] encodedSizes;
@@ -107,7 +108,7 @@ final class ForUtil {
BLOCK_SIZE, bpv, acceptableOverheadRatio);
assert formatAndBits.format.isSupported(formatAndBits.bitsPerValue);
assert formatAndBits.bitsPerValue <= 32;
- encodedSizes[bpv] = encodedSize(formatAndBits.format, formatAndBits.bitsPerValue);
+ encodedSizes[bpv] = encodedSize(formatAndBits.format, PackedInts.VERSION_CURRENT, formatAndBits.bitsPerValue);
encoders[bpv] = PackedInts.getEncoder(
formatAndBits.format, PackedInts.VERSION_CURRENT, formatAndBits.bitsPerValue);
decoders[bpv] = PackedInts.getDecoder(
@@ -123,9 +124,7 @@ final class ForUtil {
*/
ForUtil(DataInput in) throws IOException {
int packedIntsVersion = in.readVInt();
- if (packedIntsVersion != PackedInts.VERSION_START) {
- throw new CorruptIndexException("expected version=" + PackedInts.VERSION_START + " but got version=" + packedIntsVersion);
- }
+ PackedInts.checkVersion(packedIntsVersion);
encodedSizes = new int[33];
encoders = new PackedInts.Encoder[33];
decoders = new PackedInts.Decoder[33];
@@ -138,7 +137,7 @@ final class ForUtil {
final PackedInts.Format format = PackedInts.Format.byId(formatId);
assert format.isSupported(bitsPerValue);
- encodedSizes[bpv] = encodedSize(format, bitsPerValue);
+ encodedSizes[bpv] = encodedSize(format, packedIntsVersion, bitsPerValue);
encoders[bpv] = PackedInts.getEncoder(
format, packedIntsVersion, bitsPerValue);
decoders[bpv] = PackedInts.getDecoder(
Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41Codec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41Codec.java?rev=1412849&r1=1412848&r2=1412849&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41Codec.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41Codec.java Fri Nov 23 12:00:32 2012
@@ -32,7 +32,6 @@ import org.apache.lucene.codecs.lucene40
import org.apache.lucene.codecs.lucene40.Lucene40LiveDocsFormat;
import org.apache.lucene.codecs.lucene40.Lucene40NormsFormat;
import org.apache.lucene.codecs.lucene40.Lucene40SegmentInfoFormat;
-import org.apache.lucene.codecs.lucene40.Lucene40StoredFieldsFormat;
import org.apache.lucene.codecs.lucene40.Lucene40TermVectorsFormat;
import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
@@ -49,7 +48,7 @@ import org.apache.lucene.codecs.perfield
// if they are backwards compatible or smallish we can probably do the backwards in the postingsreader
// (it writes a minor version, etc).
public class Lucene41Codec extends Codec {
- private final StoredFieldsFormat fieldsFormat = new Lucene40StoredFieldsFormat();
+ private final StoredFieldsFormat fieldsFormat = new Lucene41StoredFieldsFormat();
private final TermVectorsFormat vectorsFormat = new Lucene40TermVectorsFormat();
private final FieldInfosFormat fieldInfosFormat = new Lucene40FieldInfosFormat();
private final DocValuesFormat docValuesFormat = new Lucene40DocValuesFormat();
Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/document/Document.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/document/Document.java?rev=1412849&r1=1412848&r2=1412849&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/document/Document.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/document/Document.java Fri Nov 23 12:00:32 2012
@@ -138,9 +138,8 @@ public final class Document implements I
*/
public final BytesRef[] getBinaryValues(String name) {
final List<BytesRef> result = new ArrayList<BytesRef>();
- Iterator<Field> it = storedFieldsIterator();
-
- while (it.hasNext()) {
+
+ for (Iterator<StorableField> it = storedFieldsIterator(); it.hasNext(); ) {
StorableField field = it.next();
if (field.name().equals(name)) {
final BytesRef bytes = field.binaryValue();
@@ -163,9 +162,7 @@ public final class Document implements I
* @return a <code>BytesRef</code> containing the binary field value or <code>null</code>
*/
public final BytesRef getBinaryValue(String name) {
- Iterator<Field> it = storedFieldsIterator();
-
- while (it.hasNext()) {
+ for (Iterator<StorableField> it = storedFieldsIterator(); it.hasNext(); ) {
StorableField field = it.next();
if (field.name().equals(name)) {
final BytesRef bytes = field.binaryValue();
@@ -235,9 +232,8 @@ public final class Document implements I
*/
public final String[] getValues(String name) {
List<String> result = new ArrayList<String>();
- Iterator<Field> it = storedFieldsIterator();
-
- while (it.hasNext()) {
+
+ for (Iterator<StorableField> it = storedFieldsIterator(); it.hasNext(); ) {
StorableField field = it.next();
if (field.name().equals(name) && field.stringValue() != null) {
result.add(field.stringValue());
@@ -260,9 +256,7 @@ public final class Document implements I
* the actual numeric field instance back, use {@link #getField}.
*/
public final String get(String name) {
- Iterator<Field> it = storedFieldsIterator();
-
- while (it.hasNext()) {
+ for (Iterator<StorableField> it = storedFieldsIterator(); it.hasNext(); ) {
StorableField field = it.next();
if (field.name().equals(name) && field.stringValue() != null) {
return field.stringValue();
@@ -288,29 +282,28 @@ public final class Document implements I
/** Obtains all indexed fields in document */
@Override
- public Iterable<? extends IndexableField> indexableFields() {
- return new Iterable<Field>() {
+ public Iterable<IndexableField> indexableFields() {
+ return new Iterable<IndexableField>() {
@Override
- public Iterator<Field> iterator() {
+ public Iterator<IndexableField> iterator() {
return Document.this.indexedFieldsIterator();
}
};
}
-
/** Obtains all stored fields in document. */
@Override
- public Iterable<? extends StorableField> storableFields() {
- return new Iterable<Field>() {
+ public Iterable<StorableField> storableFields() {
+ return new Iterable<StorableField>() {
@Override
- public Iterator<Field> iterator() {
+ public Iterator<StorableField> iterator() {
return Document.this.storedFieldsIterator();
}
};
}
- private Iterator<Field> storedFieldsIterator() {
- return new FilterIterator<Field>(fields.iterator()) {
+ private Iterator<StorableField> storedFieldsIterator() {
+ return new FilterIterator<StorableField, Field>(fields.iterator()) {
@Override
protected boolean predicateFunction(Field field) {
return field.type.stored() || field.type.docValueType() != null;
@@ -318,8 +311,8 @@ public final class Document implements I
};
}
- private Iterator<Field> indexedFieldsIterator() {
- return new FilterIterator<Field>(fields.iterator()) {
+ private Iterator<IndexableField> indexedFieldsIterator() {
+ return new FilterIterator<IndexableField, Field>(fields.iterator()) {
@Override
protected boolean predicateFunction(Field field) {
return field.type.indexed();
Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/ByteSliceReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/ByteSliceReader.java?rev=1412849&r1=1412848&r2=1412849&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/ByteSliceReader.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/ByteSliceReader.java Fri Nov 23 12:00:32 2012
@@ -54,7 +54,7 @@ final class ByteSliceReader extends Data
buffer = pool.buffers[bufferUpto];
upto = startIndex & ByteBlockPool.BYTE_BLOCK_MASK;
- final int firstSize = ByteBlockPool.levelSizeArray[0];
+ final int firstSize = ByteBlockPool.LEVEL_SIZE_ARRAY[0];
if (startIndex+firstSize >= endIndex) {
// There is only this one slice to read
@@ -100,8 +100,8 @@ final class ByteSliceReader extends Data
// Skip to our next slice
final int nextIndex = ((buffer[limit]&0xff)<<24) + ((buffer[1+limit]&0xff)<<16) + ((buffer[2+limit]&0xff)<<8) + (buffer[3+limit]&0xff);
- level = ByteBlockPool.nextLevelArray[level];
- final int newSize = ByteBlockPool.levelSizeArray[level];
+ level = ByteBlockPool.NEXT_LEVEL_ARRAY[level];
+ final int newSize = ByteBlockPool.LEVEL_SIZE_ARRAY[level];
bufferUpto = nextIndex / ByteBlockPool.BYTE_BLOCK_SIZE;
bufferOffset = bufferUpto * ByteBlockPool.BYTE_BLOCK_SIZE;
Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/ConcurrentMergeScheduler.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/ConcurrentMergeScheduler.java?rev=1412849&r1=1412848&r2=1412849&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/ConcurrentMergeScheduler.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/ConcurrentMergeScheduler.java Fri Nov 23 12:00:32 2012
@@ -302,7 +302,7 @@ public class ConcurrentMergeScheduler ex
}
@Override
- public void merge(IndexWriter writer) throws IOException {
+ public synchronized void merge(IndexWriter writer) throws IOException {
assert !Thread.holdsLock(writer);
@@ -328,31 +328,34 @@ public class ConcurrentMergeScheduler ex
// pending merges, until it's empty:
while (true) {
- synchronized(this) {
- long startStallTime = 0;
- while (mergeThreadCount() >= 1+maxMergeCount) {
- startStallTime = System.currentTimeMillis();
- if (verbose()) {
- message(" too many merges; stalling...");
- }
- try {
- wait();
- } catch (InterruptedException ie) {
- throw new ThreadInterruptedException(ie);
- }
- }
-
+ long startStallTime = 0;
+ while (writer.hasPendingMerges() && mergeThreadCount() >= maxMergeCount) {
+ // This means merging has fallen too far behind: we
+ // have already created maxMergeCount threads, and
+ // now there's at least one more merge pending.
+ // Note that only maxThreadCount of
+ // those created merge threads will actually be
+ // running; the rest will be paused (see
+ // updateMergeThreads). We stall this producer
+ // thread to prevent creation of new segments,
+ // until merging has caught up:
+ startStallTime = System.currentTimeMillis();
if (verbose()) {
- if (startStallTime != 0) {
- message(" stalled for " + (System.currentTimeMillis()-startStallTime) + " msec");
- }
+ message(" too many merges; stalling...");
+ }
+ try {
+ wait();
+ } catch (InterruptedException ie) {
+ throw new ThreadInterruptedException(ie);
}
}
+ if (verbose()) {
+ if (startStallTime != 0) {
+ message(" stalled for " + (System.currentTimeMillis()-startStallTime) + " msec");
+ }
+ }
- // TODO: we could be careful about which merges to do in
- // the BG (eg maybe the "biggest" ones) vs FG, which
- // merges to do first (the easiest ones?), etc.
MergePolicy.OneMerge merge = writer.getNextMerge();
if (merge == null) {
if (verbose()) {
@@ -361,34 +364,28 @@ public class ConcurrentMergeScheduler ex
return;
}
- // We do this w/ the primary thread to keep
- // deterministic assignment of segment names
- writer.mergeInit(merge);
-
boolean success = false;
try {
- synchronized(this) {
- if (verbose()) {
- message(" consider merge " + writer.segString(merge.segments));
- }
+ if (verbose()) {
+ message(" consider merge " + writer.segString(merge.segments));
+ }
- // OK to spawn a new merge thread to handle this
- // merge:
- final MergeThread merger = getMergeThread(writer, merge);
- mergeThreads.add(merger);
- if (verbose()) {
- message(" launch new thread [" + merger.getName() + "]");
- }
+ // OK to spawn a new merge thread to handle this
+ // merge:
+ final MergeThread merger = getMergeThread(writer, merge);
+ mergeThreads.add(merger);
+ if (verbose()) {
+ message(" launch new thread [" + merger.getName() + "]");
+ }
- merger.start();
+ merger.start();
- // Must call this after starting the thread else
- // the new thread is removed from mergeThreads
- // (since it's not alive yet):
- updateMergeThreads();
+ // Must call this after starting the thread else
+ // the new thread is removed from mergeThreads
+ // (since it's not alive yet):
+ updateMergeThreads();
- success = true;
- }
+ success = true;
} finally {
if (!success) {
writer.mergeFinish(merge);
@@ -482,7 +479,6 @@ public class ConcurrentMergeScheduler ex
// merge that writer says is necessary:
merge = tWriter.getNextMerge();
if (merge != null) {
- tWriter.mergeInit(merge);
updateMergeThreads();
if (verbose()) {
message(" merge thread: do another merge " + tWriter.segString(merge.segments));
@@ -546,4 +542,13 @@ public class ConcurrentMergeScheduler ex
void clearSuppressExceptions() {
suppressExceptions = false;
}
+
+ @Override
+ public String toString() {
+ StringBuilder sb = new StringBuilder(getClass().getSimpleName() + ": ");
+ sb.append("maxThreadCount=").append(maxThreadCount).append(", ");
+ sb.append("maxMergeCount=").append(maxMergeCount).append(", ");
+ sb.append("mergeThreadPriority=").append(mergeThreadPriority);
+ return sb.toString();
+ }
}
Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/DocFieldProcessor.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/DocFieldProcessor.java?rev=1412849&r1=1412848&r2=1412849&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/DocFieldProcessor.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/DocFieldProcessor.java Fri Nov 23 12:00:32 2012
@@ -59,7 +59,6 @@ final class DocFieldProcessor extends Do
int hashMask = 1;
int totalFieldCount;
- float docBoost;
int fieldGen;
final DocumentsWriterPerThread.DocState docState;