blob: 73dda609946827850583ef786479337bc2e233d3 [file] [log] [blame]
Index: common-build.xml
===================================================================
--- common-build.xml (revision 664674)
+++ common-build.xml (working copy)
@@ -339,8 +339,9 @@
<target name="test" depends="compile-test" description="Runs unit tests">
<mkdir dir="${junit.output.dir}"/>
+ <!-- nocommit maxmemory below -->
<junit printsummary="off" haltonfailure="no"
- errorProperty="tests.failed" failureProperty="tests.failed">
+ errorProperty="tests.failed" failureProperty="tests.failed" maxmemory="1024M">
<classpath refid="junit.classpath"/>
<assertions>
<enable package="org.apache.lucene"/>
Index: src/test/org/apache/lucene/TestSnapshotDeletionPolicy.java
===================================================================
--- src/test/org/apache/lucene/TestSnapshotDeletionPolicy.java (revision 664673)
+++ src/test/org/apache/lucene/TestSnapshotDeletionPolicy.java (working copy)
@@ -121,9 +121,8 @@
try {
writer.addDocument(doc);
} catch (Throwable t) {
- RuntimeException re = new RuntimeException("addDocument failed");
- re.initCause(t);
- throw re;
+ t.printStackTrace(System.out);
+ fail("addDocument failed");
}
}
try {
Index: src/test/org/apache/lucene/TestDemo.java
===================================================================
--- src/test/org/apache/lucene/TestDemo.java (revision 664673)
+++ src/test/org/apache/lucene/TestDemo.java (working copy)
@@ -32,6 +32,7 @@
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.RAMDirectory;
import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util._TestUtil;
/**
* A very simple demo used in the API documentation (src/java/overview.html).
@@ -58,6 +59,8 @@
Field.Index.TOKENIZED));
iwriter.addDocument(doc);
iwriter.close();
+
+ _TestUtil.checkIndex(directory);
// Now search the index:
IndexSearcher isearcher = new IndexSearcher(directory);
Index: src/test/org/apache/lucene/search/TestSort.java
===================================================================
--- src/test/org/apache/lucene/search/TestSort.java (revision 664673)
+++ src/test/org/apache/lucene/search/TestSort.java (working copy)
@@ -648,34 +648,35 @@
// make sure the documents returned by the search match the expected list
private void assertMatches (Searcher searcher, Query query, Sort sort, String expectedResult)
throws IOException {
- ScoreDoc[] result = searcher.search (query, null, 1000, sort).scoreDocs;
- StringBuffer buff = new StringBuffer(10);
- int n = result.length;
- for (int i=0; i<n; ++i) {
- Document doc = searcher.doc(result[i].doc);
- String[] v = doc.getValues("tracer");
- for (int j=0; j<v.length; ++j) {
- buff.append (v[j]);
- }
- }
- assertEquals (expectedResult, buff.toString());
+ ScoreDoc[] result = searcher.search (query, null, 1000, sort).scoreDocs;
+ StringBuffer buff = new StringBuffer(10);
+ int n = result.length;
+ for (int i=0; i<n; ++i) {
+ Document doc = searcher.doc(result[i].doc);
+
+ String[] v = doc.getValues("tracer");
+ for (int j=0; j<v.length; ++j) {
+ buff.append (v[j]);
+ }
+ }
+ assertEquals (expectedResult, buff.toString());
}
// make sure the documents returned by the search match the expected list pattern
private void assertMatchesPattern (Searcher searcher, Query query, Sort sort, String pattern)
throws IOException {
- ScoreDoc[] result = searcher.search (query, null, 1000, sort).scoreDocs;
- StringBuffer buff = new StringBuffer(10);
- int n = result.length;
- for (int i=0; i<n; ++i) {
- Document doc = searcher.doc(result[i].doc);
- String[] v = doc.getValues("tracer");
- for (int j=0; j<v.length; ++j) {
- buff.append (v[j]);
- }
- }
- // System.out.println ("matching \""+buff+"\" against pattern \""+pattern+"\"");
- assertTrue (Pattern.compile(pattern).matcher(buff.toString()).matches());
+ ScoreDoc[] result = searcher.search (query, null, 1000, sort).scoreDocs;
+ StringBuffer buff = new StringBuffer(10);
+ int n = result.length;
+ for (int i=0; i<n; ++i) {
+ Document doc = searcher.doc(result[i].doc);
+ String[] v = doc.getValues("tracer");
+ for (int j=0; j<v.length; ++j) {
+ buff.append (v[j]);
+ }
+ }
+ // System.out.println ("matching \""+buff+"\" against pattern \""+pattern+"\"");
+ assertTrue (Pattern.compile(pattern).matcher(buff.toString()).matches());
}
private HashMap getScores (ScoreDoc[] hits, Searcher searcher)
Index: src/test/org/apache/lucene/search/TestTermVectors.java
===================================================================
--- src/test/org/apache/lucene/search/TestTermVectors.java (revision 664673)
+++ src/test/org/apache/lucene/search/TestTermVectors.java (working copy)
@@ -93,7 +93,7 @@
public void testTermVectorsFieldOrder() throws IOException {
Directory dir = new MockRAMDirectory();
- IndexWriter writer = new IndexWriter(dir, new SimpleAnalyzer(), true);
+ IndexWriter writer = new IndexWriter(dir, new SimpleAnalyzer(), true, IndexWriter.MaxFieldLength.LIMITED);
Document doc = new Document();
doc.add(new Field("c", "some content here", Field.Store.YES, Field.Index.TOKENIZED, Field.TermVector.WITH_POSITIONS_OFFSETS));
doc.add(new Field("a", "some content here", Field.Store.YES, Field.Index.TOKENIZED, Field.TermVector.WITH_POSITIONS_OFFSETS));
Index: src/test/org/apache/lucene/index/TestStressIndexing2.java
===================================================================
--- src/test/org/apache/lucene/index/TestStressIndexing2.java (revision 664673)
+++ src/test/org/apache/lucene/index/TestStressIndexing2.java (working copy)
@@ -19,11 +19,10 @@
import org.apache.lucene.analysis.*;
import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util._TestUtil;
import org.apache.lucene.search.TermQuery;
import java.util.*;
-import java.io.ByteArrayOutputStream;
-import java.io.PrintStream;
import java.io.IOException;
import junit.framework.TestCase;
@@ -53,7 +52,7 @@
}
}
- public void testRandom() throws Exception {
+ public void testRandom() throws Throwable {
Directory dir1 = new MockRAMDirectory();
// dir1 = FSDirectory.getDirectory("foofoofoo");
Directory dir2 = new MockRAMDirectory();
@@ -68,17 +67,7 @@
verifyEquals(dir1, dir2, "id");
}
- private void checkIndex(Directory dir) throws IOException {
- ByteArrayOutputStream bos = new ByteArrayOutputStream(1024);
- CheckIndex.out = new PrintStream(bos);
- if (!CheckIndex.check(dir, false, null)) {
- System.out.println("CheckIndex failed");
- System.out.println(bos.toString());
- fail("CheckIndex failed");
- }
- }
-
- public void testMultiConfig() throws Exception {
+ public void testMultiConfig() throws Throwable {
// test lots of smaller different params together
for (int i=0; i<100; i++) { // increase iterations for better testing
sameFieldOrder=r.nextBoolean();
@@ -157,7 +146,7 @@
}
}
- checkIndex(dir);
+ _TestUtil.checkIndex(dir);
return docs;
}
@@ -187,7 +176,7 @@
w.close();
}
- public static void verifyEquals(Directory dir1, Directory dir2, String idField) throws IOException {
+ public static void verifyEquals(Directory dir1, Directory dir2, String idField) throws Throwable {
IndexReader r1 = IndexReader.open(dir1);
IndexReader r2 = IndexReader.open(dir2);
verifyEquals(r1, r2, idField);
@@ -196,7 +185,7 @@
}
- public static void verifyEquals(IndexReader r1, IndexReader r2, String idField) throws IOException {
+ public static void verifyEquals(IndexReader r1, IndexReader r2, String idField) throws Throwable {
assertEquals(r1.numDocs(), r2.numDocs());
boolean hasDeletes = !(r1.maxDoc()==r2.maxDoc() && r1.numDocs()==r1.maxDoc());
@@ -236,7 +225,7 @@
try {
// verify term vectors are equivalent
verifyEquals(r1.getTermFreqVectors(id1), r2.getTermFreqVectors(id2));
- } catch (java.lang.Error e) {
+ } catch (Throwable e) {
System.out.println("FAILED id=" + term + " id1=" + id1 + " id2=" + id2);
TermFreqVector[] tv1 = r1.getTermFreqVectors(id1);
System.out.println(" d1=" + tv1);
@@ -367,6 +356,8 @@
for(int i=0;i<d1.length;i++) {
TermFreqVector v1 = d1[i];
TermFreqVector v2 = d2[i];
+ if (v1 == null || v2 == null)
+ System.out.println("v1=" + v1 + " v2=" + v2 + " i=" + i + " of " + d1.length);
assertEquals(v1.size(), v2.size());
int numTerms = v1.size();
String[] terms1 = v1.getTerms();
@@ -572,7 +563,7 @@
indexDoc();
}
}
- } catch (Exception e) {
+ } catch (Throwable e) {
e.printStackTrace();
TestCase.fail(e.toString());
}
Index: src/test/org/apache/lucene/index/TestIndexWriter.java
===================================================================
--- src/test/org/apache/lucene/index/TestIndexWriter.java (revision 664673)
+++ src/test/org/apache/lucene/index/TestIndexWriter.java (working copy)
@@ -1584,6 +1584,7 @@
writer.flush();
writer.addDocument(new Document());
writer.close();
+ _TestUtil.checkIndex(dir);
IndexReader reader = IndexReader.open(dir);
assertEquals(2, reader.numDocs());
}
@@ -1849,7 +1850,7 @@
boolean sawAppend = false;
boolean sawFlush = false;
for (int i = 0; i < trace.length; i++) {
- if ("org.apache.lucene.index.DocumentsWriter".equals(trace[i].getClassName()) && "appendPostings".equals(trace[i].getMethodName()))
+ if ("org.apache.lucene.index.FreqProxTermsWriter".equals(trace[i].getClassName()) && "appendPostings".equals(trace[i].getMethodName()))
sawAppend = true;
if ("doFlush".equals(trace[i].getMethodName()))
sawFlush = true;
@@ -2284,9 +2285,12 @@
while(System.currentTimeMillis() < stopTime) {
try {
+ //System.out.println(Thread.currentThread().getName() + ": now updateDocument");
writer.updateDocument(new Term("id", ""+(idUpto++)), doc);
+ //System.out.println(Thread.currentThread().getName() + ": updateDocument success!");
addCount++;
} catch (IOException ioe) {
+ //System.out.println(Thread.currentThread().getName() + ": hit exc");
//ioe.printStackTrace(System.out);
if (ioe.getMessage().startsWith("fake disk full at") ||
ioe.getMessage().equals("now failing on purpose")) {
@@ -2404,6 +2408,7 @@
writer.addDocument(doc);
fail("did not hit disk full");
} catch (IOException ioe) {
+
}
// Without fix for LUCENE-1130: this call will hang:
try {
@@ -2484,6 +2489,8 @@
"flushDocument".equals(trace[i].getMethodName())) {
if (onlyOnce)
doFail = false;
+ //System.out.println(Thread.currentThread().getName() + ": now fail");
+ //new Throwable().printStackTrace(System.out);
throw new IOException("now failing on purpose");
}
}
@@ -2663,10 +2670,9 @@
if (doFail) {
StackTraceElement[] trace = new Exception().getStackTrace();
for (int i = 0; i < trace.length; i++) {
- if ("writeSegment".equals(trace[i].getMethodName())) {
+ if ("flush".equals(trace[i].getMethodName()) && "org.apache.lucene.index.DocFieldProcessor".equals(trace[i].getClassName())) {
if (onlyOnce)
doFail = false;
- // new RuntimeException().printStackTrace(System.out);
throw new IOException("now failing on purpose");
}
}
Index: src/test/org/apache/lucene/index/TestIndexWriterExceptions.java
===================================================================
--- src/test/org/apache/lucene/index/TestIndexWriterExceptions.java (revision 0)
+++ src/test/org/apache/lucene/index/TestIndexWriterExceptions.java (revision 0)
@@ -0,0 +1,137 @@
+package org.apache.lucene.index;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.util.Random;
+import java.io.IOException;
+
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util._TestUtil;
+import org.apache.lucene.store.MockRAMDirectory;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.analysis.WhitespaceAnalyzer;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+
+// nocommit -- add threaded test
+
+public class TestIndexWriterExceptions extends LuceneTestCase {
+
+ public class MockIndexWriter extends IndexWriter {
+ Random r = new java.util.Random(17);
+
+ public MockIndexWriter(Directory dir, Analyzer a, boolean create, MaxFieldLength mfl) throws IOException {
+ super(dir, false, a, create, mfl);
+ }
+
+ boolean doFail;
+
+ boolean testPoint(String name) {
+ if (doFail && !name.equals("startDoFlush") && r.nextInt(20) == 17) {
+ //System.out.println("NOW FAIL: " + name);
+ //new Throwable().printStackTrace(System.out);
+ throw new RuntimeException("intentionally failing at " + name);
+ }
+ return true;
+ }
+ }
+
+ public void testRandomExceptions() throws Throwable {
+ MockRAMDirectory dir = new MockRAMDirectory();
+
+ final Document doc = new Document();
+
+ final boolean DEBUG = false;
+
+ doc.add(new Field("content1", "aaa bbb ccc ddd", Field.Store.YES, Field.Index.TOKENIZED));
+ doc.add(new Field("content6", "aaa bbb ccc ddd", Field.Store.NO, Field.Index.TOKENIZED, Field.TermVector.WITH_POSITIONS_OFFSETS));
+ doc.add(new Field("content2", "aaa bbb ccc ddd", Field.Store.YES, Field.Index.UN_TOKENIZED));
+ doc.add(new Field("content3", "aaa bbb ccc ddd", Field.Store.YES, Field.Index.NO));
+
+ doc.add(new Field("content4", "aaa bbb ccc ddd", Field.Store.NO, Field.Index.TOKENIZED));
+ doc.add(new Field("content5", "aaa bbb ccc ddd", Field.Store.NO, Field.Index.UN_TOKENIZED));
+
+ doc.add(new Field("content7", "aaa bbb ccc ddd", Field.Store.NO, Field.Index.UN_TOKENIZED, Field.TermVector.WITH_POSITIONS_OFFSETS));
+
+ final Field idField = new Field("id", "", Field.Store.YES, Field.Index.UN_TOKENIZED);
+ doc.add(idField);
+
+ MockIndexWriter writer = new MockIndexWriter(dir, new WhitespaceAnalyzer(), true, IndexWriter.MaxFieldLength.LIMITED);
+ ((ConcurrentMergeScheduler) writer.getMergeScheduler()).setSuppressExceptions();
+ writer.setMaxBufferedDocs(10);
+
+ final Random r = new java.util.Random(47);
+
+ if (DEBUG)
+ writer.setInfoStream(System.out);
+
+ int numAdded = 0;
+
+ for(int i=0;i<3000;i++) {
+ if (DEBUG)
+ System.out.println("TEST i=" + i);
+ writer.doFail = true;
+ numAdded++;
+ try {
+ final String id = ""+r.nextInt(50);
+ idField.setValue(id);
+ writer.updateDocument(new Term("id", id), doc);
+ } catch (RuntimeException re) {
+ if (DEBUG) {
+ System.out.println("EXC: ");
+ re.printStackTrace(System.out);
+ }
+ _TestUtil.checkIndex(dir);
+ numAdded--;
+ }
+
+ writer.doFail = false;
+
+ // After a possible exception (above) I should be able
+ // to add a new document without hitting an exception:
+ if (DEBUG)
+ System.out.println("TEST good i=" + i);
+ writer.addDocument(doc);
+ numAdded++;
+
+ // TODO: assert that writer is "working" in this state
+ }
+
+ writer.commit();
+
+ // TODO: can't do this because we don't know which
+ // exceptions were aborting exceptions
+ // assertEquals(numAdded, writer.numDocs());
+
+ try {
+ writer.close();
+ } catch (Throwable t) {
+ writer.rollback();
+ }
+
+ // Confirm that when doc hits exception partway through tokenization, it's deleted:
+ IndexReader r2 = IndexReader.open(dir);
+ final int count = r2.docFreq(new Term("content4", "aaa"));
+ final int count2 = r2.docFreq(new Term("content4", "ddd"));
+ assertEquals(count, count2);
+ r2.close();
+
+ _TestUtil.checkIndex(dir);
+ }
+}
Property changes on: src/test/org/apache/lucene/index/TestIndexWriterExceptions.java
___________________________________________________________________
Name: svn:eol-style
+ native
Index: src/test/org/apache/lucene/index/TestDocumentWriter.java
===================================================================
--- src/test/org/apache/lucene/index/TestDocumentWriter.java (revision 664673)
+++ src/test/org/apache/lucene/index/TestDocumentWriter.java (working copy)
@@ -27,6 +27,7 @@
import org.apache.lucene.document.Fieldable;
import org.apache.lucene.store.RAMDirectory;
import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util._TestUtil;
import java.io.IOException;
import java.io.Reader;
@@ -250,6 +251,8 @@
writer.addDocument(doc);
writer.close();
+ _TestUtil.checkIndex(ram);
+
IndexReader reader = IndexReader.open(ram);
// f1
TermFreqVector tfv1 = reader.getTermFreqVector(0, "f1");
Index: src/java/org/apache/lucene/index/DocInverterPerField.java
===================================================================
--- src/java/org/apache/lucene/index/DocInverterPerField.java (revision 0)
+++ src/java/org/apache/lucene/index/DocInverterPerField.java (revision 0)
@@ -0,0 +1,190 @@
+package org.apache.lucene.index;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+import java.io.Reader;
+import org.apache.lucene.document.Fieldable;
+import org.apache.lucene.analysis.Token;
+import org.apache.lucene.analysis.TokenStream;
+
+/**
+ * Holds state for inverting all occurrences of a single
+ * field in the document. This class doesn't do anything
+ * itself; instead, it forwards the tokens produced by
+ * analysis to its own consumer
+ * (InvertedDocConsumerPerField). It also interacts with an
+ * endConsumer (InvertedDocEndConsumerPerField).
+ */
+
+class DocInverterPerField extends DocFieldConsumerPerField {
+
+ final private DocInverterPerThread perThread;
+ final private FieldInfo fieldInfo;
+ final InvertedDocConsumerPerField consumer;
+ final InvertedDocEndConsumerPerField endConsumer;
+ final DocumentsWriter.DocState docState;
+ final DocInverter.FieldInvertState fieldState;
+
+ public DocInverterPerField(DocInverterPerThread perThread, FieldInfo fieldInfo) {
+ this.perThread = perThread;
+ this.fieldInfo = fieldInfo;
+ docState = perThread.docState;
+ fieldState = perThread.fieldState;
+ this.consumer = perThread.consumer.addField(this, fieldInfo);
+ this.endConsumer = perThread.endConsumer.addField(this, fieldInfo);
+ }
+
+ void abort() {
+ consumer.abort();
+ endConsumer.abort();
+ }
+
+ public void processFields(final Fieldable[] fields,
+ final int count) throws IOException {
+ processFieldsInner(fields, count);
+ consumer.finish();
+ endConsumer.finish();
+ }
+
+ public void processFieldsInner(final Fieldable[] fields,
+ final int count) throws IOException {
+
+ fieldState.reset(docState.doc.getBoost());
+
+ final int maxFieldLength = docState.maxFieldLength;
+
+ final boolean doInvert = consumer.start(fields, count);
+
+ endConsumer.start(fields, count);
+
+ for(int i=0;i<count;i++) {
+
+ Fieldable field = fields[i];
+
+ // TODO FI: this should be "genericized" to querying
+ // consumer if it wants to see this particular field
+ // tokenized.
+ if (field.isIndexed() && doInvert) {
+
+ consumer.start(field);
+
+ if (fieldState.length > 0)
+ fieldState.position += docState.analyzer.getPositionIncrementGap(fieldInfo.name);
+
+ if (!field.isTokenized()) { // un-tokenized field
+ String stringValue = field.stringValue();
+ final int valueLength = stringValue.length();
+ Token token = perThread.localToken;
+ token.clear();
+ char[] termBuffer = token.termBuffer();
+ if (termBuffer.length < valueLength)
+ termBuffer = token.resizeTermBuffer(valueLength);
+ stringValue.getChars(0, valueLength, termBuffer, 0);
+ token.setTermLength(valueLength);
+ token.setStartOffset(fieldState.offset);
+ token.setEndOffset(fieldState.offset + stringValue.length());
+ boolean success = false;
+ try {
+ consumer.add(token);
+ success = true;
+ } finally {
+ if (!success)
+ docState.docWriter.setAborting();
+ }
+ fieldState.offset += stringValue.length();
+ fieldState.length++;
+ fieldState.position++;
+ } else { // tokenized field
+ final TokenStream stream;
+ final TokenStream streamValue = field.tokenStreamValue();
+
+ if (streamValue != null)
+ stream = streamValue;
+ else {
+ // the field does not have a TokenStream,
+ // so we have to obtain one from the analyzer
+ final Reader reader; // find or make Reader
+ final Reader readerValue = field.readerValue();
+
+ if (readerValue != null)
+ reader = readerValue;
+ else {
+ String stringValue = field.stringValue();
+ if (stringValue == null)
+ throw new IllegalArgumentException("field must have either TokenStream, String or Reader value");
+ perThread.stringReader.init(stringValue);
+ reader = perThread.stringReader;
+ }
+
+ // Tokenize field and add to postingTable
+ stream = docState.analyzer.reusableTokenStream(fieldInfo.name, reader);
+ }
+
+ // reset the TokenStream to the first token
+ stream.reset();
+
+ try {
+ int offsetEnd = fieldState.offset-1;
+ final Token localToken = perThread.localToken;
+ for(;;) {
+
+ // If we hit an exception in stream.next below
+ // (which is fairly common, eg if analyzer
+ // chokes on a given document), then it's
+ // non-aborting and (above) this one document
+ // will be marked as deleted, but still
+ // consume a docID
+ Token token = stream.next(localToken);
+
+ if (token == null) break;
+ fieldState.position += (token.getPositionIncrement() - 1);
+ boolean success = false;
+ try {
+ // If we hit an exception in here, we abort
+ // all buffered documents since the last
+ // flush, on the likelihood that the
+ // internal state of the consumer is now
+ // corrupt and should not be flushed to a
+ // new segment:
+ consumer.add(token);
+ success = true;
+ } finally {
+ if (!success)
+ docState.docWriter.setAborting();
+ }
+ fieldState.position++;
+ offsetEnd = fieldState.offset + token.endOffset();
+ if (++fieldState.length >= maxFieldLength) {
+ if (docState.infoStream != null)
+ docState.infoStream.println("maxFieldLength " +maxFieldLength+ " reached for field " + fieldInfo.name + ", ignoring following tokens");
+ break;
+ }
+ }
+ fieldState.offset = offsetEnd+1;
+ } finally {
+ stream.close();
+ }
+ }
+
+ fieldState.boost *= field.getBoost();
+ consumer.end(field);
+ }
+ }
+ }
+}
Property changes on: src/java/org/apache/lucene/index/DocInverterPerField.java
___________________________________________________________________
Name: svn:eol-style
+ native
Index: src/java/org/apache/lucene/index/NormsWriterPerField.java
===================================================================
--- src/java/org/apache/lucene/index/NormsWriterPerField.java (revision 0)
+++ src/java/org/apache/lucene/index/NormsWriterPerField.java (revision 0)
@@ -0,0 +1,81 @@
+package org.apache.lucene.index;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.document.Fieldable;
+import org.apache.lucene.search.Similarity;
+
+/** Taps into DocInverter, as an InvertedDocEndConsumer,
+ * which is called at the end of inverting each field. We
+ * just look at the length for the field (docState.length)
+ * and record the norm. */
+
+class NormsWriterPerField extends InvertedDocEndConsumerPerField implements Comparable {
+
+ final NormsWriterPerThread perThread;
+ final FieldInfo fieldInfo;
+ final DocumentsWriter.DocState docState;
+
+ // Holds all docID/norm pairs we've seen
+ int[] docIDs = new int[1];
+ byte[] norms = new byte[1];
+ int upto;
+
+ final DocInverter.FieldInvertState fieldState;
+
+ public void reset() {
+ // Shrink back if we are overallocated now:
+ docIDs = ArrayUtil.shrink(docIDs, upto);
+ norms = ArrayUtil.shrink(norms, upto);
+ upto = 0;
+ }
+
+ public NormsWriterPerField(final DocInverterPerField docInverterPerField, final NormsWriterPerThread perThread, final FieldInfo fieldInfo) {
+ this.perThread = perThread;
+ this.fieldInfo = fieldInfo;
+ docState = perThread.docState;
+ fieldState = docInverterPerField.fieldState;
+ }
+
+ void start(Fieldable[] fields, int count) {
+ }
+
+ void abort() {
+ upto = 0;
+ }
+
+ public int compareTo(Object other) {
+ return fieldInfo.name.compareTo(((NormsWriterPerField) other).fieldInfo.name);
+ }
+
+ void finish() {
+ assert docIDs.length == norms.length;
+ if (fieldInfo.isIndexed && !fieldInfo.omitNorms) {
+ if (docIDs.length <= upto) {
+ assert docIDs.length == upto;
+ docIDs = ArrayUtil.grow(docIDs, 1+upto);
+ norms = ArrayUtil.grow(norms, 1+upto);
+ }
+ final float norm = fieldState.boost * docState.similarity.lengthNorm(fieldInfo.name, fieldState.length);
+ norms[upto] = Similarity.encodeNorm(norm);
+ docIDs[upto] = docState.docID;
+ upto++;
+ }
+ }
+}
Property changes on: src/java/org/apache/lucene/index/NormsWriterPerField.java
___________________________________________________________________
Name: svn:eol-style
+ native
Index: src/java/org/apache/lucene/index/TermsHashConsumerPerThread.java
===================================================================
--- src/java/org/apache/lucene/index/TermsHashConsumerPerThread.java (revision 0)
+++ src/java/org/apache/lucene/index/TermsHashConsumerPerThread.java (revision 0)
@@ -0,0 +1,27 @@
+package org.apache.lucene.index;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+
+abstract class TermsHashConsumerPerThread {
+ abstract void startDocument() throws IOException;
+ abstract DocumentsWriter.DocWriter finishDocument() throws IOException;
+ abstract public TermsHashConsumerPerField addField(TermsHashPerField termsHashPerField, FieldInfo fieldInfo);
+ abstract public void abort();
+}
Property changes on: src/java/org/apache/lucene/index/TermsHashConsumerPerThread.java
___________________________________________________________________
Name: svn:eol-style
+ native
Index: src/java/org/apache/lucene/index/DocConsumerPerThread.java
===================================================================
--- src/java/org/apache/lucene/index/DocConsumerPerThread.java (revision 0)
+++ src/java/org/apache/lucene/index/DocConsumerPerThread.java (revision 0)
@@ -0,0 +1,33 @@
+package org.apache.lucene.index;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+
+abstract class DocConsumerPerThread {
+
+ /** Process the document. If there is
+ * something for this document to be done in docID order,
+ * you should encapsulate that as a
+ * DocumentsWriter.DocWriter and return it.
+ * DocumentsWriter then calls finish() on this object
+ * when it's its turn. */
+ abstract DocumentsWriter.DocWriter processDocument() throws IOException;
+
+ abstract void abort();
+}
Property changes on: src/java/org/apache/lucene/index/DocConsumerPerThread.java
___________________________________________________________________
Name: svn:eol-style
+ native
Index: src/java/org/apache/lucene/index/DocumentsWriterFieldMergeState.java
===================================================================
--- src/java/org/apache/lucene/index/DocumentsWriterFieldMergeState.java (revision 664673)
+++ src/java/org/apache/lucene/index/DocumentsWriterFieldMergeState.java (working copy)
@@ -1,89 +0,0 @@
-package org.apache.lucene.index;
-
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-import java.io.IOException;
-
-/** Used by DocumentsWriter to merge the postings from
- * multiple ThreadStates when creating a segment */
-final class DocumentsWriterFieldMergeState {
-
- DocumentsWriterFieldData field;
-
- Posting[] postings;
-
- private Posting p;
- char[] text;
- int textOffset;
-
- private int postingUpto = -1;
-
- ByteSliceReader freq = new ByteSliceReader();
- ByteSliceReader prox = new ByteSliceReader();
-
- int docID;
- int termFreq;
-
- boolean nextTerm() throws IOException {
- postingUpto++;
- if (postingUpto == field.numPostings)
- return false;
-
- p = postings[postingUpto];
- docID = 0;
-
- text = field.threadState.charPool.buffers[p.textStart >> DocumentsWriter.CHAR_BLOCK_SHIFT];
- textOffset = p.textStart & DocumentsWriter.CHAR_BLOCK_MASK;
-
- if (p.freqUpto > p.freqStart)
- freq.init(field.threadState.postingsPool, p.freqStart, p.freqUpto);
- else
- freq.bufferOffset = freq.upto = freq.endIndex = 0;
-
- prox.init(field.threadState.postingsPool, p.proxStart, p.proxUpto);
-
- // Should always be true
- boolean result = nextDoc();
- assert result;
-
- return true;
- }
-
- public boolean nextDoc() throws IOException {
- if (freq.bufferOffset + freq.upto == freq.endIndex) {
- if (p.lastDocCode != -1) {
- // Return last doc
- docID = p.lastDocID;
- termFreq = p.docFreq;
- p.lastDocCode = -1;
- return true;
- } else
- // EOF
- return false;
- }
-
- final int code = freq.readVInt();
- docID += code >>> 1;
- if ((code & 1) != 0)
- termFreq = 1;
- else
- termFreq = freq.readVInt();
-
- return true;
- }
-}
Index: src/java/org/apache/lucene/index/FreqProxFieldMergeState.java
===================================================================
--- src/java/org/apache/lucene/index/FreqProxFieldMergeState.java (revision 0)
+++ src/java/org/apache/lucene/index/FreqProxFieldMergeState.java (revision 0)
@@ -0,0 +1,97 @@
+package org.apache.lucene.index;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+
+// TODO FI: some of this is "generic" to TermsHash* so we
+// should factor it out so other consumers don't have to
+// duplicate this code
+
+/** Used by DocumentsWriter to merge the postings from
+ * multiple ThreadStates when creating a segment */
+final class FreqProxFieldMergeState {
+
+ final FreqProxTermsWriterPerField field;
+ final int numPostings;
+ final CharBlockPool charPool;
+ final RawPostingList[] postings;
+
+ private FreqProxTermsWriter.PostingList p;
+ char[] text;
+ int textOffset;
+
+ private int postingUpto = -1;
+
+ final ByteSliceReader freq = new ByteSliceReader();
+ final ByteSliceReader prox = new ByteSliceReader();
+
+ int docID;
+ int termFreq;
+
+ public FreqProxFieldMergeState(FreqProxTermsWriterPerField field) {
+ this.field = field;
+ this.charPool = field.perThread.termsHashPerThread.charPool;
+ this.numPostings = field.termsHashPerField.numPostings;
+ this.postings = field.termsHashPerField.sortPostings();
+ }
+
+ boolean nextTerm() throws IOException {
+ postingUpto++;
+ if (postingUpto == numPostings)
+ return false;
+
+ p = (FreqProxTermsWriter.PostingList) postings[postingUpto];
+ docID = 0;
+
+ text = charPool.buffers[p.textStart >> DocumentsWriter.CHAR_BLOCK_SHIFT];
+ textOffset = p.textStart & DocumentsWriter.CHAR_BLOCK_MASK;
+
+ field.termsHashPerField.initReader(freq, p, 0);
+ field.termsHashPerField.initReader(prox, p, 1);
+
+ // Should always be true
+ boolean result = nextDoc();
+ assert result;
+
+ return true;
+ }
+
+ public boolean nextDoc() throws IOException {
+ if (freq.eof()) {
+ if (p.lastDocCode != -1) {
+ // Return last doc
+ docID = p.lastDocID;
+ termFreq = p.docFreq;
+ p.lastDocCode = -1;
+ return true;
+ } else
+ // EOF
+ return false;
+ }
+
+ final int code = freq.readVInt();
+ docID += code >>> 1;
+ if ((code & 1) != 0)
+ termFreq = 1;
+ else
+ termFreq = freq.readVInt();
+
+ return true;
+ }
+}
Property changes on: src/java/org/apache/lucene/index/FreqProxFieldMergeState.java
___________________________________________________________________
Name: svn:eol-style
+ native
Index: src/java/org/apache/lucene/index/TermsHash.java
===================================================================
--- src/java/org/apache/lucene/index/TermsHash.java (revision 0)
+++ src/java/org/apache/lucene/index/TermsHash.java (revision 0)
@@ -0,0 +1,247 @@
+package org.apache.lucene.index;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.util.Collection;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.HashSet;
+import java.util.Arrays;
+import java.io.IOException;
+
+import org.apache.lucene.util.ArrayUtil;
+
+/** This class implements {@link InvertedDocConsumer}, which
+ * is passed each token produced by the analyzer on each
+ * field. It stores these tokens in a hash table, and
+ * allocates separate byte streams per token. Consumers of
+ * this class, eg {@link FreqProxTermsWriter} and {@link
+ * TermVectorsTermsWriter}, write their own byte streams
+ * under each term.
+ */
+
+class TermsHash extends InvertedDocConsumer {
+
+ final TermsHashConsumer consumer;
+ final TermsHash nextTermsHash;
+ final int bytesPerPosting;
+ final int postingsFreeChunk;
+ final int streamCount;
+ final DocumentsWriter docWriter;
+
+ TermsHash primaryTermsHash;
+
+ RawPostingList[] postingsFreeList = new RawPostingList[1];
+ int postingsFreeCount;
+ int postingsAllocCount;
+ boolean trackAllocations;
+
+ public TermsHash(final DocumentsWriter docWriter, boolean trackAllocations, final TermsHashConsumer consumer, final TermsHash nextTermsHash) {
+ this.docWriter = docWriter;
+ this.consumer = consumer;
+ this.streamCount = consumer.streamCount;
+ this.nextTermsHash = nextTermsHash;
+ this.trackAllocations = trackAllocations;
+
+ // Why + 4*POINTER_NUM_BYTE below?
+ // +1: Posting is referenced by postingsFreeList array
+ // +3: Posting is referenced by hash, which
+ // targets 25-50% fill factor; approximate this
+ // as 3X # pointers
+ bytesPerPosting = consumer.bytesPerPosting() + 4*DocumentsWriter.POINTER_NUM_BYTE + streamCount*DocumentsWriter.INT_NUM_BYTE;
+ postingsFreeChunk = (int) (DocumentsWriter.BYTE_BLOCK_SIZE / bytesPerPosting);
+ }
+
+ InvertedDocConsumerPerThread addThread(DocInverterPerThread docInverterPerThread) {
+ return new TermsHashPerThread(docInverterPerThread, this, nextTermsHash, null);
+ }
+
+ TermsHashPerThread addThread(DocInverterPerThread docInverterPerThread, TermsHashPerThread primaryPerThread) {
+ return new TermsHashPerThread(docInverterPerThread, this, nextTermsHash, primaryPerThread);
+ }
+
+ void setFieldInfos(FieldInfos fieldInfos) {
+ this.fieldInfos = fieldInfos;
+ consumer.setFieldInfos(fieldInfos);
+ }
+
+ public void abort() {
+ consumer.abort();
+ if (nextTermsHash != null)
+ nextTermsHash.abort();
+ }
+
+ void shrinkFreePostings() {
+
+ assert postingsFreeCount == postingsAllocCount;
+
+ //System.out.println("TermsHash.reset consumer=" + consumer);
+ final int newSize = ArrayUtil.getShrinkSize(postingsFreeList.length, postingsAllocCount);
+ if (newSize != postingsFreeList.length) {
+ //System.out.println(" shrink " + postingsFreeList.length + " to " + newSize);
+ RawPostingList[] newArray = new RawPostingList[newSize];
+ System.arraycopy(postingsFreeList, 0, newArray, 0, postingsFreeCount);
+ postingsFreeList = newArray;
+ } //else
+ //System.out.println(" skip shrink length=" + postingsFreeList.length + " vs freeCount=" + postingsAllocCount);
+ }
+
+ void closeDocStore(DocumentsWriter.FlushState state) throws IOException {
+ consumer.closeDocStore(state);
+ if (nextTermsHash != null)
+ nextTermsHash.closeDocStore(state);
+ }
+
+ void flush(Map threadsAndFields, final DocumentsWriter.FlushState state) throws IOException {
+ Map childThreadsAndFields = new HashMap();
+ Map nextThreadsAndFields;
+
+ if (nextTermsHash != null)
+ nextThreadsAndFields = new HashMap();
+ else
+ nextThreadsAndFields = null;
+
+ Iterator it = threadsAndFields.entrySet().iterator();
+ while(it.hasNext()) {
+
+ Map.Entry entry = (Map.Entry) it.next();
+
+ TermsHashPerThread perThread = (TermsHashPerThread) entry.getKey();
+
+ Collection fields = (Collection) entry.getValue();
+
+ Iterator fieldsIt = fields.iterator();
+ Collection childFields = new HashSet();
+ Collection nextChildFields;
+
+ if (nextTermsHash != null)
+ nextChildFields = new HashSet();
+ else
+ nextChildFields = null;
+
+ while(fieldsIt.hasNext()) {
+ TermsHashPerField perField = (TermsHashPerField) fieldsIt.next();
+ childFields.add(perField.consumer);
+ if (nextTermsHash != null)
+ nextChildFields.add(perField.nextPerField);
+ }
+
+ childThreadsAndFields.put(perThread.consumer, childFields);
+ if (nextTermsHash != null)
+ nextThreadsAndFields.put(perThread.nextPerThread, nextChildFields);
+ }
+
+ consumer.flush(childThreadsAndFields, state);
+
+ shrinkFreePostings();
+
+ if (nextTermsHash != null)
+ nextTermsHash.flush(nextThreadsAndFields, state);
+ }
+
+ synchronized public boolean freeRAM() {
+
+ if (!trackAllocations)
+ return false;
+
+ boolean any;
+ final int numToFree;
+ if (postingsFreeCount >= postingsFreeChunk)
+ numToFree = postingsFreeChunk;
+ else
+ numToFree = postingsFreeCount;
+ any = numToFree > 0;
+ if (any) {
+ Arrays.fill(postingsFreeList, postingsFreeCount-numToFree, postingsFreeCount, null);
+ postingsFreeCount -= numToFree;
+ postingsAllocCount -= numToFree;
+ docWriter.bytesAllocated(-numToFree * bytesPerPosting);
+ any = true;
+ }
+
+ if (nextTermsHash != null)
+ any |= nextTermsHash.freeRAM();
+
+ return any;
+ }
+
+ // USE ONLY FOR DEBUGGING!
+ /*
+ public String getPostingText() {
+ char[] text = charPool.buffers[p.textStart >> CHAR_BLOCK_SHIFT];
+ int upto = p.textStart & CHAR_BLOCK_MASK;
+ while(text[upto] != 0xffff)
+ upto++;
+ return new String(text, p.textStart, upto-(p.textStart & BYTE_BLOCK_MASK));
+ }
+ */
+
+ synchronized public void recyclePostings(final RawPostingList[] postings, final int numPostings) {
+
+ assert postings.length >= numPostings;
+
+ // Move all Postings from this ThreadState back to our
+ // free list. We pre-allocated this array while we were
+ // creating Postings to make sure it's large enough
+ assert postingsFreeCount + numPostings <= postingsFreeList.length;
+ System.arraycopy(postings, 0, postingsFreeList, postingsFreeCount, numPostings);
+ postingsFreeCount += numPostings;
+ }
+
+ synchronized public void getPostings(final RawPostingList[] postings) {
+
+ assert docWriter.writer.testPoint("TermsHash.getPostings start");
+
+ assert postingsFreeCount <= postingsFreeList.length;
+ assert postingsFreeCount <= postingsAllocCount;
+
+ final int numToCopy;
+ if (postingsFreeCount < postings.length)
+ numToCopy = postingsFreeCount;
+ else
+ numToCopy = postings.length;
+ final int start = postingsFreeCount-numToCopy;
+ assert start >= 0;
+ assert start + numToCopy <= postingsFreeList.length;
+ assert numToCopy <= postings.length;
+ System.arraycopy(postingsFreeList, start,
+ postings, 0, numToCopy);
+ postingsFreeCount -= numToCopy;
+
+ // Directly allocate the remainder if any
+ if (numToCopy < postings.length) {
+ final int extra = postings.length - numToCopy;
+ final int newPostingsAllocCount = postingsAllocCount + extra;
+
+ if (newPostingsAllocCount > postingsFreeList.length)
+ postingsFreeList = new RawPostingList[ArrayUtil.getNextSize(newPostingsAllocCount)];
+
+ // nocommit -- understand what OOM here will do
+ consumer.createPostings(postings, numToCopy, extra);
+ assert docWriter.writer.testPoint("TermsHash.getPostings after create");
+ postingsAllocCount += extra;
+
+ if (trackAllocations)
+ docWriter.bytesAllocated(extra * bytesPerPosting);
+ }
+
+ if (trackAllocations)
+ docWriter.bytesUsed(postings.length * bytesPerPosting);
+ }
+}
Property changes on: src/java/org/apache/lucene/index/TermsHash.java
___________________________________________________________________
Name: svn:eol-style
+ native
Index: src/java/org/apache/lucene/index/FieldsWriter.java
===================================================================
--- src/java/org/apache/lucene/index/FieldsWriter.java (revision 664673)
+++ src/java/org/apache/lucene/index/FieldsWriter.java (working copy)
@@ -112,6 +112,10 @@
doClose = false;
}
+ void setFieldsStream(IndexOutput stream) {
+ this.fieldsStream = stream;
+ }
+
// Writes the contents of buffer into the fields stream
// and adds a new entry for this document into the index
// stream. This assumes the buffer was already written
@@ -122,6 +126,11 @@
buffer.writeTo(fieldsStream);
}
+ void skipDocument() throws IOException {
+ indexStream.writeLong(fieldsStream.getFilePointer());
+ fieldsStream.writeVInt(0);
+ }
+
void flush() throws IOException {
indexStream.flush();
fieldsStream.flush();
@@ -129,10 +138,37 @@
final void close() throws IOException {
if (doClose) {
- if (fieldsStream != null)
- fieldsStream.close();
- if (indexStream != null)
- indexStream.close();
+
+ try {
+ if (fieldsStream != null) {
+ try {
+ fieldsStream.close();
+ } finally {
+ fieldsStream = null;
+ }
+ }
+ } catch (IOException ioe) {
+ try {
+ if (indexStream != null) {
+ try {
+ indexStream.close();
+ } finally {
+ indexStream = null;
+ }
+ }
+ } catch (IOException ioe2) {
+ // Ignore so we throw only first IOException hit
+ }
+ throw ioe;
+ } finally {
+ if (indexStream != null) {
+ try {
+ indexStream.close();
+ } finally {
+ indexStream = null;
+ }
+ }
+ }
}
}
Index: src/java/org/apache/lucene/index/ByteSliceReader.java
===================================================================
--- src/java/org/apache/lucene/index/ByteSliceReader.java (revision 664673)
+++ src/java/org/apache/lucene/index/ByteSliceReader.java (working copy)
@@ -39,7 +39,9 @@
public void init(ByteBlockPool pool, int startIndex, int endIndex) {
- assert endIndex-startIndex > 0;
+ assert endIndex-startIndex >= 0;
+ assert startIndex >= 0;
+ assert endIndex >= 0;
this.pool = pool;
this.endIndex = endIndex;
@@ -59,9 +61,14 @@
limit = upto+firstSize-4;
}
+ public boolean eof() {
+ assert upto + bufferOffset <= endIndex;
+ return upto + bufferOffset == endIndex;
+ }
+
public byte readByte() {
- // Assert that we are not @ EOF
- assert upto + bufferOffset < endIndex;
+ assert !eof();
+ assert upto <= limit;
if (upto == limit)
nextSlice();
return buffer[upto++];
Index: src/java/org/apache/lucene/index/DocFieldProcessor.java
===================================================================
--- src/java/org/apache/lucene/index/DocFieldProcessor.java (revision 0)
+++ src/java/org/apache/lucene/index/DocFieldProcessor.java (revision 0)
@@ -0,0 +1,75 @@
+package org.apache.lucene.index;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.Iterator;
+
+/**
+ * This is a DocConsumer that gathers all fields under the
+ * same name, and calls per-field consumers to process field
+ * by field. This class doesn't doesn't do any "real" work
+ * of its own: it just forwards the fields to a
+ * DocFieldConsumer.
+ */
+
+class DocFieldProcessor extends DocConsumer {
+
+ final DocumentsWriter docWriter;
+ final FieldInfos fieldInfos = new FieldInfos();
+ final DocFieldConsumer consumer;
+
+ public DocFieldProcessor(DocumentsWriter docWriter, DocFieldConsumer consumer) {
+ this.docWriter = docWriter;
+ this.consumer = consumer;
+ consumer.setFieldInfos(fieldInfos);
+ }
+
+ public void closeDocStore(DocumentsWriter.FlushState state) throws IOException {
+ consumer.closeDocStore(state);
+ }
+
+ public void flush(Collection threads, DocumentsWriter.FlushState state) throws IOException {
+ fieldInfos.write(state.directory, state.segmentName + ".fnm");
+
+ Map childThreadsAndFields = new HashMap();
+ Iterator it = threads.iterator();
+ while(it.hasNext()) {
+ DocFieldProcessorPerThread perThread = (DocFieldProcessorPerThread) it.next();
+ childThreadsAndFields.put(perThread.consumer, perThread.fields());
+ perThread.trimFields(state);
+ }
+
+ consumer.flush(childThreadsAndFields, state);
+ }
+
+ public void abort() {
+ consumer.abort();
+ }
+
+ public boolean freeRAM() {
+ return consumer.freeRAM();
+ }
+
+ public DocConsumerPerThread addThread(DocumentsWriterThreadState threadState) throws IOException {
+ return new DocFieldProcessorPerThread(threadState, this);
+ }
+}
Property changes on: src/java/org/apache/lucene/index/DocFieldProcessor.java
___________________________________________________________________
Name: svn:eol-style
+ native
Index: src/java/org/apache/lucene/index/TermsHashPerField.java
===================================================================
--- src/java/org/apache/lucene/index/TermsHashPerField.java (revision 0)
+++ src/java/org/apache/lucene/index/TermsHashPerField.java (revision 0)
@@ -0,0 +1,507 @@
+package org.apache.lucene.index;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+import java.util.Arrays;
+
+import org.apache.lucene.document.Fieldable;
+import org.apache.lucene.analysis.Token;
+import org.apache.lucene.util.UnicodeUtil;
+
+class TermsHashPerField extends InvertedDocConsumerPerField {
+
+ final TermsHashConsumerPerField consumer;
+ final TermsHashPerField nextPerField;
+ final TermsHashPerThread perThread;
+ final DocumentsWriter.DocState docState;
+ final DocInverter.FieldInvertState fieldState;
+
+ final int streamCount;
+ final int numPostingInt;
+
+ final FieldInfo fieldInfo;
+
+ boolean postingsCompacted;
+ int numPostings;
+ private int postingsHashSize = 4;
+ private int postingsHashHalfSize = postingsHashSize/2;
+ private int postingsHashMask = postingsHashSize-1;
+ private RawPostingList[] postingsHash = new RawPostingList[postingsHashSize];
+ private RawPostingList p;
+
+ public TermsHashPerField(DocInverterPerField docInverterPerField, final TermsHashPerThread perThread, final TermsHashPerThread nextPerThread, final FieldInfo fieldInfo) {
+ this.perThread = perThread;
+ docState = perThread.docState;
+ fieldState = docInverterPerField.fieldState;
+ streamCount = perThread.termsHash.streamCount;
+ numPostingInt = 2*streamCount;
+ this.consumer = perThread.consumer.addField(this, fieldInfo);
+ this.fieldInfo = fieldInfo;
+ if (nextPerThread != null)
+ nextPerField = (TermsHashPerField) nextPerThread.addField(docInverterPerField, fieldInfo);
+ else
+ nextPerField = null;
+ }
+
+ void shrinkHash(int targetSize) {
+ assert postingsCompacted || numPostings == 0;
+ int newSize = postingsHash.length;
+ while(newSize >= 8 && newSize/4 > targetSize) {
+ newSize /= 2;
+ }
+
+ if (newSize != postingsHash.length) {
+ //System.out.println("TermsHashPerField consumer=" + consumer + " field=" + fieldInfo.name + " shrink hash from " + postingsHash.length + " to " + newSize + " targetSize=" + targetSize);
+ postingsHash = new RawPostingList[newSize];
+ postingsHashSize = newSize;
+ postingsHashHalfSize = newSize/2;
+ postingsHashMask = newSize-1;
+ }
+ }
+
+ public void reset() {
+ if (!postingsCompacted)
+ compactPostings();
+ assert numPostings <= postingsHash.length;
+ perThread.termsHash.recyclePostings(postingsHash, numPostings);
+ Arrays.fill(postingsHash, 0, numPostings, null);
+ postingsCompacted = false;
+ numPostings = 0;
+ if (nextPerField != null)
+ nextPerField.reset();
+ }
+
+ public void abort() {
+ reset();
+ if (nextPerField != null)
+ nextPerField.abort();
+ }
+
+ public void initReader(ByteSliceReader reader, RawPostingList p, int stream) {
+ assert stream < streamCount;
+ final int[] ints = perThread.intPool.buffers[p.intStart >> DocumentsWriter.INT_BLOCK_SHIFT];
+ final int upto = p.intStart & DocumentsWriter.INT_BLOCK_MASK;
+ reader.init(perThread.bytePool,
+ p.byteStart+stream*ByteBlockPool.FIRST_LEVEL_SIZE,
+ ints[upto+stream]);
+ }
+
+ private void compactPostings() {
+ int upto = 0;
+ for(int i=0;i<postingsHashSize;i++) {
+ if (postingsHash[i] != null) {
+ if (upto < i) {
+ postingsHash[upto] = postingsHash[i];
+ postingsHash[i] = null;
+ }
+ upto++;
+ }
+ }
+
+ assert upto == numPostings;
+ postingsCompacted = true;
+ }
+
+ /** Collapse the hash table & sort in-place. */
+ public RawPostingList[] sortPostings() {
+ compactPostings();
+ quickSort(postingsHash, 0, numPostings-1);
+ return postingsHash;
+ }
+
+ void quickSort(RawPostingList[] postings, int lo, int hi) {
+ if (lo >= hi)
+ return;
+ else if (hi == 1+lo) {
+ if (comparePostings(postings[lo], postings[hi]) > 0) {
+ final RawPostingList tmp = postings[lo];
+ postings[lo] = postings[hi];
+ postings[hi] = tmp;
+ }
+ return;
+ }
+
+ int mid = (lo + hi) >>> 1;
+
+ if (comparePostings(postings[lo], postings[mid]) > 0) {
+ RawPostingList tmp = postings[lo];
+ postings[lo] = postings[mid];
+ postings[mid] = tmp;
+ }
+
+ if (comparePostings(postings[mid], postings[hi]) > 0) {
+ RawPostingList tmp = postings[mid];
+ postings[mid] = postings[hi];
+ postings[hi] = tmp;
+
+ if (comparePostings(postings[lo], postings[mid]) > 0) {
+ RawPostingList tmp2 = postings[lo];
+ postings[lo] = postings[mid];
+ postings[mid] = tmp2;
+ }
+ }
+
+ int left = lo + 1;
+ int right = hi - 1;
+
+ if (left >= right)
+ return;
+
+ RawPostingList partition = postings[mid];
+
+ for (; ;) {
+ while (comparePostings(postings[right], partition) > 0)
+ --right;
+
+ while (left < right && comparePostings(postings[left], partition) <= 0)
+ ++left;
+
+ if (left < right) {
+ RawPostingList tmp = postings[left];
+ postings[left] = postings[right];
+ postings[right] = tmp;
+ --right;
+ } else {
+ break;
+ }
+ }
+
+ quickSort(postings, lo, left);
+ quickSort(postings, left + 1, hi);
+ }
+
+ /** Compares term text for two Posting instance and
+ * returns -1 if p1 < p2; 1 if p1 > p2; else 0. */
+ int comparePostings(RawPostingList p1, RawPostingList p2) {
+
+ if (p1 == p2)
+ return 0;
+
+ final char[] text1 = perThread.charPool.buffers[p1.textStart >> DocumentsWriter.CHAR_BLOCK_SHIFT];
+ int pos1 = p1.textStart & DocumentsWriter.CHAR_BLOCK_MASK;
+ final char[] text2 = perThread.charPool.buffers[p2.textStart >> DocumentsWriter.CHAR_BLOCK_SHIFT];
+ int pos2 = p2.textStart & DocumentsWriter.CHAR_BLOCK_MASK;
+
+ assert text1 != text2 || pos1 != pos2;
+
+ while(true) {
+ final char c1 = text1[pos1++];
+ final char c2 = text2[pos2++];
+ if (c1 != c2) {
+ if (0xffff == c2)
+ return 1;
+ else if (0xffff == c1)
+ return -1;
+ else
+ return c1-c2;
+ } else
+ // This method should never compare equal postings
+ // unless p1==p2
+ assert c1 != 0xffff;
+ }
+ }
+
+ /** Test whether the text for current RawPostingList p equals
+ * current tokenText. */
+ private boolean postingEquals(final char[] tokenText, final int tokenTextLen) {
+
+ final char[] text = perThread.charPool.buffers[p.textStart >> DocumentsWriter.CHAR_BLOCK_SHIFT];
+ assert text != null;
+ int pos = p.textStart & DocumentsWriter.CHAR_BLOCK_MASK;
+
+ int tokenPos = 0;
+ for(;tokenPos<tokenTextLen;pos++,tokenPos++)
+ if (tokenText[tokenPos] != text[pos])
+ return false;
+ return 0xffff == text[pos];
+ }
+
+ private boolean doCall;
+ private boolean doNextCall;
+
+ boolean start(Fieldable[] fields, int count) throws IOException {
+ doCall = consumer.start(fields, count);
+ if (nextPerField != null)
+ doNextCall = nextPerField.start(fields, count);
+ return doCall || doNextCall;
+ }
+
+ void start(Fieldable field) {
+ }
+ void end(Fieldable field) {
+ }
+
+ // Secondary entry point (for 2nd & subsequent TermsHash),
+ // because token text has already been "interned" into
+ // textStart, so we hash by textStart
+ public void add(Token t, int textStart) throws IOException {
+
+ int code = textStart;
+
+ int hashPos = code & postingsHashMask;
+
+ assert !postingsCompacted;
+
+ // Locate RawPostingList in hash
+ p = postingsHash[hashPos];
+
+ if (p != null && p.textStart != textStart) {
+ // Conflict: keep searching different locations in
+ // the hash table.
+ final int inc = ((code>>8)+code)|1;
+ do {
+ code += inc;
+ hashPos = code & postingsHashMask;
+ p = postingsHash[hashPos];
+ } while (p != null && p.textStart != textStart);
+ }
+
+ addToken(t, p, hashPos, textStart);
+ }
+
+ // Primary entry point (for first TermsHash)
+ void add(Token token) throws IOException {
+
+ assert !postingsCompacted;
+
+ // We are first in the chain so we must "intern" the
+ // term text into textStart address
+
+ // Get the text of this term.
+ final char[] tokenText = token.termBuffer();
+ final int tokenTextLen = token.termLength();
+
+ // Compute hashcode & replace any invalid UTF16 sequences
+ int downto = tokenTextLen;
+ int code = 0;
+ while (downto > 0) {
+ char ch = tokenText[--downto];
+
+ if (ch >= UnicodeUtil.UNI_SUR_LOW_START && ch <= UnicodeUtil.UNI_SUR_LOW_END) {
+ if (0 == downto) {
+ // Unpaired
+ ch = tokenText[downto] = UnicodeUtil.UNI_REPLACEMENT_CHAR;
+ } else {
+ final char ch2 = tokenText[downto-1];
+ if (ch2 >= UnicodeUtil.UNI_SUR_HIGH_START && ch2 <= UnicodeUtil.UNI_SUR_HIGH_END) {
+ // OK: high followed by low. This is a valid
+ // surrogate pair.
+ code = ((code*31) + ch)*31+ch2;
+ downto--;
+ continue;
+ } else {
+ // Unpaired
+ ch = tokenText[downto] = UnicodeUtil.UNI_REPLACEMENT_CHAR;
+ }
+ }
+ } else if (ch >= UnicodeUtil.UNI_SUR_HIGH_START && ch <= UnicodeUtil.UNI_SUR_HIGH_END)
+ // Unpaired
+ ch = tokenText[downto] = UnicodeUtil.UNI_REPLACEMENT_CHAR;
+
+ code = (code*31) + ch;
+ }
+
+ int hashPos = code & postingsHashMask;
+
+ // Locate RawPostingList in hash
+ p = postingsHash[hashPos];
+
+ if (p != null && !postingEquals(tokenText, tokenTextLen)) {
+ // Conflict: keep searching different locations in
+ // the hash table.
+ final int inc = ((code>>8)+code)|1;
+ do {
+ code += inc;
+ hashPos = code & postingsHashMask;
+ p = postingsHash[hashPos];
+ } while (p != null && !postingEquals(tokenText, tokenTextLen));
+ }
+
+ addToken(token, p, hashPos, -1);
+ }
+
+ void addToken(Token token, RawPostingList p, int hashPos, int textStart) throws IOException {
+
+ final CharBlockPool charPool = perThread.charPool;
+
+ if (p == null) {
+
+ // Get the text of this term.
+ final char[] tokenText = token.termBuffer();
+ final int tokenTextLen = token.termLength();
+
+ // First time we are seeing this token since we last
+ // flushed the hash.
+ final int textLen1 = 1+tokenTextLen;
+ if (perThread.primary) {
+ if (textLen1 + charPool.charUpto > DocumentsWriter.CHAR_BLOCK_SIZE) {
+ if (textLen1 > DocumentsWriter.CHAR_BLOCK_SIZE) {
+ // Just skip this term, to remain as robust as
+ // possible during indexing. A TokenFilter
+ // can be inserted into the analyzer chain if
+ // other behavior is wanted (pruning the term
+ // to a prefix, throwing an exception, etc).
+
+ if (docState.maxTermPrefix == null)
+ docState.maxTermPrefix = new String(tokenText, 0, 30);
+
+ consumer.skippingLongTerm(token);
+ return;
+ }
+ charPool.nextBuffer();
+ }
+ }
+
+ // Refill?
+ if (0 == perThread.freePostingsCount)
+ perThread.morePostings();
+
+ // Pull next free RawPostingList from free list
+ p = perThread.freePostings[--perThread.freePostingsCount];
+ assert p != null;
+
+ if (perThread.primary) {
+ final char[] text = charPool.buffer;
+ final int textUpto = charPool.charUpto;
+ p.textStart = textUpto + charPool.charOffset;
+ charPool.charUpto += textLen1;
+ System.arraycopy(tokenText, 0, text, textUpto, tokenTextLen);
+ text[textUpto+tokenTextLen] = 0xffff;
+ } else
+ p.textStart = textStart;
+
+ assert postingsHash[hashPos] == null;
+ postingsHash[hashPos] = p;
+ numPostings++;
+
+ if (numPostings == postingsHashHalfSize)
+ rehashPostings(2*postingsHashSize);
+
+ // Init stream slices
+ if (numPostingInt + perThread.intPool.intUpto > DocumentsWriter.INT_BLOCK_SIZE)
+ perThread.intPool.nextBuffer();
+
+ if (DocumentsWriter.BYTE_BLOCK_SIZE - perThread.bytePool.byteUpto < numPostingInt*ByteBlockPool.FIRST_LEVEL_SIZE)
+ perThread.bytePool.nextBuffer();
+
+ intUptos = perThread.intPool.buffer;
+ intUptoStart = perThread.intPool.intUpto;
+ perThread.intPool.intUpto += streamCount;
+
+ p.intStart = intUptoStart + perThread.intPool.intOffset;
+
+ for(int i=0;i<streamCount;i++) {
+ final int upto = perThread.bytePool.newSlice(ByteBlockPool.FIRST_LEVEL_SIZE);
+ intUptos[intUptoStart+i] = upto + perThread.bytePool.byteOffset;
+ }
+ p.byteStart = intUptos[intUptoStart];
+
+ consumer.newTerm(token, p);
+
+ } else {
+ intUptos = perThread.intPool.buffers[p.intStart >> DocumentsWriter.INT_BLOCK_SHIFT];
+ intUptoStart = p.intStart & DocumentsWriter.INT_BLOCK_MASK;
+ consumer.addTerm(token, p);
+ }
+
+ if (doNextCall)
+ nextPerField.add(token, p.textStart);
+ }
+
+ int[] intUptos;
+ int intUptoStart;
+
+ void writeByte(int stream, byte b) {
+ int upto = intUptos[intUptoStart+stream];
+ byte[] bytes = perThread.bytePool.buffers[upto >> DocumentsWriter.BYTE_BLOCK_SHIFT];
+ assert bytes != null;
+ int offset = upto & DocumentsWriter.BYTE_BLOCK_MASK;
+ if (bytes[offset] != 0) {
+ // End of slice; allocate a new one
+ offset = perThread.bytePool.allocSlice(bytes, offset);
+ bytes = perThread.bytePool.buffer;
+ intUptos[intUptoStart+stream] = offset + perThread.bytePool.byteOffset;
+ }
+ bytes[offset] = b;
+ (intUptos[intUptoStart+stream])++;
+ }
+
+ public void writeBytes(int stream, byte[] b, int offset, int len) {
+ // TODO: optimize
+ final int end = offset + len;
+ for(int i=offset;i<end;i++)
+ writeByte(stream, b[i]);
+ }
+
+ void writeVInt(int stream, int i) {
+ while ((i & ~0x7F) != 0) {
+ writeByte(stream, (byte)((i & 0x7f) | 0x80));
+ i >>>= 7;
+ }
+ writeByte(stream, (byte) i);
+ }
+
+ void finish() throws IOException {
+ consumer.finish();
+ if (nextPerField != null)
+ nextPerField.finish();
+ }
+
+ /** Called when postings hash is too small (> 50%
+ * occupied) or too large (< 20% occupied). */
+ void rehashPostings(final int newSize) {
+
+ final int newMask = newSize-1;
+
+ RawPostingList[] newHash = new RawPostingList[newSize];
+ for(int i=0;i<postingsHashSize;i++) {
+ RawPostingList p0 = postingsHash[i];
+ if (p0 != null) {
+ int code;
+ if (perThread.primary) {
+ final int start = p0.textStart & DocumentsWriter.CHAR_BLOCK_MASK;
+ final char[] text = perThread.charPool.buffers[p0.textStart >> DocumentsWriter.CHAR_BLOCK_SHIFT];
+ int pos = start;
+ while(text[pos] != 0xffff)
+ pos++;
+ code = 0;
+ while (pos > start)
+ code = (code*31) + text[--pos];
+ } else
+ code = p0.textStart;
+
+ int hashPos = code & newMask;
+ assert hashPos >= 0;
+ if (newHash[hashPos] != null) {
+ final int inc = ((code>>8)+code)|1;
+ do {
+ code += inc;
+ hashPos = code & newMask;
+ } while (newHash[hashPos] != null);
+ }
+ newHash[hashPos] = p0;
+ }
+ }
+
+ postingsHashMask = newMask;
+ postingsHash = newHash;
+ postingsHashSize = newSize;
+ postingsHashHalfSize = newSize >> 1;
+ }
+}
Property changes on: src/java/org/apache/lucene/index/TermsHashPerField.java
___________________________________________________________________
Name: svn:eol-style
+ native
Index: src/java/org/apache/lucene/index/FreqProxTermsWriter.java
===================================================================
--- src/java/org/apache/lucene/index/FreqProxTermsWriter.java (revision 0)
+++ src/java/org/apache/lucene/index/FreqProxTermsWriter.java (revision 0)
@@ -0,0 +1,362 @@
+package org.apache.lucene.index;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.util.UnicodeUtil;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Map;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Iterator;
+
+class FreqProxTermsWriter extends TermsHashConsumer {
+
+ FreqProxTermsWriter() {
+ streamCount = 2;
+ }
+
+ public TermsHashConsumerPerThread addThread(TermsHashPerThread perThread) {
+ return new FreqProxTermsWriterPerThread(perThread);
+ }
+
+ void createPostings(RawPostingList[] postings, int start, int count) {
+ final int end = start + count;
+ for(int i=start;i<end;i++)
+ postings[i] = new PostingList();
+ }
+
+ private static int compareText(final char[] text1, int pos1, final char[] text2, int pos2) {
+ while(true) {
+ final char c1 = text1[pos1++];
+ final char c2 = text2[pos2++];
+ if (c1 != c2) {
+ if (0xffff == c2)
+ return 1;
+ else if (0xffff == c1)
+ return -1;
+ else
+ return c1-c2;
+ } else if (0xffff == c1)
+ return 0;
+ }
+ }
+
+ void closeDocStore(DocumentsWriter.FlushState state) {}
+ void abort() {}
+
+
+ // TODO: would be nice to factor out morme of this, eg the
+ // FreqProxFieldMergeState, and code to visit all Fields
+ // under the same FieldInfo together, up into TermsHash*.
+ // Other writers would presumably share alot of this...
+
+ public void flush(Map threadsAndFields, final DocumentsWriter.FlushState state) throws IOException {
+
+ // Gather all FieldData's that have postings, across all
+ // ThreadStates
+ List allFields = new ArrayList();
+
+ Iterator it = threadsAndFields.entrySet().iterator();
+ while(it.hasNext()) {
+
+ Map.Entry entry = (Map.Entry) it.next();
+
+ FreqProxTermsWriterPerThread perThread = (FreqProxTermsWriterPerThread) entry.getKey();
+
+ Collection fields = (Collection) entry.getValue();
+
+ Iterator fieldsIt = fields.iterator();
+
+ while(fieldsIt.hasNext()) {
+ FreqProxTermsWriterPerField perField = (FreqProxTermsWriterPerField) fieldsIt.next();
+ if (perField.termsHashPerField.numPostings > 0)
+ allFields.add(perField);
+ }
+ }
+
+ // Sort by field name
+ Collections.sort(allFields);
+ final int numAllFields = allFields.size();
+
+ final TermInfosWriter termsOut = new TermInfosWriter(state.directory,
+ state.segmentName,
+ fieldInfos,
+ state.docWriter.writer.getTermIndexInterval());
+
+ final IndexOutput freqOut = state.directory.createOutput(state.segmentFileName(IndexFileNames.FREQ_EXTENSION));
+ final IndexOutput proxOut = state.directory.createOutput(state.segmentFileName(IndexFileNames.PROX_EXTENSION));
+
+ final DefaultSkipListWriter skipListWriter = new DefaultSkipListWriter(termsOut.skipInterval,
+ termsOut.maxSkipLevels,
+ state.numDocsInRAM, freqOut, proxOut);
+
+ int start = 0;
+ while(start < numAllFields) {
+ final String fieldName = ((FreqProxTermsWriterPerField) allFields.get(start)).fieldInfo.name;
+
+ int end = start+1;
+ while(end < numAllFields && ((FreqProxTermsWriterPerField) allFields.get(end)).fieldInfo.name.equals(fieldName))
+ end++;
+
+ FreqProxTermsWriterPerField[] fields = new FreqProxTermsWriterPerField[end-start];
+ for(int i=start;i<end;i++)
+ fields[i-start] = (FreqProxTermsWriterPerField) allFields.get(i);
+
+ // If this field has postings then add them to the
+ // segment
+ appendPostings(state, fields, termsOut, freqOut, proxOut, skipListWriter);
+
+ for(int i=0;i<fields.length;i++) {
+ TermsHashPerField perField = fields[i].termsHashPerField;
+ int numPostings = perField.numPostings;
+ perField.reset();
+ perField.shrinkHash(numPostings);
+ }
+
+ start = end;
+ }
+
+ it = threadsAndFields.entrySet().iterator();
+ while(it.hasNext()) {
+ Map.Entry entry = (Map.Entry) it.next();
+ FreqProxTermsWriterPerThread perThread = (FreqProxTermsWriterPerThread) entry.getKey();
+ perThread.termsHashPerThread.reset(true);
+ }
+
+ freqOut.close();
+ proxOut.close();
+ termsOut.close();
+
+ // Record all files we have flushed
+ state.flushedFiles.add(state.segmentFileName(IndexFileNames.FIELD_INFOS_EXTENSION));
+ state.flushedFiles.add(state.segmentFileName(IndexFileNames.FREQ_EXTENSION));
+ state.flushedFiles.add(state.segmentFileName(IndexFileNames.PROX_EXTENSION));
+ state.flushedFiles.add(state.segmentFileName(IndexFileNames.TERMS_EXTENSION));
+ state.flushedFiles.add(state.segmentFileName(IndexFileNames.TERMS_INDEX_EXTENSION));
+ }
+
+ final byte[] copyByteBuffer = new byte[4096];
+
+ /** Copy numBytes from srcIn to destIn */
+ void copyBytes(IndexInput srcIn, IndexOutput destIn, long numBytes) throws IOException {
+ // TODO: we could do this more efficiently (save a copy)
+ // because it's always from a ByteSliceReader ->
+ // IndexOutput
+ while(numBytes > 0) {
+ final int chunk;
+ if (numBytes > 4096)
+ chunk = 4096;
+ else
+ chunk = (int) numBytes;
+ srcIn.readBytes(copyByteBuffer, 0, chunk);
+ destIn.writeBytes(copyByteBuffer, 0, chunk);
+ numBytes -= chunk;
+ }
+ }
+
+ /* Walk through all unique text tokens (Posting
+ * instances) found in this field and serialize them
+ * into a single RAM segment. */
+ void appendPostings(final DocumentsWriter.FlushState flushState,
+ FreqProxTermsWriterPerField[] fields,
+ TermInfosWriter termsOut,
+ IndexOutput freqOut,
+ IndexOutput proxOut,
+ DefaultSkipListWriter skipListWriter)
+ throws CorruptIndexException, IOException {
+
+ final int fieldNumber = fields[0].fieldInfo.number;
+ int numFields = fields.length;
+
+ final FreqProxFieldMergeState[] mergeStates = new FreqProxFieldMergeState[numFields];
+
+ for(int i=0;i<numFields;i++) {
+ FreqProxFieldMergeState fms = mergeStates[i] = new FreqProxFieldMergeState(fields[i]);
+
+ assert fms.field.fieldInfo == fields[0].fieldInfo;
+
+ // Should always be true
+ boolean result = fms.nextTerm();
+ assert result;
+ }
+
+ final int skipInterval = termsOut.skipInterval;
+ final boolean currentFieldStorePayloads = fields[0].fieldInfo.storePayloads;
+
+ FreqProxFieldMergeState[] termStates = new FreqProxFieldMergeState[numFields];
+
+ while(numFields > 0) {
+
+ // Get the next term to merge
+ termStates[0] = mergeStates[0];
+ int numToMerge = 1;
+
+ for(int i=1;i<numFields;i++) {
+ final char[] text = mergeStates[i].text;
+ final int textOffset = mergeStates[i].textOffset;
+ final int cmp = compareText(text, textOffset, termStates[0].text, termStates[0].textOffset);
+
+ if (cmp < 0) {
+ termStates[0] = mergeStates[i];
+ numToMerge = 1;
+ } else if (cmp == 0)
+ termStates[numToMerge++] = mergeStates[i];
+ }
+
+ int df = 0;
+ int lastPayloadLength = -1;
+
+ int lastDoc = 0;
+
+ final char[] text = termStates[0].text;
+ final int start = termStates[0].textOffset;
+
+ long freqPointer = freqOut.getFilePointer();
+ long proxPointer = proxOut.getFilePointer();
+
+ skipListWriter.resetSkip();
+
+ // Now termStates has numToMerge FieldMergeStates
+ // which all share the same term. Now we must
+ // interleave the docID streams.
+ while(numToMerge > 0) {
+
+ if ((++df % skipInterval) == 0) {
+ skipListWriter.setSkipData(lastDoc, currentFieldStorePayloads, lastPayloadLength);
+ skipListWriter.bufferSkip(df);
+ }
+
+ FreqProxFieldMergeState minState = termStates[0];
+ for(int i=1;i<numToMerge;i++)
+ if (termStates[i].docID < minState.docID)
+ minState = termStates[i];
+
+ final int doc = minState.docID;
+ final int termDocFreq = minState.termFreq;
+
+ assert doc < flushState.numDocsInRAM;
+ assert doc > lastDoc || df == 1;
+
+ final int newDocCode = (doc-lastDoc)<<1;
+
+ lastDoc = doc;
+
+ final ByteSliceReader prox = minState.prox;
+
+ // Carefully copy over the prox + payload info,
+ // changing the format to match Lucene's segment
+ // format.
+ for(int j=0;j<termDocFreq;j++) {
+ final int code = prox.readVInt();
+ if (currentFieldStorePayloads) {
+ final int payloadLength;
+ if ((code & 1) != 0) {
+ // This position has a payload
+ payloadLength = prox.readVInt();
+ } else
+ payloadLength = 0;
+ if (payloadLength != lastPayloadLength) {
+ proxOut.writeVInt(code|1);
+ proxOut.writeVInt(payloadLength);
+ lastPayloadLength = payloadLength;
+ } else
+ proxOut.writeVInt(code & (~1));
+ if (payloadLength > 0)
+ copyBytes(prox, proxOut, payloadLength);
+ } else {
+ assert 0 == (code & 1);
+ proxOut.writeVInt(code>>1);
+ }
+ }
+
+ if (1 == termDocFreq) {
+ freqOut.writeVInt(newDocCode|1);
+ } else {
+ freqOut.writeVInt(newDocCode);
+ freqOut.writeVInt(termDocFreq);
+ }
+
+ if (!minState.nextDoc()) {
+
+ // Remove from termStates
+ int upto = 0;
+ for(int i=0;i<numToMerge;i++)
+ if (termStates[i] != minState)
+ termStates[upto++] = termStates[i];
+ numToMerge--;
+ assert upto == numToMerge;
+
+ // Advance this state to the next term
+
+ if (!minState.nextTerm()) {
+ // OK, no more terms, so remove from mergeStates
+ // as well
+ upto = 0;
+ for(int i=0;i<numFields;i++)
+ if (mergeStates[i] != minState)
+ mergeStates[upto++] = mergeStates[i];
+ numFields--;
+ assert upto == numFields;
+ }
+ }
+ }
+
+ assert df > 0;
+
+ // Done merging this term
+
+ long skipPointer = skipListWriter.writeSkip(freqOut);
+
+ // Write term
+ termInfo.set(df, freqPointer, proxPointer, (int) (skipPointer - freqPointer));
+
+ // TODO: we could do this incrementally
+ UnicodeUtil.UTF16toUTF8(text, start, termsUTF8);
+
+ // TODO: we could save O(n) re-scan of the term by
+ // computing the shared prefix with the last term
+ // while during the UTF8 encoding
+ termsOut.add(fieldNumber,
+ termsUTF8.result,
+ termsUTF8.length,
+ termInfo);
+ }
+ }
+
+ private final TermInfo termInfo = new TermInfo(); // minimize consing
+
+ final UnicodeUtil.UTF8Result termsUTF8 = new UnicodeUtil.UTF8Result();
+
+ void files(Collection files) {}
+
+ static final class PostingList extends RawPostingList {
+ int docFreq; // # times this term occurs in the current doc
+ int lastDocID; // Last docID where this term occurred
+ int lastDocCode; // Code for prior doc
+ int lastPosition; // Last position where this term occurred
+ }
+
+ int bytesPerPosting() {
+ return RawPostingList.BYTES_SIZE + 4 * DocumentsWriter.INT_NUM_BYTE;
+ }
+}
Property changes on: src/java/org/apache/lucene/index/FreqProxTermsWriter.java
___________________________________________________________________
Name: svn:eol-style
+ native
Index: src/java/org/apache/lucene/index/CharBlockPool.java
===================================================================
--- src/java/org/apache/lucene/index/CharBlockPool.java (revision 664673)
+++ src/java/org/apache/lucene/index/CharBlockPool.java (working copy)
@@ -23,11 +23,11 @@
int numBuffer;
int bufferUpto = -1; // Which buffer we are upto
- public int byteUpto = DocumentsWriter.CHAR_BLOCK_SIZE; // Where we are in head buffer
+ public int charUpto = DocumentsWriter.CHAR_BLOCK_SIZE; // Where we are in head buffer
public char[] buffer; // Current head buffer
- public int byteOffset = -DocumentsWriter.CHAR_BLOCK_SIZE; // Current head offset
- private DocumentsWriter docWriter;
+ public int charOffset = -DocumentsWriter.CHAR_BLOCK_SIZE; // Current head offset
+ final private DocumentsWriter docWriter;
public CharBlockPool(DocumentsWriter docWriter) {
this.docWriter = docWriter;
@@ -36,8 +36,8 @@
public void reset() {
docWriter.recycleCharBlocks(buffers, 1+bufferUpto);
bufferUpto = -1;
- byteUpto = DocumentsWriter.CHAR_BLOCK_SIZE;
- byteOffset = -DocumentsWriter.CHAR_BLOCK_SIZE;
+ charUpto = DocumentsWriter.CHAR_BLOCK_SIZE;
+ charOffset = -DocumentsWriter.CHAR_BLOCK_SIZE;
}
public void nextBuffer() {
@@ -49,8 +49,8 @@
buffer = buffers[1+bufferUpto] = docWriter.getCharBlock();
bufferUpto++;
- byteUpto = 0;
- byteOffset += DocumentsWriter.CHAR_BLOCK_SIZE;
+ charUpto = 0;
+ charOffset += DocumentsWriter.CHAR_BLOCK_SIZE;
}
}
Index: src/java/org/apache/lucene/index/InvertedDocConsumer.java
===================================================================
--- src/java/org/apache/lucene/index/InvertedDocConsumer.java (revision 0)
+++ src/java/org/apache/lucene/index/InvertedDocConsumer.java (revision 0)
@@ -0,0 +1,46 @@
+package org.apache.lucene.index;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.util.Map;
+import java.io.IOException;
+
+abstract class InvertedDocConsumer {
+
+ /** Add a new thread */
+ abstract InvertedDocConsumerPerThread addThread(DocInverterPerThread docInverterPerThread);
+
+ /** Abort (called after hitting AbortException) */
+ abstract void abort();
+
+ /** Flush a new segment */
+ abstract void flush(Map threadsAndFields, DocumentsWriter.FlushState state) throws IOException;
+
+ /** Close doc stores */
+ abstract void closeDocStore(DocumentsWriter.FlushState state) throws IOException;
+
+ /** Attempt to free RAM, returning true if any RAM was
+ * freed */
+ abstract boolean freeRAM();
+
+ FieldInfos fieldInfos;
+
+ void setFieldInfos(FieldInfos fieldInfos) {
+ this.fieldInfos = fieldInfos;
+ }
+}
Property changes on: src/java/org/apache/lucene/index/InvertedDocConsumer.java
___________________________________________________________________
Name: svn:eol-style
+ native
Index: src/java/org/apache/lucene/index/DocFieldProcessorPerField.java
===================================================================
--- src/java/org/apache/lucene/index/DocFieldProcessorPerField.java (revision 0)
+++ src/java/org/apache/lucene/index/DocFieldProcessorPerField.java (revision 0)
@@ -0,0 +1,45 @@
+package org.apache.lucene.index;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import org.apache.lucene.document.Fieldable;
+
+/**
+ * Holds all per thread, per field state.
+ */
+
+class DocFieldProcessorPerField {
+
+ final DocFieldConsumerPerField consumer;
+ final FieldInfo fieldInfo;
+
+ DocFieldProcessorPerField next;
+ int lastGen = -1;
+
+ int fieldCount;
+ Fieldable[] fields = new Fieldable[1];
+
+ public DocFieldProcessorPerField(final DocFieldProcessorPerThread perThread, final FieldInfo fieldInfo) {
+ this.consumer = perThread.consumer.addField(fieldInfo);
+ this.fieldInfo = fieldInfo;
+ }
+
+ public void abort() {
+ consumer.abort();
+ }
+}
Property changes on: src/java/org/apache/lucene/index/DocFieldProcessorPerField.java
___________________________________________________________________
Name: svn:eol-style
+ native
Index: src/java/org/apache/lucene/index/DocFieldConsumer.java
===================================================================
--- src/java/org/apache/lucene/index/DocFieldConsumer.java (revision 0)
+++ src/java/org/apache/lucene/index/DocFieldConsumer.java (revision 0)
@@ -0,0 +1,49 @@
+package org.apache.lucene.index;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+import java.util.Map;
+
+abstract class DocFieldConsumer {
+
+ FieldInfos fieldInfos;
+
+ /** Called when DocumentsWriter decides to create a new
+ * segment */
+ abstract void flush(Map threadsAndFields, DocumentsWriter.FlushState state) throws IOException;
+
+ /** Called when DocumentsWriter decides to close the doc
+ * stores */
+ abstract void closeDocStore(DocumentsWriter.FlushState state) throws IOException;
+
+ /** Called when an aborting exception is hit */
+ abstract void abort();
+
+ /** Add a new thread */
+ abstract DocFieldConsumerPerThread addThread(DocFieldProcessorPerThread docFieldProcessorPerThread) throws IOException;
+
+ /** Called when DocumentsWriter is using too much RAM.
+ * The consumer should free RAM, if possible, returning
+ * true if any RAM was in fact freed. */
+ abstract boolean freeRAM();
+
+ void setFieldInfos(FieldInfos fieldInfos) {
+ this.fieldInfos = fieldInfos;
+ }
+}
Property changes on: src/java/org/apache/lucene/index/DocFieldConsumer.java
___________________________________________________________________
Name: svn:eol-style
+ native
Index: src/java/org/apache/lucene/index/FreqProxTermsWriterPerField.java
===================================================================
--- src/java/org/apache/lucene/index/FreqProxTermsWriterPerField.java (revision 0)
+++ src/java/org/apache/lucene/index/FreqProxTermsWriterPerField.java (revision 0)
@@ -0,0 +1,122 @@
+package org.apache.lucene.index;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+import org.apache.lucene.document.Fieldable;
+import org.apache.lucene.analysis.Token;
+
+// TODO: break into separate freq and prox writers as
+// codecs; make separate container (tii/tis/skip/*) that can
+// be configured as any number of files 1..N
+class FreqProxTermsWriterPerField extends TermsHashConsumerPerField implements Comparable {
+
+ final FreqProxTermsWriterPerThread perThread;
+ final TermsHashPerField termsHashPerField;
+ final FieldInfo fieldInfo;
+ final DocumentsWriter.DocState docState;
+ final DocInverter.FieldInvertState fieldState;
+
+ public FreqProxTermsWriterPerField(TermsHashPerField termsHashPerField, FreqProxTermsWriterPerThread perThread, FieldInfo fieldInfo) {
+ this.termsHashPerField = termsHashPerField;
+ this.perThread = perThread;
+ this.fieldInfo = fieldInfo;
+ docState = termsHashPerField.docState;
+ fieldState = termsHashPerField.fieldState;
+ }
+
+ void finish() {}
+
+ boolean doNext;
+
+ void skippingLongTerm(Token t) throws IOException {}
+
+ public int compareTo(Object other0) {
+ FreqProxTermsWriterPerField other = (FreqProxTermsWriterPerField) other0;
+ return fieldInfo.name.compareTo(other.fieldInfo.name);
+ }
+
+ boolean start(Fieldable[] fields, int count) {
+ for(int i=0;i<count;i++) {
+ Fieldable field = fields[i];
+ if (field.isIndexed())
+ return true;
+ }
+ return false;
+ }
+
+ void start(Fieldable field) {}
+ void end(Fieldable field) {}
+
+ final void writeProx(Token t, FreqProxTermsWriter.PostingList p, int proxCode) {
+ final Payload payload = t.getPayload();
+ if (payload != null && payload.length > 0) {
+ termsHashPerField.writeVInt(1, (proxCode<<1)|1);
+ termsHashPerField.writeVInt(1, payload.length);
+ termsHashPerField.writeBytes(1, payload.data, payload.offset, payload.length);
+ // nocommit -- not thread safe (JMM)?
+ fieldInfo.storePayloads = true;
+ } else
+ termsHashPerField.writeVInt(1, proxCode<<1);
+ p.lastPosition = fieldState.position;
+ }
+
+ final void newTerm(Token t, RawPostingList p0) {
+ // First time we're seeing this term since the last
+ // flush
+ assert docState.testPoint("FreqProxTermsWriterPerField.newTerm start");
+ FreqProxTermsWriter.PostingList p = (FreqProxTermsWriter.PostingList) p0;
+ p.lastDocCode = docState.docID << 1;
+ p.lastDocID = docState.docID;
+ p.docFreq = 1;
+ writeProx(t, p, fieldState.position);
+ }
+
+ final void addTerm(Token t, RawPostingList p0) {
+
+ assert docState.testPoint("FreqProxTermsWriterPerField.addTerm start");
+
+ FreqProxTermsWriter.PostingList p = (FreqProxTermsWriter.PostingList) p0;
+
+ assert p.docFreq > 0;
+
+ if (docState.docID != p.lastDocID) {
+ // Term not yet seen in the current doc but previously
+ // seen in other doc(s) since the last flush
+
+ // Now that we know doc freq for previous doc,
+ // write it & lastDocCode
+ if (1 == p.docFreq)
+ termsHashPerField.writeVInt(0, p.lastDocCode|1);
+ else {
+ termsHashPerField.writeVInt(0, p.lastDocCode);
+ termsHashPerField.writeVInt(0, p.docFreq);
+ }
+ p.docFreq = 1;
+ p.lastDocCode = (docState.docID - p.lastDocID) << 1;
+ p.lastDocID = docState.docID;
+ writeProx(t, p, fieldState.position);
+ } else {
+ p.docFreq++;
+ writeProx(t, p, fieldState.position-p.lastPosition);
+ }
+ }
+
+ public void abort() {}
+}
+
Property changes on: src/java/org/apache/lucene/index/FreqProxTermsWriterPerField.java
___________________________________________________________________
Name: svn:eol-style
+ native
Index: src/java/org/apache/lucene/index/DocumentsWriterFieldData.java
===================================================================
--- src/java/org/apache/lucene/index/DocumentsWriterFieldData.java (revision 664673)
+++ src/java/org/apache/lucene/index/DocumentsWriterFieldData.java (working copy)
@@ -1,682 +0,0 @@
-package org.apache.lucene.index;
-
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-import org.apache.lucene.document.Fieldable;
-import org.apache.lucene.analysis.Analyzer;
-import org.apache.lucene.analysis.Token;
-import org.apache.lucene.analysis.TokenStream;
-import org.apache.lucene.store.IndexOutput;
-import org.apache.lucene.util.UnicodeUtil;
-import java.io.IOException;
-import java.io.Reader;
-import java.util.Arrays;
-
-/** Used by DocumentsWriter to hold data associated with a
- * single field in a single ThreadState, including the
- * Postings hash. A document may have many occurrences for
- * a given field name; we gather all such occurrences here
- * (in docFields) so that we can process the entire field
- * at once. */
-
-final class DocumentsWriterFieldData implements Comparable {
-
- final DocumentsWriterThreadState threadState;
- FieldInfo fieldInfo;
-
- int fieldCount;
- Fieldable[] docFields = new Fieldable[1];
-
- int lastGen = -1;
- DocumentsWriterFieldData next;
-
- boolean doNorms;
- boolean doVectors;
- boolean doVectorPositions;
- boolean doVectorOffsets;
- boolean postingsCompacted;
-
- int numPostings;
-
- Posting[] postingsHash;
- int postingsHashSize;
- int postingsHashHalfSize;
- int postingsHashMask;
-
- int position;
- int length;
- int offset;
- float boost;
- int postingsVectorsUpto;
-
- final ByteSliceWriter sliceWriter;
- final ByteSliceWriter vectorsSliceWriter;
-
- public DocumentsWriterFieldData(DocumentsWriterThreadState threadState, FieldInfo fieldInfo) {
- this.fieldInfo = fieldInfo;
- this.threadState = threadState;
- sliceWriter = new ByteSliceWriter(threadState.postingsPool);
- vectorsSliceWriter = new ByteSliceWriter(threadState.vectorsPool);
- }
-
- void resetPostingArrays() {
- if (!postingsCompacted)
- compactPostings();
- threadState.docWriter.recyclePostings(this.postingsHash, numPostings);
- Arrays.fill(postingsHash, 0, postingsHash.length, null);
- postingsCompacted = false;
- numPostings = 0;
- }
-
- void initPostingArrays() {
- // Target hash fill factor of <= 50%
- // NOTE: must be a power of two for hash collision
- // strategy to work correctly
- postingsHashSize = 4;
- postingsHashHalfSize = 2;
- postingsHashMask = postingsHashSize-1;
- postingsHash = new Posting[postingsHashSize];
- }
-
- public int compareTo(Object o) {
- return fieldInfo.name.compareTo(((DocumentsWriterFieldData) o).fieldInfo.name);
- }
-
- private void compactPostings() {
- int upto = 0;
- for(int i=0;i<postingsHashSize;i++)
- if (postingsHash[i] != null)
- postingsHash[upto++] = postingsHash[i];
-
- assert upto == numPostings;
- postingsCompacted = true;
- }
-
- /** Collapse the hash table & sort in-place. */
- public Posting[] sortPostings() {
- compactPostings();
- threadState.doPostingSort(postingsHash, numPostings);
- return postingsHash;
- }
-
- /** Process all occurrences of one field in the document. */
- public void processField(Analyzer analyzer) throws IOException, AbortException {
- length = 0;
- position = 0;
- offset = 0;
- boost = threadState.docBoost;
-
- final int maxFieldLength = threadState.docWriter.writer.getMaxFieldLength();
-
- final int limit = fieldCount;
- final Fieldable[] docFieldsFinal = docFields;
-
- boolean doWriteVectors = true;
-
- // Walk through all occurrences in this doc for this
- // field:
- try {
- for(int j=0;j<limit;j++) {
- Fieldable field = docFieldsFinal[j];
-
- if (field.isIndexed())
- invertField(field, analyzer, maxFieldLength);
-
- if (field.isStored()) {
- threadState.numStoredFields++;
- boolean success = false;
- try {
- threadState.localFieldsWriter.writeField(fieldInfo, field);
- success = true;
- } finally {
- // If we hit an exception inside
- // localFieldsWriter.writeField, the
- // contents of fdtLocal can be corrupt, so
- // we must discard all stored fields for
- // this document:
- if (!success)
- threadState.fdtLocal.reset();
- }
- }
-
- docFieldsFinal[j] = null;
- }
- } catch (AbortException ae) {
- doWriteVectors = false;
- throw ae;
- } finally {
- if (postingsVectorsUpto > 0) {
- try {
- if (doWriteVectors) {
- // Add term vectors for this field
- boolean success = false;
- try {
- writeVectors(fieldInfo);
- success = true;
- } finally {
- if (!success) {
- // If we hit an exception inside
- // writeVectors, the contents of tvfLocal
- // can be corrupt, so we must discard all
- // term vectors for this document:
- threadState.numVectorFields = 0;
- threadState.tvfLocal.reset();
- }
- }
- }
- } finally {
- if (postingsVectorsUpto > threadState.maxPostingsVectors)
- threadState.maxPostingsVectors = postingsVectorsUpto;
- postingsVectorsUpto = 0;
- threadState.vectorsPool.reset();
- }
- }
- }
- }
-
- int offsetEnd;
- Token localToken = new Token();
-
- /* Invert one occurrence of one field in the document */
- public void invertField(Fieldable field, Analyzer analyzer, final int maxFieldLength) throws IOException, AbortException {
-
- if (length>0)
- position += analyzer.getPositionIncrementGap(fieldInfo.name);
-
- if (!field.isTokenized()) { // un-tokenized field
- String stringValue = field.stringValue();
- final int valueLength = stringValue.length();
- Token token = localToken;
- token.clear();
- char[] termBuffer = token.termBuffer();
- if (termBuffer.length < valueLength)
- termBuffer = token.resizeTermBuffer(valueLength);
- stringValue.getChars(0, valueLength, termBuffer, 0);
- token.setTermLength(valueLength);
- token.setStartOffset(offset);
- token.setEndOffset(offset + stringValue.length());
- addPosition(token);
- offset += stringValue.length();
- length++;
- } else { // tokenized field
- final TokenStream stream;
- final TokenStream streamValue = field.tokenStreamValue();
-
- if (streamValue != null)
- stream = streamValue;
- else {
- // the field does not have a TokenStream,
- // so we have to obtain one from the analyzer
- final Reader reader; // find or make Reader
- final Reader readerValue = field.readerValue();
-
- if (readerValue != null)
- reader = readerValue;
- else {
- String stringValue = field.stringValue();
- if (stringValue == null)
- throw new IllegalArgumentException("field must have either TokenStream, String or Reader value");
- threadState.stringReader.init(stringValue);
- reader = threadState.stringReader;
- }
-
- // Tokenize field and add to postingTable
- stream = analyzer.reusableTokenStream(fieldInfo.name, reader);
- }
-
- // reset the TokenStream to the first token
- stream.reset();
-
- try {
- offsetEnd = offset-1;
- for(;;) {
- Token token = stream.next(localToken);
- if (token == null) break;
- position += (token.getPositionIncrement() - 1);
- addPosition(token);
- if (++length >= maxFieldLength) {
- if (threadState.docWriter.infoStream != null)
- threadState.docWriter.infoStream.println("maxFieldLength " +maxFieldLength+ " reached for field " + fieldInfo.name + ", ignoring following tokens");
- break;
- }
- }
- offset = offsetEnd+1;
- } finally {
- stream.close();
- }
- }
-
- boost *= field.getBoost();
- }
-
- /** Only called when term vectors are enabled. This
- * is called the first time we see a given term for
- * each document, to allocate a PostingVector
- * instance that is used to record data needed to
- * write the posting vectors. */
- private PostingVector addNewVector() {
-
- if (postingsVectorsUpto == threadState.postingsVectors.length) {
- final int newSize;
- if (threadState.postingsVectors.length < 2)
- newSize = 2;
- else
- newSize = (int) (1.5*threadState.postingsVectors.length);
- PostingVector[] newArray = new PostingVector[newSize];
- System.arraycopy(threadState.postingsVectors, 0, newArray, 0, threadState.postingsVectors.length);
- threadState.postingsVectors = newArray;
- }
-
- p.vector = threadState.postingsVectors[postingsVectorsUpto];
- if (p.vector == null)
- p.vector = threadState.postingsVectors[postingsVectorsUpto] = new PostingVector();
-
- postingsVectorsUpto++;
-
- final PostingVector v = p.vector;
- v.p = p;
-
- if (doVectorPositions) {
- final int upto = threadState.vectorsPool.newSlice(ByteBlockPool.FIRST_LEVEL_SIZE);
- v.posStart = v.posUpto = threadState.vectorsPool.byteOffset + upto;
- }
-
- if (doVectorOffsets) {
- final int upto = threadState.vectorsPool.newSlice(ByteBlockPool.FIRST_LEVEL_SIZE);
- v.offsetStart = v.offsetUpto = threadState.vectorsPool.byteOffset + upto;
- }
-
- return v;
- }
-
- int offsetStartCode;
- int offsetStart;
-
- // Current posting we are working on
- Posting p;
- PostingVector vector;
-
- /** Test whether the text for current Posting p equals
- * current tokenText. */
- boolean postingEquals(final char[] tokenText, final int tokenTextLen) {
-
- final char[] text = threadState.charPool.buffers[p.textStart >> DocumentsWriter.CHAR_BLOCK_SHIFT];
- assert text != null;
- int pos = p.textStart & DocumentsWriter.CHAR_BLOCK_MASK;
-
- int tokenPos = 0;
- for(;tokenPos<tokenTextLen;pos++,tokenPos++)
- if (tokenText[tokenPos] != text[pos])
- return false;
- return 0xffff == text[pos];
- }
-
- /** This is the hotspot of indexing: it's called once
- * for every term of every document. Its job is to *
- * update the postings byte stream (Postings hash) *
- * based on the occurence of a single term. */
- private void addPosition(Token token) throws AbortException {
-
- final Payload payload = token.getPayload();
-
- // Get the text of this term. Term can either
- // provide a String token or offset into a char[]
- // array
- final char[] tokenText = token.termBuffer();
- final int tokenTextLen = token.termLength();
-
- int code = 0;
-
- // Compute hashcode & replace any invalid UTF16 sequences
- int downto = tokenTextLen;
- while (downto > 0) {
- char ch = tokenText[--downto];
-
- if (ch >= UnicodeUtil.UNI_SUR_LOW_START && ch <= UnicodeUtil.UNI_SUR_LOW_END) {
- if (0 == downto) {
- // Unpaired
- ch = tokenText[downto] = UnicodeUtil.UNI_REPLACEMENT_CHAR;
- } else {
- final char ch2 = tokenText[downto-1];
- if (ch2 >= UnicodeUtil.UNI_SUR_HIGH_START && ch2 <= UnicodeUtil.UNI_SUR_HIGH_END) {
- // OK: high followed by low. This is a valid
- // surrogate pair.
- code = ((code*31) + ch)*31+ch2;
- downto--;
- continue;
- } else {
- // Unpaired
- ch = tokenText[downto] = UnicodeUtil.UNI_REPLACEMENT_CHAR;
- }
- }
- } else if (ch >= UnicodeUtil.UNI_SUR_HIGH_START && ch <= UnicodeUtil.UNI_SUR_HIGH_END)
- // Unpaired
- ch = tokenText[downto] = UnicodeUtil.UNI_REPLACEMENT_CHAR;
-
- code = (code*31) + ch;
- }
-
- // System.out.println(" addPosition: field=" + fieldInfo.name + " buffer=" + new String(tokenText, 0, tokenTextLen) + " pos=" + position + " offsetStart=" + (offset+token.startOffset()) + " offsetEnd=" + (offset + token.endOffset()) + " docID=" + docID + " doPos=" + doVectorPositions + " doOffset=" + doVectorOffsets);
-
- int hashPos = code & postingsHashMask;
-
- assert !postingsCompacted;
-
- // Locate Posting in hash
- p = postingsHash[hashPos];
-
- if (p != null && !postingEquals(tokenText, tokenTextLen)) {
- // Conflict: keep searching different locations in
- // the hash table.
- final int inc = ((code>>8)+code)|1;
- do {
- code += inc;
- hashPos = code & postingsHashMask;
- p = postingsHash[hashPos];
- } while (p != null && !postingEquals(tokenText, tokenTextLen));
- }
-
- final int proxCode;
-
- // If we hit an exception below, it's possible the
- // posting list or term vectors data will be
- // partially written and thus inconsistent if
- // flushed, so we have to abort all documents
- // since the last flush:
-
- try {
-
- if (p != null) { // term seen since last flush
-
- if (threadState.docID != p.lastDocID) { // term not yet seen in this doc
-
- // System.out.println(" seen before (new docID=" + docID + ") freqUpto=" + p.freqUpto +" proxUpto=" + p.proxUpto);
-
- assert p.docFreq > 0;
-
- // Now that we know doc freq for previous doc,
- // write it & lastDocCode
- sliceWriter.init(p.freqUpto);
-
- if (1 == p.docFreq)
- sliceWriter.writeVInt(p.lastDocCode|1);
- else {
- sliceWriter.writeVInt(p.lastDocCode);
- sliceWriter.writeVInt(p.docFreq);
- }
- p.freqUpto = sliceWriter.getAddress();
-
- if (doVectors) {
- vector = addNewVector();
- if (doVectorOffsets) {
- offsetStartCode = offsetStart = offset + token.startOffset();
- offsetEnd = offset + token.endOffset();
- }
- }
-
- proxCode = position;
-
- p.docFreq = 1;
-
- // Store code so we can write this after we're
- // done with this new doc
- p.lastDocCode = (threadState.docID-p.lastDocID) << 1;
- p.lastDocID = threadState.docID;
-
- } else { // term already seen in this doc
- // System.out.println(" seen before (same docID=" + docID + ") proxUpto=" + p.proxUpto);
- p.docFreq++;
-
- proxCode = position-p.lastPosition;
-
- if (doVectors) {
- vector = p.vector;
- if (vector == null)
- vector = addNewVector();
- if (doVectorOffsets) {
- offsetStart = offset + token.startOffset();
- offsetEnd = offset + token.endOffset();
- offsetStartCode = offsetStart-vector.lastOffset;
- }
- }
- }
- } else { // term not seen before
- // System.out.println(" never seen docID=" + docID);
-
- // Refill?
- if (0 == threadState.postingsFreeCount) {
- threadState.docWriter.getPostings(threadState.postingsFreeList);
- threadState.postingsFreeCount = threadState.postingsFreeList.length;
- }
-
- final int textLen1 = 1+tokenTextLen;
- if (textLen1 + threadState.charPool.byteUpto > DocumentsWriter.CHAR_BLOCK_SIZE) {
- if (textLen1 > DocumentsWriter.CHAR_BLOCK_SIZE) {
- // Just skip this term, to remain as robust as
- // possible during indexing. A TokenFilter
- // can be inserted into the analyzer chain if
- // other behavior is wanted (pruning the term
- // to a prefix, throwing an exception, etc).
- if (threadState.maxTermPrefix == null)
- threadState.maxTermPrefix = new String(tokenText, 0, 30);
-
- // Still increment position:
- position++;
- return;
- }
- threadState.charPool.nextBuffer();
- }
-
- final char[] text = threadState.charPool.buffer;
- final int textUpto = threadState.charPool.byteUpto;
-
- // Pull next free Posting from free list
- p = threadState.postingsFreeList[--threadState.postingsFreeCount];
-
- p.textStart = textUpto + threadState.charPool.byteOffset;
- threadState.charPool.byteUpto += textLen1;
-
- System.arraycopy(tokenText, 0, text, textUpto, tokenTextLen);
-
- text[textUpto+tokenTextLen] = 0xffff;
-
- assert postingsHash[hashPos] == null;
-
- postingsHash[hashPos] = p;
- numPostings++;
-
- if (numPostings == postingsHashHalfSize)
- rehashPostings(2*postingsHashSize);
-
- // Init first slice for freq & prox streams
- final int upto1 = threadState.postingsPool.newSlice(ByteBlockPool.FIRST_LEVEL_SIZE);
- p.freqStart = p.freqUpto = threadState.postingsPool.byteOffset + upto1;
-
- final int upto2 = threadState.postingsPool.newSlice(ByteBlockPool.FIRST_LEVEL_SIZE);
- p.proxStart = p.proxUpto = threadState.postingsPool.byteOffset + upto2;
-
- p.lastDocCode = threadState.docID << 1;
- p.lastDocID = threadState.docID;
- p.docFreq = 1;
-
- if (doVectors) {
- vector = addNewVector();
- if (doVectorOffsets) {
- offsetStart = offsetStartCode = offset + token.startOffset();
- offsetEnd = offset + token.endOffset();
- }
- }
-
- proxCode = position;
- }
-
- sliceWriter.init(p.proxUpto);
-
- if (payload != null && payload.length > 0) {
- sliceWriter.writeVInt((proxCode<<1)|1);
- sliceWriter.writeVInt(payload.length);
- sliceWriter.writeBytes(payload.data, payload.offset, payload.length);
- fieldInfo.storePayloads = true;
- } else
- sliceWriter.writeVInt(proxCode<<1);
-
- p.proxUpto = sliceWriter.getAddress();
- p.lastPosition = position++;
-
- if (doVectorPositions) {
- vectorsSliceWriter.init(vector.posUpto);
- vectorsSliceWriter.writeVInt(proxCode);
- vector.posUpto = vectorsSliceWriter.getAddress();
- }
-
- if (doVectorOffsets) {
- vectorsSliceWriter.init(vector.offsetUpto);
- vectorsSliceWriter.writeVInt(offsetStartCode);
- vectorsSliceWriter.writeVInt(offsetEnd-offsetStart);
- vector.lastOffset = offsetEnd;
- vector.offsetUpto = vectorsSliceWriter.getAddress();
- }
- } catch (Throwable t) {
- throw new AbortException(t, threadState.docWriter);
- }
- }
-
- /** Called when postings hash is too small (> 50%
- * occupied) or too large (< 20% occupied). */
- void rehashPostings(final int newSize) {
-
- final int newMask = newSize-1;
-
- Posting[] newHash = new Posting[newSize];
- for(int i=0;i<postingsHashSize;i++) {
- Posting p0 = postingsHash[i];
- if (p0 != null) {
- final int start = p0.textStart & DocumentsWriter.CHAR_BLOCK_MASK;
- final char[] text = threadState.charPool.buffers[p0.textStart >> DocumentsWriter.CHAR_BLOCK_SHIFT];
- int pos = start;
- while(text[pos] != 0xffff)
- pos++;
- int code = 0;
- while (pos > start)
- code = (code*31) + text[--pos];
-
- int hashPos = code & newMask;
- assert hashPos >= 0;
- if (newHash[hashPos] != null) {
- final int inc = ((code>>8)+code)|1;
- do {
- code += inc;
- hashPos = code & newMask;
- } while (newHash[hashPos] != null);
- }
- newHash[hashPos] = p0;
- }
- }
-
- postingsHashMask = newMask;
- postingsHash = newHash;
- postingsHashSize = newSize;
- postingsHashHalfSize = newSize >> 1;
- }
-
- final ByteSliceReader vectorSliceReader = new ByteSliceReader();
-
- /** Called once per field per document if term vectors
- * are enabled, to write the vectors to *
- * RAMOutputStream, which is then quickly flushed to
- * * the real term vectors files in the Directory. */
- void writeVectors(FieldInfo fieldInfo) throws IOException {
-
- assert fieldInfo.storeTermVector;
- assert threadState.vectorFieldsInOrder(fieldInfo);
-
- threadState.vectorFieldNumbers[threadState.numVectorFields] = fieldInfo.number;
- threadState.vectorFieldPointers[threadState.numVectorFields] = threadState.tvfLocal.getFilePointer();
- threadState.numVectorFields++;
-
- final int numPostingsVectors = postingsVectorsUpto;
- final PostingVector[] postingsVectors = threadState.postingsVectors;
-
- final IndexOutput tvfLocal = threadState.tvfLocal;
-
- threadState.tvfLocal.writeVInt(numPostingsVectors);
- byte bits = 0x0;
- if (doVectorPositions)
- bits |= TermVectorsReader.STORE_POSITIONS_WITH_TERMVECTOR;
- if (doVectorOffsets)
- bits |= TermVectorsReader.STORE_OFFSET_WITH_TERMVECTOR;
- threadState.tvfLocal.writeByte(bits);
-
- threadState.doVectorSort(postingsVectors, numPostingsVectors);
-
- int encoderUpto = 0;
- int lastTermBytesCount = 0;
-
- final ByteSliceReader reader = vectorSliceReader;
- final char[][] charBuffers = threadState.charPool.buffers;
-
- for(int j=0;j<numPostingsVectors;j++) {
- final PostingVector vector = postingsVectors[j];
- Posting posting = vector.p;
- final int freq = posting.docFreq;
-
- final char[] text2 = charBuffers[posting.textStart >> DocumentsWriter.CHAR_BLOCK_SHIFT];
- final int start2 = posting.textStart & DocumentsWriter.CHAR_BLOCK_MASK;
-
- // We swap between two encoders to save copying
- // last Term's byte array
- final UnicodeUtil.UTF8Result utf8Result = threadState.utf8Results[encoderUpto];
-
- // TODO: we could do this incrementally
- UnicodeUtil.UTF16toUTF8(text2, start2, utf8Result);
- final int termBytesCount = utf8Result.length;
-
- // TODO: UTF16toUTF8 could tell us this prefix
- // Compute common prefix between last term and
- // this term
- int prefix = 0;
- if (j > 0) {
- final byte[] lastTermBytes = threadState.utf8Results[1-encoderUpto].result;
- final byte[] termBytes = threadState.utf8Results[encoderUpto].result;
- while(prefix < lastTermBytesCount && prefix < termBytesCount) {
- if (lastTermBytes[prefix] != termBytes[prefix])
- break;
- prefix++;
- }
- }
- encoderUpto = 1-encoderUpto;
- lastTermBytesCount = termBytesCount;
-
- final int suffix = termBytesCount - prefix;
- tvfLocal.writeVInt(prefix);
- tvfLocal.writeVInt(suffix);
- tvfLocal.writeBytes(utf8Result.result, prefix, suffix);
- tvfLocal.writeVInt(freq);
-
- if (doVectorPositions) {
- reader.init(threadState.vectorsPool, vector.posStart, vector.posUpto);
- reader.writeTo(tvfLocal);
- }
-
- if (doVectorOffsets) {
- reader.init(threadState.vectorsPool, vector.offsetStart, vector.offsetUpto);
- reader.writeTo(tvfLocal);
- }
- }
- }
-}
Index: src/java/org/apache/lucene/index/InvertedDocConsumerPerField.java
===================================================================
--- src/java/org/apache/lucene/index/InvertedDocConsumerPerField.java (revision 0)
+++ src/java/org/apache/lucene/index/InvertedDocConsumerPerField.java (revision 0)
@@ -0,0 +1,47 @@
+package org.apache.lucene.index;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import org.apache.lucene.document.Fieldable;
+import org.apache.lucene.analysis.Token;
+import java.io.IOException;
+
+abstract class InvertedDocConsumerPerField {
+
+ // Called once per field, and is given all Fieldable
+ // occurrences for this field in the document. Return
+ // true if you wish to see inverted tokens for these
+ // fields:
+ abstract boolean start(Fieldable[] fields, int count) throws IOException;
+
+ // Called once per Fieldable occurrence, before inversion
+ abstract void start(Fieldable field) throws IOException;
+
+ // Called once per inverted token
+ abstract void add(Token token) throws IOException;
+
+ // Called once per Fieldable occurrence, after inversion
+ abstract void end(Fieldable field) throws IOException;
+
+ // Called once per field per document, after all Fieldable
+ // occurrences are inverted
+ abstract void finish() throws IOException;
+
+ // Called on hitting an aborting exception
+ abstract void abort();
+}
Property changes on: src/java/org/apache/lucene/index/InvertedDocConsumerPerField.java
___________________________________________________________________
Name: svn:eol-style
+ native
Index: src/java/org/apache/lucene/index/DocFieldConsumerPerField.java
===================================================================
--- src/java/org/apache/lucene/index/DocFieldConsumerPerField.java (revision 0)
+++ src/java/org/apache/lucene/index/DocFieldConsumerPerField.java (revision 0)
@@ -0,0 +1,27 @@
+package org.apache.lucene.index;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+import org.apache.lucene.document.Fieldable;
+
+abstract class DocFieldConsumerPerField {
+ /** Processes all occurrences of a single field */
+ abstract void processFields(Fieldable[] fields, int count) throws IOException;
+ abstract void abort();
+}
Property changes on: src/java/org/apache/lucene/index/DocFieldConsumerPerField.java
___________________________________________________________________
Name: svn:eol-style
+ native
Index: src/java/org/apache/lucene/index/DocFieldConsumersPerThread.java
===================================================================
--- src/java/org/apache/lucene/index/DocFieldConsumersPerThread.java (revision 0)
+++ src/java/org/apache/lucene/index/DocFieldConsumersPerThread.java (revision 0)
@@ -0,0 +1,73 @@
+package org.apache.lucene.index;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+
+class DocFieldConsumersPerThread extends DocFieldConsumerPerThread {
+
+ final DocFieldConsumerPerThread one;
+ final DocFieldConsumerPerThread two;
+ final DocFieldConsumers parent;
+ final DocumentsWriter.DocState docState;
+
+ public DocFieldConsumersPerThread(DocFieldProcessorPerThread docFieldProcessorPerThread,
+ DocFieldConsumers parent, DocFieldConsumerPerThread one, DocFieldConsumerPerThread two) {
+ this.parent = parent;
+ this.one = one;
+ this.two = two;
+ docState = docFieldProcessorPerThread.docState;
+ }
+
+ public void startDocument() throws IOException {
+ one.startDocument();
+ two.startDocument();
+ }
+
+ public void abort() {
+ try {
+ one.abort();
+ } finally {
+ two.abort();
+ }
+ }
+
+ public DocumentsWriter.DocWriter finishDocument() throws IOException {
+ final DocumentsWriter.DocWriter oneDoc = one.finishDocument();
+ final DocumentsWriter.DocWriter twoDoc = two.finishDocument();
+ if (oneDoc == null)
+ return twoDoc;
+ else if (twoDoc == null)
+ return oneDoc;
+ else {
+ DocFieldConsumers.PerDoc both = parent.getPerDoc();
+ both.docID = docState.docID;
+ //System.out.println("oneDoc.docID=" + oneDoc.docID + " vs docState.docID=" + docState.docID + " one=" + one);
+ assert oneDoc.docID == docState.docID;
+ //System.out.println("twoDoc.docID=" + twoDoc.docID + " vs docState.docID=" + docState.docID + " two=" + two);
+ assert twoDoc.docID == docState.docID;
+ both.one = oneDoc;
+ both.two = twoDoc;
+ return both;
+ }
+ }
+
+ public DocFieldConsumerPerField addField(FieldInfo fi) {
+ return new DocFieldConsumersPerField(this, one.addField(fi), two.addField(fi));
+ }
+}
Property changes on: src/java/org/apache/lucene/index/DocFieldConsumersPerThread.java
___________________________________________________________________
Name: svn:eol-style
+ native
Index: src/java/org/apache/lucene/index/DocFieldConsumers.java
===================================================================
--- src/java/org/apache/lucene/index/DocFieldConsumers.java (revision 0)
+++ src/java/org/apache/lucene/index/DocFieldConsumers.java (revision 0)
@@ -0,0 +1,162 @@
+package org.apache.lucene.index;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.util.HashMap;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.HashSet;
+import java.io.IOException;
+
+import org.apache.lucene.util.ArrayUtil;
+
+/** This is just a "splitter" class: it lets you wrap two
+ * DocFieldConsumer instances as a single consumer. */
+
+class DocFieldConsumers extends DocFieldConsumer {
+ final DocFieldConsumer one;
+ final DocFieldConsumer two;
+
+ public DocFieldConsumers(DocFieldConsumer one, DocFieldConsumer two) {
+ this.one = one;
+ this.two = two;
+ }
+
+ void setFieldInfos(FieldInfos fieldInfos) {
+ super.setFieldInfos(fieldInfos);
+ one.setFieldInfos(fieldInfos);
+ two.setFieldInfos(fieldInfos);
+ }
+
+ public void flush(Map threadsAndFields, DocumentsWriter.FlushState state) throws IOException {
+
+ Map oneThreadsAndFields = new HashMap();
+ Map twoThreadsAndFields = new HashMap();
+
+ Iterator it = threadsAndFields.entrySet().iterator();
+ while(it.hasNext()) {
+
+ Map.Entry entry = (Map.Entry) it.next();
+
+ DocFieldConsumersPerThread perThread = (DocFieldConsumersPerThread) entry.getKey();
+
+ Collection fields = (Collection) entry.getValue();
+
+ Iterator fieldsIt = fields.iterator();
+ Collection oneFields = new HashSet();
+ Collection twoFields = new HashSet();
+ while(fieldsIt.hasNext()) {
+ DocFieldConsumersPerField perField = (DocFieldConsumersPerField) fieldsIt.next();
+ oneFields.add(perField.one);
+ twoFields.add(perField.two);
+ }
+
+ oneThreadsAndFields.put(perThread.one, oneFields);
+ twoThreadsAndFields.put(perThread.two, twoFields);
+ }
+
+
+ one.flush(oneThreadsAndFields, state);
+ two.flush(twoThreadsAndFields, state);
+ }
+
+ public void closeDocStore(DocumentsWriter.FlushState state) throws IOException {
+ try {
+ one.closeDocStore(state);
+ } finally {
+ two.closeDocStore(state);
+ }
+ }
+
+ public void abort() {
+ try {
+ one.abort();
+ } finally {
+ two.abort();
+ }
+ }
+
+ public boolean freeRAM() {
+ boolean any = one.freeRAM();
+ any |= two.freeRAM();
+ return any;
+ }
+
+ public DocFieldConsumerPerThread addThread(DocFieldProcessorPerThread docFieldProcessorPerThread) throws IOException {
+ return new DocFieldConsumersPerThread(docFieldProcessorPerThread, this, one.addThread(docFieldProcessorPerThread), two.addThread(docFieldProcessorPerThread));
+ }
+
+ PerDoc[] docFreeList = new PerDoc[1];
+ int freeCount;
+ int allocCount;
+
+ synchronized PerDoc getPerDoc() {
+ if (freeCount == 0) {
+ allocCount++;
+ if (allocCount > docFreeList.length) {
+ // Grow our free list up front to make sure we have
+ // enough space to recycle all outstanding PerDoc
+ // instances
+ assert allocCount == 1+docFreeList.length;
+ docFreeList = new PerDoc[ArrayUtil.getNextSize(allocCount)];
+ }
+ return new PerDoc();
+ } else
+ return docFreeList[--freeCount];
+ }
+
+ synchronized void freePerDoc(PerDoc perDoc) {
+ assert freeCount < docFreeList.length;
+ docFreeList[freeCount++] = perDoc;
+ }
+
+ class PerDoc extends DocumentsWriter.DocWriter {
+
+ DocumentsWriter.DocWriter one;
+ DocumentsWriter.DocWriter two;
+
+ public long sizeInBytes() {
+ return one.sizeInBytes() + two.sizeInBytes();
+ }
+
+ public void finish() throws IOException {
+ try {
+ try {
+ one.finish();
+ } finally {
+ two.finish();
+ }
+ } finally {
+ freePerDoc(this);
+ }
+ }
+
+ public void abort() {
+ try {
+ try {
+ one.abort();
+ } finally {
+ two.abort();
+ }
+ } finally {
+ freePerDoc(this);
+ }
+ }
+ }
+}
Property changes on: src/java/org/apache/lucene/index/DocFieldConsumers.java
___________________________________________________________________
Name: svn:eol-style
+ native
Index: src/java/org/apache/lucene/index/DocFieldProcessorPerThread.java
===================================================================
--- src/java/org/apache/lucene/index/DocFieldProcessorPerThread.java (revision 0)
+++ src/java/org/apache/lucene/index/DocFieldProcessorPerThread.java (revision 0)
@@ -0,0 +1,305 @@
+package org.apache.lucene.index;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.io.IOException;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Fieldable;
+
+/**
+ * Gathers all Fieldables for a document under the same
+ * name, updates FieldInfos, and calls per-field consumers
+ * to process field by field.
+ *
+ * Currently, only a single thread visits the fields,
+ * sequentially, for processing.
+ */
+
+class DocFieldProcessorPerThread extends DocConsumerPerThread {
+
+ float docBoost;
+ int fieldGen;
+ final DocFieldProcessor docFieldProcessor;
+ final FieldInfos fieldInfos;
+ final DocFieldConsumerPerThread consumer;
+
+ // Holds all fields seen in current doc
+ DocFieldProcessorPerField[] fields = new DocFieldProcessorPerField[1];
+ int fieldCount;
+
+ // Hash table for all fields ever seen
+ DocFieldProcessorPerField[] fieldHash = new DocFieldProcessorPerField[2];
+ int hashMask = 1;
+ int totalFieldCount;
+
+ final DocumentsWriter.DocState docState;
+
+ public DocFieldProcessorPerThread(DocumentsWriterThreadState threadState, DocFieldProcessor docFieldProcessor) throws IOException {
+ this.docState = threadState.docState;
+ this.docFieldProcessor = docFieldProcessor;
+ this.fieldInfos = docFieldProcessor.fieldInfos;
+ this.consumer = docFieldProcessor.consumer.addThread(this);
+ }
+
+ public void abort() {
+ for(int i=0;i<fieldHash.length;i++) {
+ DocFieldProcessorPerField field = fieldHash[i];
+ while(field != null) {
+ final DocFieldProcessorPerField next = field.next;
+ field.abort();
+ field = next;
+ }
+ }
+ consumer.abort();
+ }
+
+ public Collection fields() {
+ Collection fields = new HashSet();
+ for(int i=0;i<fieldHash.length;i++) {
+ DocFieldProcessorPerField field = fieldHash[i];
+ while(field != null) {
+ fields.add(field.consumer);
+ field = field.next;
+ }
+ }
+ assert fields.size() == totalFieldCount;
+ return fields;
+ }
+
+ /** If there are fields we've seen but did not see again
+ * in the last run, then free them up. */
+
+ void trimFields(DocumentsWriter.FlushState state) {
+
+ for(int i=0;i<fieldHash.length;i++) {
+ DocFieldProcessorPerField perField = fieldHash[i];
+ DocFieldProcessorPerField lastPerField = null;
+
+ while(perField != null) {
+
+ if (perField.lastGen == -1) {
+
+ // This field was not seen since the previous
+ // flush, so, free up its resources now
+
+ // Unhash
+ if (lastPerField == null)
+ fieldHash[i] = perField.next;
+ else
+ lastPerField.next = perField.next;
+
+ if (state.docWriter.infoStream != null)
+ state.docWriter.infoStream.println(" purge field=" + perField.fieldInfo.name);
+
+ totalFieldCount--;
+
+ } else {
+ // Reset
+ perField.lastGen = -1;
+ lastPerField = perField;
+ }
+
+ perField = perField.next;
+ }
+ }
+ }
+
+ public DocumentsWriter.DocWriter processDocument() throws IOException {
+
+ consumer.startDocument();
+ final Document doc = docState.doc;
+
+ assert docFieldProcessor.docWriter.writer.testPoint("DocumentsWriter.ThreadState.init start");
+
+ fieldCount = 0;
+
+ final int thisFieldGen = fieldGen++;
+
+ final List docFields = doc.getFields();
+ final int numDocFields = docFields.size();
+
+ // Absorb any new fields first seen in this document.
+ // Also absorb any changes to fields we had already
+ // seen before (eg suddenly turning on norms or
+ // vectors, etc.):
+
+ // TODO: we could avoid this sync eg by only locking if
+ // a change is necessary to FieldInfos
+
+ // FieldInfos isn't thread safe:
+ synchronized(fieldInfos) {
+
+ for(int i=0;i<numDocFields;i++) {
+ Fieldable field = (Fieldable) docFields.get(i);
+ final String fieldName = field.name();
+
+ // TODO FI: we need to genericize the "flags" that a
+ // field holds, and, how these flags are merged; it
+ // needs to be more "pluggable" such that if I want
+ // to have a new "thing" my Fields can do, I can
+ // easily add it
+ FieldInfo fi = fieldInfos.add(fieldName, field.isIndexed(), field.isTermVectorStored(),
+ field.isStorePositionWithTermVector(), field.isStoreOffsetWithTermVector(),
+ field.getOmitNorms(), false);
+
+ // Make sure we have a PerField allocated
+ final int hashPos = fieldName.hashCode() & hashMask;
+ DocFieldProcessorPerField fp = fieldHash[hashPos];
+ while(fp != null && !fp.fieldInfo.name.equals(fi.name))
+ fp = fp.next;
+
+ if (fp == null) {
+
+ fp = new DocFieldProcessorPerField(this, fi);
+ fp.next = fieldHash[hashPos];
+ fieldHash[hashPos] = fp;
+ totalFieldCount++;
+
+ if (totalFieldCount >= fieldHash.length/2) {
+ final int newHashSize = (int) (fieldHash.length*2);
+ assert newHashSize > fieldHash.length;
+
+ final DocFieldProcessorPerField newHashArray[] = new DocFieldProcessorPerField[newHashSize];
+
+ // Rehash
+ int newHashMask = newHashSize-1;
+ for(int j=0;j<fieldHash.length;j++) {
+ DocFieldProcessorPerField fp0 = fieldHash[j];
+ while(fp0 != null) {
+ final int hashPos2 = fp0.fieldInfo.name.hashCode() & newHashMask;
+ DocFieldProcessorPerField nextFP0 = fp0.next;
+ fp0.next = newHashArray[hashPos2];
+ newHashArray[hashPos2] = fp0;
+ fp0 = nextFP0;
+ }
+ }
+
+ fieldHash = newHashArray;
+ hashMask = newHashMask;
+ }
+
+ } else
+ assert fp.fieldInfo == fi;
+
+ if (thisFieldGen != fp.lastGen) {
+
+ // First time we're seeing this field for this doc
+ fp.fieldCount = 0;
+
+ if (fieldCount == fields.length) {
+ final int newSize = fields.length*2;
+ DocFieldProcessorPerField newArray[] = new DocFieldProcessorPerField[newSize];
+ System.arraycopy(fields, 0, newArray, 0, fieldCount);
+ fields = newArray;
+ }
+
+ fields[fieldCount++] = fp;
+ fp.lastGen = thisFieldGen;
+ }
+
+ if (fp.fieldCount == fp.fields.length) {
+ Fieldable[] newArray = new Fieldable[fp.fields.length*2];
+ System.arraycopy(fp.fields, 0, newArray, 0, fp.fieldCount);
+ fp.fields = newArray;
+ }
+
+ fp.fields[fp.fieldCount++] = field;
+ }
+ }
+
+ // If we are writing vectors then we must visit
+ // fields in sorted order so they are written in
+ // sorted order. TODO: we actually only need to
+ // sort the subset of fields that have vectors
+ // enabled; we could save [small amount of] CPU
+ // here.
+ quickSort(fields, 0, fieldCount-1);
+
+ for(int i=0;i<fieldCount;i++)
+ fields[i].consumer.processFields(fields[i].fields, fields[i].fieldCount);
+
+ if (docState.maxTermPrefix != null && docState.infoStream != null)
+ docState.infoStream.println("WARNING: document contains at least one immense term (longer than the max length " + DocumentsWriter.MAX_TERM_LENGTH + "), all of which were skipped. Please correct the analyzer to not produce such terms. The prefix of the first immense term is: '" + docState.maxTermPrefix + "...'");
+
+ return consumer.finishDocument();
+ }
+
+ void quickSort(DocFieldProcessorPerField[] array, int lo, int hi) {
+ if (lo >= hi)
+ return;
+ else if (hi == 1+lo) {
+ if (array[lo].fieldInfo.name.compareTo(array[hi].fieldInfo.name) > 0) {
+ final DocFieldProcessorPerField tmp = array[lo];
+ array[lo] = array[hi];
+ array[hi] = tmp;
+ }
+ return;
+ }
+
+ int mid = (lo + hi) >>> 1;
+
+ if (array[lo].fieldInfo.name.compareTo(array[mid].fieldInfo.name) > 0) {
+ DocFieldProcessorPerField tmp = array[lo];
+ array[lo] = array[mid];
+ array[mid] = tmp;
+ }
+
+ if (array[mid].fieldInfo.name.compareTo(array[hi].fieldInfo.name) > 0) {
+ DocFieldProcessorPerField tmp = array[mid];
+ array[mid] = array[hi];
+ array[hi] = tmp;
+
+ if (array[lo].fieldInfo.name.compareTo(array[mid].fieldInfo.name) > 0) {
+ DocFieldProcessorPerField tmp2 = array[lo];
+ array[lo] = array[mid];
+ array[mid] = tmp2;
+ }
+ }
+
+ int left = lo + 1;
+ int right = hi - 1;
+
+ if (left >= right)
+ return;
+
+ DocFieldProcessorPerField partition = array[mid];
+
+ for (; ;) {
+ while (array[right].fieldInfo.name.compareTo(partition.fieldInfo.name) > 0)
+ --right;
+
+ while (left < right && array[left].fieldInfo.name.compareTo(partition.fieldInfo.name) <= 0)
+ ++left;
+
+ if (left < right) {
+ DocFieldProcessorPerField tmp = array[left];
+ array[left] = array[right];
+ array[right] = tmp;
+ --right;
+ } else {
+ break;
+ }
+ }
+
+ quickSort(array, lo, left);
+ quickSort(array, left + 1, hi);
+ }
+}
Property changes on: src/java/org/apache/lucene/index/DocFieldProcessorPerThread.java
___________________________________________________________________
Name: svn:eol-style
+ native
Index: src/java/org/apache/lucene/index/InvertedDocEndConsumer.java
===================================================================
--- src/java/org/apache/lucene/index/InvertedDocEndConsumer.java (revision 0)
+++ src/java/org/apache/lucene/index/InvertedDocEndConsumer.java (revision 0)
@@ -0,0 +1,29 @@
+package org.apache.lucene.index;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.util.Map;
+import java.io.IOException;
+
+abstract class InvertedDocEndConsumer {
+ abstract InvertedDocEndConsumerPerThread addThread(DocInverterPerThread docInverterPerThread);
+ abstract void flush(Map threadsAndFields, DocumentsWriter.FlushState state) throws IOException;
+ abstract void closeDocStore(DocumentsWriter.FlushState state) throws IOException;
+ abstract void abort();
+ abstract void setFieldInfos(FieldInfos fieldInfos);
+}
Property changes on: src/java/org/apache/lucene/index/InvertedDocEndConsumer.java
___________________________________________________________________
Name: svn:eol-style
+ native
Index: src/java/org/apache/lucene/index/TermsHashConsumer.java
===================================================================
--- src/java/org/apache/lucene/index/TermsHashConsumer.java (revision 0)
+++ src/java/org/apache/lucene/index/TermsHashConsumer.java (revision 0)
@@ -0,0 +1,38 @@
+package org.apache.lucene.index;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+import java.util.Map;
+
+abstract class TermsHashConsumer {
+ abstract int bytesPerPosting();
+ abstract void createPostings(RawPostingList[] postings, int start, int count);
+ abstract TermsHashConsumerPerThread addThread(TermsHashPerThread perThread);
+ abstract void flush(Map threadsAndFields, final DocumentsWriter.FlushState state) throws IOException;
+ abstract void abort();
+ abstract void closeDocStore(DocumentsWriter.FlushState state) throws IOException;
+
+ int streamCount;
+
+ FieldInfos fieldInfos;
+
+ void setFieldInfos(FieldInfos fieldInfos) {
+ this.fieldInfos = fieldInfos;
+ }
+}
Property changes on: src/java/org/apache/lucene/index/TermsHashConsumer.java
___________________________________________________________________
Name: svn:eol-style
+ native
Index: src/java/org/apache/lucene/index/Posting.java
===================================================================
--- src/java/org/apache/lucene/index/Posting.java (revision 664673)
+++ src/java/org/apache/lucene/index/Posting.java (working copy)
@@ -1,36 +0,0 @@
-package org.apache.lucene.index;
-
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-/* Used by DocumentsWriter to track postings for a single
- * term. One of these exists per unique term seen since the
- * last flush. If you alter this class you must also fix
- * DocumentWriter.POSTING_NUM_BYTE to reflect the change as
- * this is how RAM usage is measured. */
-final class Posting {
- int textStart; // Address into char[] blocks where our text is stored
- int docFreq; // # times this term occurs in the current doc
- int freqStart; // Address of first byte[] slice for freq
- int freqUpto; // Next write address for freq
- int proxStart; // Address of first byte[] slice
- int proxUpto; // Next write address for prox
- int lastDocID; // Last docID where this term occurred
- int lastDocCode; // Code for prior doc
- int lastPosition; // Last position where this term occurred
- PostingVector vector; // Corresponding PostingVector instance
-}
Index: src/java/org/apache/lucene/index/DocFieldConsumersPerField.java
===================================================================
--- src/java/org/apache/lucene/index/DocFieldConsumersPerField.java (revision 0)
+++ src/java/org/apache/lucene/index/DocFieldConsumersPerField.java (revision 0)
@@ -0,0 +1,47 @@
+package org.apache.lucene.index;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+import org.apache.lucene.document.Fieldable;
+
+class DocFieldConsumersPerField extends DocFieldConsumerPerField {
+
+ final DocFieldConsumerPerField one;
+ final DocFieldConsumerPerField two;
+ final DocFieldConsumersPerThread perThread;
+
+ public DocFieldConsumersPerField(DocFieldConsumersPerThread perThread, DocFieldConsumerPerField one, DocFieldConsumerPerField two) {
+ this.perThread = perThread;
+ this.one = one;
+ this.two = two;
+ }
+
+ public void processFields(Fieldable[] fields, int count) throws IOException {
+ one.processFields(fields, count);
+ two.processFields(fields, count);
+ }
+
+ public void abort() {
+ try {
+ one.abort();
+ } finally {
+ two.abort();
+ }
+ }
+}
Property changes on: src/java/org/apache/lucene/index/DocFieldConsumersPerField.java
___________________________________________________________________
Name: svn:eol-style
+ native
Index: src/java/org/apache/lucene/index/StoredFieldsWriterPerThread.java
===================================================================
--- src/java/org/apache/lucene/index/StoredFieldsWriterPerThread.java (revision 0)
+++ src/java/org/apache/lucene/index/StoredFieldsWriterPerThread.java (revision 0)
@@ -0,0 +1,67 @@
+package org.apache.lucene.index;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+import org.apache.lucene.store.IndexOutput;
+
+class StoredFieldsWriterPerThread extends DocFieldConsumerPerThread {
+
+ final FieldsWriter localFieldsWriter;
+ final StoredFieldsWriter storedFieldsWriter;
+ final DocumentsWriter.DocState docState;
+
+ StoredFieldsWriter.PerDoc doc;
+
+ public StoredFieldsWriterPerThread(DocFieldProcessorPerThread docFieldProcessorPerThread, StoredFieldsWriter storedFieldsWriter) throws IOException {
+ this.storedFieldsWriter = storedFieldsWriter;
+ this.docState = docFieldProcessorPerThread.docState;
+ localFieldsWriter = new FieldsWriter((IndexOutput) null, (IndexOutput) null, storedFieldsWriter.fieldInfos);
+ }
+
+ public void startDocument() {
+ if (doc != null) {
+ // Only happens if previous document hit non-aborting
+ // exception while writing stored fields into
+ // localFieldsWriter:
+ doc.reset();
+ doc.docID = docState.docID;
+ }
+ }
+
+ public DocumentsWriter.DocWriter finishDocument() {
+ // If there were any stored fields in this doc, doc will
+ // be non-null; else it's null.
+ try {
+ return doc;
+ } finally {
+ doc = null;
+ }
+ }
+
+ public void abort() {
+ if (doc != null) {
+ doc.abort();
+ doc = null;
+ }
+ }
+
+ public DocFieldConsumerPerField addField(FieldInfo fieldInfo) {
+ return new StoredFieldsWriterPerField(this, fieldInfo);
+ }
+}
Property changes on: src/java/org/apache/lucene/index/StoredFieldsWriterPerThread.java
___________________________________________________________________
Name: svn:eol-style
+ native
Index: src/java/org/apache/lucene/index/DocFieldConsumerPerThread.java
===================================================================
--- src/java/org/apache/lucene/index/DocFieldConsumerPerThread.java (revision 0)
+++ src/java/org/apache/lucene/index/DocFieldConsumerPerThread.java (revision 0)
@@ -0,0 +1,27 @@
+package org.apache.lucene.index;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+
+abstract class DocFieldConsumerPerThread {
+ abstract void startDocument() throws IOException;
+ abstract DocumentsWriter.DocWriter finishDocument() throws IOException;
+ abstract DocFieldConsumerPerField addField(FieldInfo fi);
+ abstract void abort();
+}
Property changes on: src/java/org/apache/lucene/index/DocFieldConsumerPerThread.java
___________________________________________________________________
Name: svn:eol-style
+ native
Index: src/java/org/apache/lucene/index/InvertedDocEndConsumerPerField.java
===================================================================
--- src/java/org/apache/lucene/index/InvertedDocEndConsumerPerField.java (revision 0)
+++ src/java/org/apache/lucene/index/InvertedDocEndConsumerPerField.java (revision 0)
@@ -0,0 +1,26 @@
+package org.apache.lucene.index;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import org.apache.lucene.document.Fieldable;
+
+abstract class InvertedDocEndConsumerPerField {
+ abstract void start(Fieldable[] fields, int count);
+ abstract void finish();
+ abstract void abort();
+}
Property changes on: src/java/org/apache/lucene/index/InvertedDocEndConsumerPerField.java
___________________________________________________________________
Name: svn:eol-style
+ native
Index: src/java/org/apache/lucene/index/StoredFieldsWriter.java
===================================================================
--- src/java/org/apache/lucene/index/StoredFieldsWriter.java (revision 0)
+++ src/java/org/apache/lucene/index/StoredFieldsWriter.java (revision 0)
@@ -0,0 +1,166 @@
+package org.apache.lucene.index;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.util.Map;
+import java.io.IOException;
+import org.apache.lucene.store.RAMOutputStream;
+import org.apache.lucene.util.ArrayUtil;
+
+/** This is a DocFieldConsumer that writes stored fields. */
+class StoredFieldsWriter extends DocFieldConsumer {
+
+ FieldsWriter fieldsWriter;
+ final DocumentsWriter docWriter;
+ int lastDocID;
+
+ PerDoc[] docFreeList = new PerDoc[1];
+ int freeCount;
+
+ public StoredFieldsWriter(DocumentsWriter docWriter) {
+ this.docWriter = docWriter;
+ }
+
+ public DocFieldConsumerPerThread addThread(DocFieldProcessorPerThread docFieldProcessorPerThread) throws IOException {
+ return new StoredFieldsWriterPerThread(docFieldProcessorPerThread, this);
+ }
+
+ // We do nothing on flush because we write, immediately,
+ // to a doc store file
+ public void flush(Map threadsAndFields, DocumentsWriter.FlushState state) {}
+
+ private void initFieldsWriter() throws IOException {
+ if (fieldsWriter == null) {
+ fieldsWriter = new FieldsWriter(docWriter.directory,
+ docWriter.docStoreSegment,
+ fieldInfos);
+
+ assert docWriter.docStoreSegment != null;
+ docWriter.addOpenFile(docWriter.docStoreSegment + "." + IndexFileNames.FIELDS_EXTENSION);
+ docWriter.addOpenFile(docWriter.docStoreSegment + "." + IndexFileNames.FIELDS_INDEX_EXTENSION);
+ lastDocID = 0;
+ }
+ }
+
+ synchronized public void closeDocStore(DocumentsWriter.FlushState state) throws IOException {
+
+ initFieldsWriter();
+
+ fill(state.numDocsInStore - docWriter.getDocStoreOffset());
+ fieldsWriter.close();
+ fieldsWriter = null;
+
+ assert state.docStoreSegmentName != null;
+ state.flushedFiles.add(state.docStoreSegmentName + "." + IndexFileNames.FIELDS_EXTENSION);
+ state.flushedFiles.add(state.docStoreSegmentName + "." + IndexFileNames.FIELDS_INDEX_EXTENSION);
+
+ state.docWriter.removeOpenFile(state.docStoreSegmentName + "." + IndexFileNames.FIELDS_EXTENSION);
+ state.docWriter.removeOpenFile(state.docStoreSegmentName + "." + IndexFileNames.FIELDS_INDEX_EXTENSION);
+
+ if (4+state.numDocsInStore*8 != state.directory.fileLength(state.docStoreSegmentName + "." + IndexFileNames.FIELDS_INDEX_EXTENSION))
+ throw new RuntimeException("after flush: fdx size mismatch: " + state.numDocsInStore + " docs vs " + state.directory.fileLength(state.docStoreSegmentName + "." + IndexFileNames.FIELDS_INDEX_EXTENSION) + " length in bytes of " + state.docStoreSegmentName + "." + IndexFileNames.FIELDS_INDEX_EXTENSION);
+
+ }
+
+ int allocCount;
+
+ synchronized PerDoc getPerDoc() {
+ if (freeCount == 0) {
+ allocCount++;
+ if (allocCount > docFreeList.length) {
+ // Grow our free list up front to make sure we have
+ // enough space to recycle all outstanding PerDoc
+ // instances
+ assert allocCount == 1+docFreeList.length;
+ docFreeList = new PerDoc[ArrayUtil.getNextSize(allocCount)];
+ }
+ return new PerDoc();
+ } else
+ return docFreeList[--freeCount];
+ }
+
+ synchronized void abort() {
+ if (fieldsWriter != null) {
+ try {
+ fieldsWriter.close();
+ } catch (Throwable t) {
+ }
+ fieldsWriter = null;
+ }
+ }
+
+ /** Fills in any hole in the docIDs */
+ void fill(int docID) throws IOException {
+ final int docStoreOffset = docWriter.getDocStoreOffset();
+
+ // We must "catch up" for all docs before us
+ // that had no stored fields:
+ final int end = docID+docStoreOffset;
+ while(lastDocID < end) {
+ fieldsWriter.skipDocument();
+ lastDocID++;
+ }
+ }
+
+ synchronized void finishDocument(PerDoc perDoc) throws IOException {
+ initFieldsWriter();
+
+ fill(perDoc.docID);
+
+ // Append stored fields to the real FieldsWriter:
+ fieldsWriter.flushDocument(perDoc.numStoredFields, perDoc.fdt);
+ lastDocID++;
+ perDoc.reset();
+ free(perDoc);
+ }
+
+ public boolean freeRAM() {
+ return false;
+ }
+
+ void free(PerDoc perDoc) {
+ assert freeCount < docFreeList.length;
+ docFreeList[freeCount++] = perDoc;
+ }
+
+ class PerDoc extends DocumentsWriter.DocWriter {
+
+ // TODO: use something more memory efficient; for small
+ // docs the 1024 buffer size of RAMOutputStream wastes alot
+ RAMOutputStream fdt = new RAMOutputStream();
+ int numStoredFields;
+
+ void reset() {
+ fdt.reset();
+ numStoredFields = 0;
+ }
+
+ void abort() {
+ reset();
+ free(this);
+ }
+
+ public long sizeInBytes() {
+ return fdt.sizeInBytes();
+ }
+
+ public void finish() throws IOException {
+ finishDocument(this);
+ }
+ }
+}
Property changes on: src/java/org/apache/lucene/index/StoredFieldsWriter.java
___________________________________________________________________
Name: svn:eol-style
+ native
Index: src/java/org/apache/lucene/index/TermVectorsTermsWriter.java
===================================================================
--- src/java/org/apache/lucene/index/TermVectorsTermsWriter.java (revision 0)
+++ src/java/org/apache/lucene/index/TermVectorsTermsWriter.java (revision 0)
@@ -0,0 +1,281 @@
+package org.apache.lucene.index;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.store.RAMOutputStream;
+import org.apache.lucene.util.ArrayUtil;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.Map;
+
+class TermVectorsTermsWriter extends TermsHashConsumer {
+
+ final DocumentsWriter docWriter;
+ TermVectorsWriter termVectorsWriter;
+ PerDoc[] docFreeList = new PerDoc[1];
+ int freeCount;
+ IndexOutput tvx;
+ IndexOutput tvd;
+ IndexOutput tvf;
+ int lastDocID;
+
+ public TermVectorsTermsWriter(DocumentsWriter docWriter) {
+ this.docWriter = docWriter;
+ streamCount = 2;
+ }
+
+ public TermsHashConsumerPerThread addThread(TermsHashPerThread termsHashPerThread) {
+ return new TermVectorsTermsWriterPerThread(termsHashPerThread, this);
+ }
+
+ void createPostings(RawPostingList[] postings, int start, int count) {
+ System.out.println("TV: create postings");
+ final int end = start + count;
+ for(int i=start;i<end;i++)
+ postings[i] = new PostingList();
+ }
+
+ void flush(Map threadsAndFields, final DocumentsWriter.FlushState state) {
+ Iterator it = threadsAndFields.entrySet().iterator();
+ while(it.hasNext()) {
+ Map.Entry entry = (Map.Entry) it.next();
+ Iterator it2 = ((Collection) entry.getValue()).iterator();
+ while(it2.hasNext()) {
+ TermVectorsTermsWriterPerField perField = (TermVectorsTermsWriterPerField) it2.next();
+ perField.shrinkHash();
+ }
+
+ TermVectorsTermsWriterPerThread perThread = (TermVectorsTermsWriterPerThread) entry.getKey();
+ perThread.termsHashPerThread.reset(true);
+ }
+ }
+
+ synchronized void closeDocStore(final DocumentsWriter.FlushState state) throws IOException {
+ if (tvx != null) {
+ // At least one doc in this run had term vectors
+ // enabled
+ fill(state.numDocsInStore - docWriter.getDocStoreOffset());
+ tvx.close();
+ tvf.close();
+ tvd.close();
+ tvx = null;
+ assert state.docStoreSegmentName != null;
+ if (4+state.numDocsInStore*16 != state.directory.fileLength(state.docStoreSegmentName + "." + IndexFileNames.VECTORS_INDEX_EXTENSION))
+ throw new RuntimeException("after flush: tvx size mismatch: " + state.numDocsInStore + " docs vs " + state.directory.fileLength(state.docStoreSegmentName + "." + IndexFileNames.VECTORS_INDEX_EXTENSION) + " length in bytes of " + state.docStoreSegmentName + "." + IndexFileNames.VECTORS_INDEX_EXTENSION);
+
+ state.flushedFiles.add(state.docStoreSegmentName + "." + IndexFileNames.VECTORS_INDEX_EXTENSION);
+ state.flushedFiles.add(state.docStoreSegmentName + "." + IndexFileNames.VECTORS_FIELDS_EXTENSION);
+ state.flushedFiles.add(state.docStoreSegmentName + "." + IndexFileNames.VECTORS_DOCUMENTS_EXTENSION);
+
+ docWriter.removeOpenFile(state.docStoreSegmentName + "." + IndexFileNames.VECTORS_INDEX_EXTENSION);
+ docWriter.removeOpenFile(state.docStoreSegmentName + "." + IndexFileNames.VECTORS_FIELDS_EXTENSION);
+ docWriter.removeOpenFile(state.docStoreSegmentName + "." + IndexFileNames.VECTORS_DOCUMENTS_EXTENSION);
+ }
+ }
+
+ int allocCount;
+
+ synchronized PerDoc getPerDoc() {
+ if (freeCount == 0) {
+ allocCount++;
+ if (allocCount > docFreeList.length) {
+ // Grow our free list up front to make sure we have
+ // enough space to recycle all outstanding PerDoc
+ // instances
+ assert allocCount == 1+docFreeList.length;
+ docFreeList = new PerDoc[ArrayUtil.getNextSize(allocCount)];
+ }
+ return new PerDoc();
+ } else
+ return docFreeList[--freeCount];
+ }
+
+ void fill(int docID) throws IOException {
+ final int docStoreOffset = docWriter.getDocStoreOffset();
+ // We must "catch up" for all docs before us
+ // that had no vectors:
+ final long tvfPosition = tvf.getFilePointer();
+ final int end = docID+docStoreOffset;
+ while(lastDocID < end) {
+ tvx.writeLong(tvd.getFilePointer());
+ tvd.writeVInt(0);
+ tvx.writeLong(tvfPosition);
+ lastDocID++;
+ }
+ }
+
+ synchronized void finishDocument(PerDoc perDoc) throws IOException {
+
+ // nocommit -- add testPoints in here
+
+ if (tvx == null) {
+
+ assert docWriter.docStoreSegment != null;
+
+ // If we hit an exception while init'ing the term
+ // vector output files, we must abort this segment
+ // because those files will be in an unknown
+ // state:
+ tvx = docWriter.directory.createOutput(docWriter.docStoreSegment + "." + IndexFileNames.VECTORS_INDEX_EXTENSION);
+ tvd = docWriter.directory.createOutput(docWriter.docStoreSegment + "." + IndexFileNames.VECTORS_DOCUMENTS_EXTENSION);
+ tvf = docWriter.directory.createOutput(docWriter.docStoreSegment + "." + IndexFileNames.VECTORS_FIELDS_EXTENSION);
+
+ tvx.writeInt(TermVectorsReader.FORMAT_CURRENT);
+ tvd.writeInt(TermVectorsReader.FORMAT_CURRENT);
+ tvf.writeInt(TermVectorsReader.FORMAT_CURRENT);
+
+ docWriter.addOpenFile(docWriter.docStoreSegment + "." + IndexFileNames.VECTORS_INDEX_EXTENSION);
+ docWriter.addOpenFile(docWriter.docStoreSegment + "." + IndexFileNames.VECTORS_FIELDS_EXTENSION);
+ docWriter.addOpenFile(docWriter.docStoreSegment + "." + IndexFileNames.VECTORS_DOCUMENTS_EXTENSION);
+
+ lastDocID = 0;
+ }
+
+ //System.out.println("TermVectorsTermsWriter.finishDocument: docID=" + perDoc.docID + " perDoc=" + perDoc);
+
+ fill(perDoc.docID);
+
+ // Append term vectors to the real outputs:
+ tvx.writeLong(tvd.getFilePointer());
+ tvx.writeLong(tvf.getFilePointer());
+ tvd.writeVInt(perDoc.numVectorFields);
+ if (perDoc.numVectorFields > 0) {
+ for(int i=0;i<perDoc.numVectorFields;i++)
+ tvd.writeVInt(perDoc.fieldNumbers[i]);
+ assert 0 == perDoc.fieldPointers[0];
+ long lastPos = perDoc.fieldPointers[0];
+ for(int i=1;i<perDoc.numVectorFields;i++) {
+ long pos = perDoc.fieldPointers[i];
+ tvd.writeVLong(pos-lastPos);
+ lastPos = pos;
+ }
+ perDoc.tvf.writeTo(tvf);
+ perDoc.tvf.reset();
+ perDoc.numVectorFields = 0;
+ }
+
+ //System.out.println("lastDocID=" + lastDocID + " docID=" + perDoc.docID + " dso=" + docWriter.getDocStoreOffset());
+ //
+ //if (lastDocID != perDoc.docID + docWriter.getDocStoreOffset())
+ //new Throwable().printStackTrace(System.out);
+
+ assert lastDocID == perDoc.docID + docWriter.getDocStoreOffset();
+
+ lastDocID++;
+
+ free(perDoc);
+ }
+
+ public boolean freeRAM() {
+ // We don't hold any state beyond one doc, so we don't
+ // free persistent RAM here
+ return false;
+ }
+
+ public void abort() {
+ // Reset vectors writer
+ if (tvx != null) {
+ try {
+ tvx.close();
+ } catch (Throwable t) {
+ }
+ tvx = null;
+ }
+ if (tvd != null) {
+ try {
+ tvd.close();
+ } catch (Throwable t) {
+ }
+ tvd = null;
+ }
+ if (tvf != null) {
+ try {
+ tvf.close();
+ } catch (Throwable t) {
+ }
+ tvf = null;
+ }
+ }
+
+ synchronized void free(PerDoc doc) {
+ assert freeCount < docFreeList.length;
+ docFreeList[freeCount++] = doc;
+ }
+
+ class PerDoc extends DocumentsWriter.DocWriter {
+
+ // TODO: use something more memory efficient; for small
+ // docs the 1024 buffer size of RAMOutputStream wastes alot
+ RAMOutputStream tvf = new RAMOutputStream();
+ int numVectorFields;
+
+ int[] fieldNumbers = new int[1];
+ long[] fieldPointers = new long[1];
+
+ void reset() {
+ tvf.reset();
+ numVectorFields = 0;
+ }
+
+ void abort() {
+ reset();
+ free(this);
+ }
+
+ void addField(final int fieldNumber) {
+ if (numVectorFields == fieldNumbers.length) {
+ fieldNumbers = ArrayUtil.grow(fieldNumbers);
+ fieldPointers = ArrayUtil.grow(fieldPointers);
+ }
+ fieldNumbers[numVectorFields] = fieldNumber;
+ fieldPointers[numVectorFields] = tvf.getFilePointer();
+ numVectorFields++;
+ }
+
+ public long sizeInBytes() {
+ return tvf.sizeInBytes();
+ }
+
+ public void finish() throws IOException {
+ finishDocument(this);
+ }
+ }
+
+ synchronized void files(Collection files) throws IOException {
+ // Vectors:
+ if (tvx != null) {
+ assert docWriter.docStoreSegment != null;
+ files.add(docWriter.docStoreSegment + "." + IndexFileNames.VECTORS_INDEX_EXTENSION);
+ files.add(docWriter.docStoreSegment + "." + IndexFileNames.VECTORS_FIELDS_EXTENSION);
+ files.add(docWriter.docStoreSegment + "." + IndexFileNames.VECTORS_DOCUMENTS_EXTENSION);
+ }
+ }
+
+ static final class PostingList extends RawPostingList {
+ int freq; // How many times this term occurred in the current doc
+ int lastOffset; // Last offset we saw
+ int lastPosition; // Last position where this term occurred
+ }
+
+ int bytesPerPosting() {
+ return RawPostingList.BYTES_SIZE + 3 * DocumentsWriter.INT_NUM_BYTE;
+ }
+}
Property changes on: src/java/org/apache/lucene/index/TermVectorsTermsWriter.java
___________________________________________________________________
Name: svn:eol-style
+ native
Index: src/java/org/apache/lucene/index/TermsHashConsumerPerField.java
===================================================================
--- src/java/org/apache/lucene/index/TermsHashConsumerPerField.java (revision 0)
+++ src/java/org/apache/lucene/index/TermsHashConsumerPerField.java (revision 0)
@@ -0,0 +1,37 @@
+package org.apache.lucene.index;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/** Implement this class to plug into the TermsHash
+ * processor, which inverts & stores Tokens into a hash
+ * table and provides an API for writing bytes into
+ * multiple streams for each unique Token. */
+
+import java.io.IOException;
+import org.apache.lucene.document.Fieldable;
+import org.apache.lucene.analysis.Token;
+
+abstract class TermsHashConsumerPerField {
+ abstract boolean start(Fieldable[] fields, int count) throws IOException;
+ abstract void start(Fieldable field) throws IOException;
+ abstract void end(Fieldable field) throws IOException;
+ abstract void finish() throws IOException;
+ abstract void skippingLongTerm(Token t) throws IOException;
+ abstract void newTerm(Token t, RawPostingList p) throws IOException;
+ abstract void addTerm(Token t, RawPostingList p) throws IOException;
+}
Property changes on: src/java/org/apache/lucene/index/TermsHashConsumerPerField.java
___________________________________________________________________
Name: svn:eol-style
+ native
Index: src/java/org/apache/lucene/index/AbortException.java
===================================================================
--- src/java/org/apache/lucene/index/AbortException.java (revision 664673)
+++ src/java/org/apache/lucene/index/AbortException.java (working copy)
@@ -1,29 +0,0 @@
-package org.apache.lucene.index;
-
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-import java.io.IOException;
-
-// Used only internally to DW to call abort "up the stack"
-class AbortException extends IOException {
- public AbortException(Throwable cause, DocumentsWriter docWriter) {
- super();
- initCause(cause);
- docWriter.setAborting();
- }
-}
Index: src/java/org/apache/lucene/index/DocumentsWriterThreadState.java
===================================================================
--- src/java/org/apache/lucene/index/DocumentsWriterThreadState.java (revision 664673)
+++ src/java/org/apache/lucene/index/DocumentsWriterThreadState.java (working copy)
@@ -18,13 +18,6 @@
*/
import java.io.IOException;
-import java.util.List;
-import org.apache.lucene.store.RAMOutputStream;
-import org.apache.lucene.store.IndexOutput;
-import org.apache.lucene.document.Document;
-import org.apache.lucene.document.Fieldable;
-import org.apache.lucene.analysis.Analyzer;
-import org.apache.lucene.util.UnicodeUtil;
/** Used by DocumentsWriter to maintain per-thread state.
* We keep a separate Posting hash and other state for each
@@ -32,693 +25,26 @@
* when writing the segment. */
final class DocumentsWriterThreadState {
- Posting[] postingsFreeList; // Free Posting instances
- int postingsFreeCount;
+ boolean isIdle = true; // false if this is currently in use by a thread
+ int numThreads = 1; // Number of threads that share this instance
+ boolean doFlushAfter; // true if we should flush after processing current doc
+ final DocConsumerPerThread consumer;
+ final DocumentsWriter.DocState docState;
- RAMOutputStream tvfLocal = new RAMOutputStream(); // Term vectors for one doc
- RAMOutputStream fdtLocal = new RAMOutputStream(); // Stored fields for one doc
- FieldsWriter localFieldsWriter; // Fields for one doc
-
- long[] vectorFieldPointers;
- int[] vectorFieldNumbers;
-
- boolean isIdle = true; // Whether we are in use
- int numThreads = 1; // Number of threads that use this instance
-
- int docID; // docID we are now working on
- int numStoredFields; // How many stored fields in current doc
- float docBoost; // Boost for current doc
-
- DocumentsWriterFieldData[] fieldDataArray; // Fields touched by current doc
- int numFieldData; // How many fields in current doc
- int numVectorFields; // How many vector fields in current doc
-
- DocumentsWriterFieldData[] allFieldDataArray = new DocumentsWriterFieldData[10]; // All FieldData instances
- int numAllFieldData;
- DocumentsWriterFieldData[] fieldDataHash; // Hash FieldData instances by field name
- int fieldDataHashMask;
- String maxTermPrefix; // Non-null prefix of a too-large term if this
- // doc has one
-
- boolean doFlushAfter;
-
final DocumentsWriter docWriter;
- final ByteBlockPool postingsPool;
- final ByteBlockPool vectorsPool;
- final CharBlockPool charPool;
-
- public DocumentsWriterThreadState(DocumentsWriter docWriter) {
+ public DocumentsWriterThreadState(DocumentsWriter docWriter) throws IOException {
this.docWriter = docWriter;
- fieldDataArray = new DocumentsWriterFieldData[8];
-
- fieldDataHash = new DocumentsWriterFieldData[16];
- fieldDataHashMask = 15;
-
- vectorFieldPointers = new long[10];
- vectorFieldNumbers = new int[10];
- postingsFreeList = new Posting[256];
- postingsFreeCount = 0;
-
- postingsPool = new ByteBlockPool(docWriter.byteBlockAllocator, true);
- vectorsPool = new ByteBlockPool(docWriter.byteBlockAllocator, false);
- charPool = new CharBlockPool(docWriter);
+ docState = new DocumentsWriter.DocState();
+ docState.maxFieldLength = docWriter.maxFieldLength;
+ docState.infoStream = docWriter.infoStream;
+ docState.similarity = docWriter.similarity;
+ docState.docWriter = docWriter;
+ consumer = docWriter.consumer.addThread(this);
}
- /** Clear the postings hash and return objects back to
- * shared pool */
- public void resetPostings() throws IOException {
- fieldGen = 0;
- maxPostingsVectors = 0;
+ void doAfterFlush() {
+ numThreads = 0;
doFlushAfter = false;
- if (localFieldsWriter != null) {
- localFieldsWriter.close();
- localFieldsWriter = null;
- }
- postingsPool.reset();
- charPool.reset();
- docWriter.recyclePostings(postingsFreeList, postingsFreeCount);
- postingsFreeCount = 0;
- for(int i=0;i<numAllFieldData;i++) {
- DocumentsWriterFieldData fp = allFieldDataArray[i];
- fp.lastGen = -1;
- if (fp.numPostings > 0)
- fp.resetPostingArrays();
- }
}
-
- /** Move all per-document state that was accumulated in
- * the ThreadState into the "real" stores. */
- public void writeDocument() throws IOException, AbortException {
-
- // If we hit an exception while appending to the
- // stored fields or term vectors files, we have to
- // abort all documents since we last flushed because
- // it means those files are possibly inconsistent.
- try {
-
- docWriter.numDocsInStore++;
-
- // Append stored fields to the real FieldsWriter:
- docWriter.fieldsWriter.flushDocument(numStoredFields, fdtLocal);
- fdtLocal.reset();
-
- // Append term vectors to the real outputs:
- final IndexOutput tvx = docWriter.tvx;
- final IndexOutput tvd = docWriter.tvd;
- final IndexOutput tvf = docWriter.tvf;
- if (tvx != null) {
- tvx.writeLong(tvd.getFilePointer());
- tvx.writeLong(tvf.getFilePointer());
- tvd.writeVInt(numVectorFields);
- if (numVectorFields > 0) {
- for(int i=0;i<numVectorFields;i++)
- tvd.writeVInt(vectorFieldNumbers[i]);
- assert 0 == vectorFieldPointers[0];
- long lastPos = vectorFieldPointers[0];
- for(int i=1;i<numVectorFields;i++) {
- long pos = vectorFieldPointers[i];
- tvd.writeVLong(pos-lastPos);
- lastPos = pos;
- }
- tvfLocal.writeTo(tvf);
- tvfLocal.reset();
- }
- }
-
- // Append norms for the fields we saw:
- for(int i=0;i<numFieldData;i++) {
- DocumentsWriterFieldData fp = fieldDataArray[i];
- if (fp.doNorms) {
- BufferedNorms bn = docWriter.norms[fp.fieldInfo.number];
- assert bn != null;
- assert bn.upto <= docID;
- bn.fill(docID);
- float norm = fp.boost * docWriter.writer.getSimilarity().lengthNorm(fp.fieldInfo.name, fp.length);
- bn.add(norm);
- }
- }
- } catch (Throwable t) {
- // Forcefully idle this threadstate -- its state will
- // be reset by abort()
- isIdle = true;
- throw new AbortException(t, docWriter);
- }
-
- if (docWriter.bufferIsFull && !docWriter.flushPending) {
- docWriter.flushPending = true;
- doFlushAfter = true;
- }
- }
-
- int fieldGen;
-
- /** Initializes shared state for this new document */
- void init(Document doc, int docID) throws IOException, AbortException {
-
- assert !isIdle;
- assert docWriter.writer.testPoint("DocumentsWriter.ThreadState.init start");
-
- this.docID = docID;
- docBoost = doc.getBoost();
- numStoredFields = 0;
- numFieldData = 0;
- numVectorFields = 0;
- maxTermPrefix = null;
-
- assert 0 == fdtLocal.length();
- assert 0 == fdtLocal.getFilePointer();
- assert 0 == tvfLocal.length();
- assert 0 == tvfLocal.getFilePointer();
- final int thisFieldGen = fieldGen++;
-
- List docFields = doc.getFields();
- final int numDocFields = docFields.size();
- boolean docHasVectors = false;
-
- // Absorb any new fields first seen in this document.
- // Also absorb any changes to fields we had already
- // seen before (eg suddenly turning on norms or
- // vectors, etc.):
-
- for(int i=0;i<numDocFields;i++) {
- Fieldable field = (Fieldable) docFields.get(i);
-
- FieldInfo fi = docWriter.fieldInfos.add(field.name(), field.isIndexed(), field.isTermVectorStored(),
- field.isStorePositionWithTermVector(), field.isStoreOffsetWithTermVector(),
- field.getOmitNorms(), false);
- if (fi.isIndexed && !fi.omitNorms) {
- // Maybe grow our buffered norms
- if (docWriter.norms.length <= fi.number) {
- int newSize = (int) ((1+fi.number)*1.25);
- BufferedNorms[] newNorms = new BufferedNorms[newSize];
- System.arraycopy(docWriter.norms, 0, newNorms, 0, docWriter.norms.length);
- docWriter.norms = newNorms;
- }
-
- if (docWriter.norms[fi.number] == null)
- docWriter.norms[fi.number] = new BufferedNorms();
-
- docWriter.hasNorms = true;
- }
-
- // Make sure we have a FieldData allocated
- int hashPos = fi.name.hashCode() & fieldDataHashMask;
- DocumentsWriterFieldData fp = fieldDataHash[hashPos];
- while(fp != null && !fp.fieldInfo.name.equals(fi.name))
- fp = fp.next;
-
- if (fp == null) {
-
- fp = new DocumentsWriterFieldData(this, fi);
- fp.next = fieldDataHash[hashPos];
- fieldDataHash[hashPos] = fp;
-
- if (numAllFieldData == allFieldDataArray.length) {
- int newSize = (int) (allFieldDataArray.length*1.5);
- int newHashSize = fieldDataHash.length*2;
-
- DocumentsWriterFieldData newArray[] = new DocumentsWriterFieldData[newSize];
- DocumentsWriterFieldData newHashArray[] = new DocumentsWriterFieldData[newHashSize];
- System.arraycopy(allFieldDataArray, 0, newArray, 0, numAllFieldData);
-
- // Rehash
- fieldDataHashMask = newSize-1;
- for(int j=0;j<fieldDataHash.length;j++) {
- DocumentsWriterFieldData fp0 = fieldDataHash[j];
- while(fp0 != null) {
- hashPos = fp0.fieldInfo.name.hashCode() & fieldDataHashMask;
- DocumentsWriterFieldData nextFP0 = fp0.next;
- fp0.next = newHashArray[hashPos];
- newHashArray[hashPos] = fp0;
- fp0 = nextFP0;
- }
- }
-
- allFieldDataArray = newArray;
- fieldDataHash = newHashArray;
- }
- allFieldDataArray[numAllFieldData++] = fp;
- } else {
- assert fp.fieldInfo == fi;
- }
-
- if (thisFieldGen != fp.lastGen) {
-
- // First time we're seeing this field for this doc
- fp.lastGen = thisFieldGen;
- fp.fieldCount = 0;
- fp.doVectors = fp.doVectorPositions = fp.doVectorOffsets = false;
- fp.doNorms = fi.isIndexed && !fi.omitNorms;
-
- if (numFieldData == fieldDataArray.length) {
- int newSize = fieldDataArray.length*2;
- DocumentsWriterFieldData newArray[] = new DocumentsWriterFieldData[newSize];
- System.arraycopy(fieldDataArray, 0, newArray, 0, numFieldData);
- fieldDataArray = newArray;
-
- }
- fieldDataArray[numFieldData++] = fp;
- }
-
- if (field.isTermVectorStored()) {
- if (!fp.doVectors && numVectorFields++ == vectorFieldPointers.length) {
- final int newSize = (int) (numVectorFields*1.5);
- vectorFieldPointers = new long[newSize];
- vectorFieldNumbers = new int[newSize];
- }
- fp.doVectors = true;
- docHasVectors = true;
-
- fp.doVectorPositions |= field.isStorePositionWithTermVector();
- fp.doVectorOffsets |= field.isStoreOffsetWithTermVector();
- }
-
- if (fp.fieldCount == fp.docFields.length) {
- Fieldable[] newArray = new Fieldable[fp.docFields.length*2];
- System.arraycopy(fp.docFields, 0, newArray, 0, fp.docFields.length);
- fp.docFields = newArray;
- }
-
- // Lazily allocate arrays for postings:
- if (field.isIndexed() && fp.postingsHash == null)
- fp.initPostingArrays();
-
- fp.docFields[fp.fieldCount++] = field;
- }
-
- // Maybe init the local & global fieldsWriter
- if (localFieldsWriter == null) {
- if (docWriter.fieldsWriter == null) {
- assert docWriter.docStoreSegment == null;
- assert docWriter.segment != null;
- docWriter.files = null;
- docWriter.docStoreSegment = docWriter.segment;
- // If we hit an exception while init'ing the
- // fieldsWriter, we must abort this segment
- // because those files will be in an unknown
- // state:
- try {
- docWriter.fieldsWriter = new FieldsWriter(docWriter.directory, docWriter.docStoreSegment, docWriter.fieldInfos);
- } catch (Throwable t) {
- throw new AbortException(t, docWriter);
- }
- }
- localFieldsWriter = new FieldsWriter(null, fdtLocal, docWriter.fieldInfos);
- }
-
- // First time we see a doc that has field(s) with
- // stored vectors, we init our tvx writer
- if (docHasVectors) {
- if (docWriter.tvx == null) {
- assert docWriter.docStoreSegment != null;
- docWriter.files = null;
- // If we hit an exception while init'ing the term
- // vector output files, we must abort this segment
- // because those files will be in an unknown
- // state:
- try {
- docWriter.tvx = docWriter.directory.createOutput(docWriter.docStoreSegment + "." + IndexFileNames.VECTORS_INDEX_EXTENSION);
- docWriter.tvx.writeInt(TermVectorsReader.FORMAT_CURRENT);
- docWriter.tvd = docWriter.directory.createOutput(docWriter.docStoreSegment + "." + IndexFileNames.VECTORS_DOCUMENTS_EXTENSION);
- docWriter.tvd.writeInt(TermVectorsReader.FORMAT_CURRENT);
- docWriter.tvf = docWriter.directory.createOutput(docWriter.docStoreSegment + "." + IndexFileNames.VECTORS_FIELDS_EXTENSION);
- docWriter.tvf.writeInt(TermVectorsReader.FORMAT_CURRENT);
-
- // We must "catch up" for all docs before us
- // that had no vectors:
- for(int i=0;i<docWriter.numDocsInStore;i++) {
- docWriter.tvx.writeLong(docWriter.tvd.getFilePointer());
- docWriter.tvd.writeVInt(0);
- docWriter.tvx.writeLong(0);
- }
- } catch (Throwable t) {
- throw new AbortException(t, docWriter);
- }
- }
- numVectorFields = 0;
- }
- }
-
- /** Do in-place sort of Posting array */
- void doPostingSort(Posting[] postings, int numPosting) {
- quickSort(postings, 0, numPosting-1);
- }
-
- void quickSort(Posting[] postings, int lo, int hi) {
- if (lo >= hi)
- return;
- else if (hi == 1+lo) {
- if (comparePostings(postings[lo], postings[hi]) > 0) {
- final Posting tmp = postings[lo];
- postings[lo] = postings[hi];
- postings[hi] = tmp;
- }
- return;
- }
-
- int mid = (lo + hi) >>> 1;
-
- if (comparePostings(postings[lo], postings[mid]) > 0) {
- Posting tmp = postings[lo];
- postings[lo] = postings[mid];
- postings[mid] = tmp;
- }
-
- if (comparePostings(postings[mid], postings[hi]) > 0) {
- Posting tmp = postings[mid];
- postings[mid] = postings[hi];
- postings[hi] = tmp;
-
- if (comparePostings(postings[lo], postings[mid]) > 0) {
- Posting tmp2 = postings[lo];
- postings[lo] = postings[mid];
- postings[mid] = tmp2;
- }
- }
-
- int left = lo + 1;
- int right = hi - 1;
-
- if (left >= right)
- return;
-
- Posting partition = postings[mid];
-
- for (; ;) {
- while (comparePostings(postings[right], partition) > 0)
- --right;
-
- while (left < right && comparePostings(postings[left], partition) <= 0)
- ++left;
-
- if (left < right) {
- Posting tmp = postings[left];
- postings[left] = postings[right];
- postings[right] = tmp;
- --right;
- } else {
- break;
- }
- }
-
- quickSort(postings, lo, left);
- quickSort(postings, left + 1, hi);
- }
-
- /** Do in-place sort of PostingVector array */
- void doVectorSort(PostingVector[] postings, int numPosting) {
- quickSort(postings, 0, numPosting-1);
- }
-
- void quickSort(PostingVector[] postings, int lo, int hi) {
- if (lo >= hi)
- return;
- else if (hi == 1+lo) {
- if (comparePostings(postings[lo].p, postings[hi].p) > 0) {
- final PostingVector tmp = postings[lo];
- postings[lo] = postings[hi];
- postings[hi] = tmp;
- }
- return;
- }
-
- int mid = (lo + hi) >>> 1;
-
- if (comparePostings(postings[lo].p, postings[mid].p) > 0) {
- PostingVector tmp = postings[lo];
- postings[lo] = postings[mid];
- postings[mid] = tmp;
- }
-
- if (comparePostings(postings[mid].p, postings[hi].p) > 0) {
- PostingVector tmp = postings[mid];
- postings[mid] = postings[hi];
- postings[hi] = tmp;
-
- if (comparePostings(postings[lo].p, postings[mid].p) > 0) {
- PostingVector tmp2 = postings[lo];
- postings[lo] = postings[mid];
- postings[mid] = tmp2;
- }
- }
-
- int left = lo + 1;
- int right = hi - 1;
-
- if (left >= right)
- return;
-
- PostingVector partition = postings[mid];
-
- for (; ;) {
- while (comparePostings(postings[right].p, partition.p) > 0)
- --right;
-
- while (left < right && comparePostings(postings[left].p, partition.p) <= 0)
- ++left;
-
- if (left < right) {
- PostingVector tmp = postings[left];
- postings[left] = postings[right];
- postings[right] = tmp;
- --right;
- } else {
- break;
- }
- }
-
- quickSort(postings, lo, left);
- quickSort(postings, left + 1, hi);
- }
-
- void quickSort(DocumentsWriterFieldData[] array, int lo, int hi) {
- if (lo >= hi)
- return;
- else if (hi == 1+lo) {
- if (array[lo].compareTo(array[hi]) > 0) {
- final DocumentsWriterFieldData tmp = array[lo];
- array[lo] = array[hi];
- array[hi] = tmp;
- }
- return;
- }
-
- int mid = (lo + hi) >>> 1;
-
- if (array[lo].compareTo(array[mid]) > 0) {
- DocumentsWriterFieldData tmp = array[lo];
- array[lo] = array[mid];
- array[mid] = tmp;
- }
-
- if (array[mid].compareTo(array[hi]) > 0) {
- DocumentsWriterFieldData tmp = array[mid];
- array[mid] = array[hi];
- array[hi] = tmp;
-
- if (array[lo].compareTo(array[mid]) > 0) {
- DocumentsWriterFieldData tmp2 = array[lo];
- array[lo] = array[mid];
- array[mid] = tmp2;
- }
- }
-
- int left = lo + 1;
- int right = hi - 1;
-
- if (left >= right)
- return;
-
- DocumentsWriterFieldData partition = array[mid];
-
- for (; ;) {
- while (array[right].compareTo(partition) > 0)
- --right;
-
- while (left < right && array[left].compareTo(partition) <= 0)
- ++left;
-
- if (left < right) {
- DocumentsWriterFieldData tmp = array[left];
- array[left] = array[right];
- array[right] = tmp;
- --right;
- } else {
- break;
- }
- }
-
- quickSort(array, lo, left);
- quickSort(array, left + 1, hi);
- }
-
- /** If there are fields we've seen but did not see again
- * in the last run, then free them up. Also reduce
- * postings hash size. */
- void trimFields() {
-
- int upto = 0;
- for(int i=0;i<numAllFieldData;i++) {
- DocumentsWriterFieldData fp = allFieldDataArray[i];
- if (fp.lastGen == -1) {
- // This field was not seen since the previous
- // flush, so, free up its resources now
-
- // Unhash
- final int hashPos = fp.fieldInfo.name.hashCode() & fieldDataHashMask;
- DocumentsWriterFieldData last = null;
- DocumentsWriterFieldData fp0 = fieldDataHash[hashPos];
- while(fp0 != fp) {
- last = fp0;
- fp0 = fp0.next;
- }
-
- if (last == null)
- fieldDataHash[hashPos] = fp.next;
- else
- last.next = fp.next;
-
- if (docWriter.infoStream != null)
- docWriter.infoStream.println(" remove field=" + fp.fieldInfo.name);
-
- } else {
- // Reset
- fp.lastGen = -1;
- allFieldDataArray[upto++] = fp;
-
- if (fp.numPostings > 0 && ((float) fp.numPostings) / fp.postingsHashSize < 0.2) {
- int hashSize = fp.postingsHashSize;
-
- // Reduce hash so it's between 25-50% full
- while (fp.numPostings < (hashSize>>1) && hashSize >= 2)
- hashSize >>= 1;
- hashSize <<= 1;
-
- if (hashSize != fp.postingsHash.length)
- fp.rehashPostings(hashSize);
- }
- }
- }
-
- // If we didn't see any norms for this field since
- // last flush, free it
- for(int i=0;i<docWriter.norms.length;i++) {
- BufferedNorms n = docWriter.norms[i];
- if (n != null && n.upto == 0)
- docWriter.norms[i] = null;
- }
-
- numAllFieldData = upto;
-
- // Also pare back PostingsVectors if it's excessively
- // large
- if (maxPostingsVectors * 1.5 < postingsVectors.length) {
- final int newSize;
- if (0 == maxPostingsVectors)
- newSize = 1;
- else
- newSize = (int) (1.5*maxPostingsVectors);
- PostingVector[] newArray = new PostingVector[newSize];
- System.arraycopy(postingsVectors, 0, newArray, 0, newSize);
- postingsVectors = newArray;
- }
- }
-
- /** Tokenizes the fields of a document into Postings */
- void processDocument(Analyzer analyzer)
- throws IOException, AbortException {
-
- final int numFields = numFieldData;
- assert clearLastVectorFieldName();
-
- assert 0 == fdtLocal.length();
-
- if (docWriter.tvx != null)
- // If we are writing vectors then we must visit
- // fields in sorted order so they are written in
- // sorted order. TODO: we actually only need to
- // sort the subset of fields that have vectors
- // enabled; we could save [small amount of] CPU
- // here.
- quickSort(fieldDataArray, 0, numFields-1);
-
- // We process the document one field at a time
- for(int i=0;i<numFields;i++)
- fieldDataArray[i].processField(analyzer);
-
- if (docWriter.infoStream != null && maxTermPrefix != null)
- docWriter.infoStream.println("WARNING: document contains at least one immense term (longer than the max length " + DocumentsWriter.MAX_TERM_LENGTH + "), all of which were skipped. Please correct the analyzer to not produce such terms. The prefix of the first immense term is: '" + maxTermPrefix + "...'");
- }
-
- // USE ONLY FOR DEBUGGING!
- /*
- public String getPostingText() {
- char[] text = charPool.buffers[p.textStart >> CHAR_BLOCK_SHIFT];
- int upto = p.textStart & CHAR_BLOCK_MASK;
- while(text[upto] != 0xffff)
- upto++;
- return new String(text, p.textStart, upto-(p.textStart & BYTE_BLOCK_MASK));
- }
- */
-
- /** Compares term text for two Posting instance and
- * returns -1 if p1 < p2; 1 if p1 > p2; else 0.
- */
- int comparePostings(Posting p1, Posting p2) {
- if (p1 == p2)
- return 0;
- final char[] text1 = charPool.buffers[p1.textStart >> DocumentsWriter.CHAR_BLOCK_SHIFT];
- int pos1 = p1.textStart & DocumentsWriter.CHAR_BLOCK_MASK;
- final char[] text2 = charPool.buffers[p2.textStart >> DocumentsWriter.CHAR_BLOCK_SHIFT];
- int pos2 = p2.textStart & DocumentsWriter.CHAR_BLOCK_MASK;
-
- assert text1 != text2 || pos1 != pos2;
-
- while(true) {
- final char c1 = text1[pos1++];
- final char c2 = text2[pos2++];
- if (c1 != c2) {
- if (0xffff == c2)
- return 1;
- else if (0xffff == c1)
- return -1;
- else
- return c1-c2;
- } else
- // This method should never compare equal postings
- // unless p1==p2
- assert c1 != 0xffff;
- }
- }
-
- String lastVectorFieldName;
-
- // Called only by assert
- final boolean clearLastVectorFieldName() {
- lastVectorFieldName = null;
- return true;
- }
-
- // Called only by assert
- final boolean vectorFieldsInOrder(FieldInfo fi) {
- try {
- if (lastVectorFieldName != null)
- return lastVectorFieldName.compareTo(fi.name) < 0;
- else
- return true;
- } finally {
- lastVectorFieldName = fi.name;
- }
- }
-
- PostingVector[] postingsVectors = new PostingVector[1];
- int maxPostingsVectors;
-
- // Used to read a string value for a field
- ReusableStringReader stringReader = new ReusableStringReader();
-
- final UnicodeUtil.UTF8Result utf8Results[] = {new UnicodeUtil.UTF8Result(),
- new UnicodeUtil.UTF8Result()};
}
-
Index: src/java/org/apache/lucene/index/RawPostingList.java
===================================================================
--- src/java/org/apache/lucene/index/RawPostingList.java (revision 0)
+++ src/java/org/apache/lucene/index/RawPostingList.java (revision 0)
@@ -0,0 +1,36 @@
+package org.apache.lucene.index;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+/** This is the base class for an in-memory posting list,
+ * keyed by a Token. {@link TermsHash} maintains a hash
+ * table holding one instance of this per unique Token.
+ * Consumers of TermsHash (@link TermsHashConsumer} must
+ * subclass this class with its own concrete class.
+ * {@link FreqProxTermsWriter.RawPostingList} is the
+ * subclass used for the freq/prox postings, and {@link
+ * TermVectorsTermsWriter.PostingList} is the subclass
+ * used to hold TermVectors postings. */
+
+abstract class RawPostingList {
+ final static int BYTES_SIZE = DocumentsWriter.OBJECT_HEADER_BYTES + 3*DocumentsWriter.INT_NUM_BYTE;
+ int textStart;
+ int intStart;
+ int byteStart;
+}
Property changes on: src/java/org/apache/lucene/index/RawPostingList.java
___________________________________________________________________
Name: svn:eol-style
+ native
Index: src/java/org/apache/lucene/index/NormsWriterPerThread.java
===================================================================
--- src/java/org/apache/lucene/index/NormsWriterPerThread.java (revision 0)
+++ src/java/org/apache/lucene/index/NormsWriterPerThread.java (revision 0)
@@ -0,0 +1,41 @@
+package org.apache.lucene.index;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+class NormsWriterPerThread extends InvertedDocEndConsumerPerThread {
+ final NormsWriter normsWriter;
+ final DocumentsWriter.DocState docState;
+
+ public NormsWriterPerThread(DocInverterPerThread docInverterPerThread, NormsWriter normsWriter) {
+ this.normsWriter = normsWriter;
+ docState = docInverterPerThread.docState;
+ }
+
+ InvertedDocEndConsumerPerField addField(DocInverterPerField docInverterPerField, final FieldInfo fieldInfo) {
+ return new NormsWriterPerField(docInverterPerField, this, fieldInfo);
+ }
+
+ void abort() {}
+
+ void startDocument() {}
+ void finishDocument() {}
+
+ boolean freeRAM() {
+ return false;
+ }
+}
Property changes on: src/java/org/apache/lucene/index/NormsWriterPerThread.java
___________________________________________________________________
Name: svn:eol-style
+ native
Index: src/java/org/apache/lucene/index/TermsHashPerThread.java
===================================================================
--- src/java/org/apache/lucene/index/TermsHashPerThread.java (revision 0)
+++ src/java/org/apache/lucene/index/TermsHashPerThread.java (revision 0)
@@ -0,0 +1,115 @@
+package org.apache.lucene.index;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+
+class TermsHashPerThread extends InvertedDocConsumerPerThread {
+
+ final TermsHash termsHash;
+ final TermsHashConsumerPerThread consumer;
+ final TermsHashPerThread nextPerThread;
+
+ final CharBlockPool charPool;
+ final IntBlockPool intPool;
+ final ByteBlockPool bytePool;
+ final boolean primary;
+ final DocumentsWriter.DocState docState;
+
+ final RawPostingList freePostings[] = new RawPostingList[256];
+ int freePostingsCount;
+
+ public TermsHashPerThread(DocInverterPerThread docInverterPerThread, final TermsHash termsHash, final TermsHash nextTermsHash, final TermsHashPerThread primaryPerThread) {
+ docState = docInverterPerThread.docState;
+
+ this.termsHash = termsHash;
+ this.consumer = termsHash.consumer.addThread(this);
+
+ if (nextTermsHash != null) {
+ // We are primary
+ charPool = new CharBlockPool(termsHash.docWriter);
+ primary = true;
+ } else {
+ charPool = primaryPerThread.charPool;
+ primary = false;
+ }
+
+ intPool = new IntBlockPool(termsHash.docWriter, termsHash.trackAllocations);
+ bytePool = new ByteBlockPool(termsHash.docWriter.byteBlockAllocator, termsHash.trackAllocations);
+
+ if (nextTermsHash != null)
+ nextPerThread = nextTermsHash.addThread(docInverterPerThread, this);
+ else
+ nextPerThread = null;
+ }
+
+ InvertedDocConsumerPerField addField(DocInverterPerField docInverterPerField, final FieldInfo fieldInfo) {
+ return new TermsHashPerField(docInverterPerField, this, nextPerThread, fieldInfo);
+ }
+
+ public void abort() {
+ reset(true);
+ consumer.abort();
+ if (nextPerThread != null)
+ nextPerThread.abort();
+ }
+
+ // perField calls this when it needs more postings:
+ void morePostings() throws IOException {
+ termsHash.getPostings(freePostings);
+ freePostingsCount = freePostings.length;
+ for(int i=0;i<freePostingsCount;i++)
+ assert freePostings[i] != null;
+ }
+
+ public void startDocument() throws IOException {
+ consumer.startDocument();
+ if (nextPerThread != null)
+ nextPerThread.consumer.startDocument();
+ }
+
+ public DocumentsWriter.DocWriter finishDocument() throws IOException {
+ final DocumentsWriter.DocWriter doc = consumer.finishDocument();
+
+ final DocumentsWriter.DocWriter doc2;
+ if (nextPerThread != null)
+ doc2 = nextPerThread.consumer.finishDocument();
+ else
+ doc2 = null;
+ if (doc == null)
+ return doc2;
+ else {
+ doc.setNext(doc2);
+ return doc;
+ }
+ }
+
+ // Clear all state
+ void reset(boolean recyclePostings) {
+ intPool.reset();
+ bytePool.reset();
+
+ if (primary)
+ charPool.reset();
+
+ if (recyclePostings) {
+ termsHash.recyclePostings(freePostings, freePostingsCount);
+ freePostingsCount = 0;
+ }
+ }
+}
Property changes on: src/java/org/apache/lucene/index/TermsHashPerThread.java
___________________________________________________________________
Name: svn:eol-style
+ native
Index: src/java/org/apache/lucene/index/DocumentsWriter.java
===================================================================
--- src/java/org/apache/lucene/index/DocumentsWriter.java (revision 664673)
+++ src/java/org/apache/lucene/index/DocumentsWriter.java (working copy)
@@ -25,21 +25,19 @@
import org.apache.lucene.search.Scorer;
import org.apache.lucene.search.Weight;
import org.apache.lucene.store.Directory;
-import org.apache.lucene.store.IndexOutput;
-import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.AlreadyClosedException;
-import org.apache.lucene.util.UnicodeUtil;
+import org.apache.lucene.util.ArrayUtil;
import java.io.IOException;
import java.io.PrintStream;
-import java.util.Arrays;
+import java.util.Collection;
import java.util.Iterator;
import java.util.List;
import java.util.HashMap;
+import java.util.HashSet;
import java.util.ArrayList;
import java.util.Map.Entry;
import java.text.NumberFormat;
-import java.util.Collections;
/**
* This class accepts multiple added documents and directly
@@ -48,32 +46,24 @@
* (with DocumentWriter) and doing standard merges on those
* segments.
*
- * When a document is added, its stored fields (if any) and
- * term vectors (if any) are immediately written to the
- * Directory (ie these do not consume RAM). The freq/prox
- * postings are accumulated into a Postings hash table keyed
- * by term. Each entry in this hash table holds a separate
- * byte stream (allocated as incrementally growing slices
- * into large shared byte[] arrays) for freq and prox, that
- * contains the postings data for multiple documents. If
- * vectors are enabled, each unique term for each document
- * also allocates a PostingVector instance to similarly
- * track the offsets & positions byte stream.
+ * Each added document is passed to the {@link DocConsumer},
+ * which in turn processes the document and interacts with
+ * other consumers in the indexing chain. Certain
+ * consumers, like {@link StoredFieldsWriter} and {@link
+ * TermVectorsTermsWriter}, digest a document and
+ * immediately write bytes to the "doc store" files (ie,
+ * they do not consume RAM per document, except while they
+ * are processing the document).
*
- * Once the Postings hash is full (ie is consuming the
- * allowed RAM) or the number of added docs is large enough
- * (in the case we are flushing by doc count instead of RAM
- * usage), we create a real segment and flush it to disk and
- * reset the Postings hash.
+ * Other consumers, eg {@link FreqProxTermsWriter} and
+ * {@link NormsWriter}, buffer bytes in RAM and flush only
+ * when a new segment is produced.
+
+ * Once we have used our allowed RAM buffer, or the number
+ * of added docs is large enough (in the case we are
+ * flushing by doc count instead of RAM usage), we create a
+ * real segment and flush it to the Directory.
*
- * In adding a document we first organize all of its fields
- * by field name. We then process field by field, and
- * record the Posting hash per-field. After each field we
- * flush its term vectors. When it's time to flush the full
- * segment we first sort the fields by name, and then go
- * field by field and sorts its postings.
- *
- *
* Threads:
*
* Multiple threads are allowed into addDocument at once.
@@ -88,12 +78,6 @@
* call). Finally the synchronized "finishDocument" is
* called to flush changes to the directory.
*
- * Each ThreadState instance has its own Posting hash. Once
- * we're using too much RAM, we flush all Posting hashes to
- * a segment by merging the docIDs in the posting lists for
- * the same term across multiple thread states (see
- * writeSegment and appendPostings).
- *
* When flush is called by IndexWriter, or, we flush
* internally when autoCommit=false, we forcefully idle all
* threads and flush only once they are all idle. This
@@ -128,10 +112,6 @@
IndexWriter writer;
Directory directory;
- FieldInfos fieldInfos = new FieldInfos(); // All fields we've seen
- IndexOutput tvx, tvf, tvd; // To write term vectors
- FieldsWriter fieldsWriter; // To write stored fields
-
String segment; // Current segment we are working on
String docStoreSegment; // Current doc-store segment we are writing
private int docStoreOffset; // Current starting doc-store offset of current segment
@@ -139,27 +119,72 @@
private int nextDocID; // Next docID to be added
private int numDocsInRAM; // # docs buffered in RAM
int numDocsInStore; // # docs written to doc stores
- private int nextWriteDocID; // Next docID to be written
// Max # ThreadState instances; if there are more threads
// than this they share ThreadStates
private final static int MAX_THREAD_STATE = 5;
private DocumentsWriterThreadState[] threadStates = new DocumentsWriterThreadState[0];
private final HashMap threadBindings = new HashMap();
- private int numWaiting;
- private final DocumentsWriterThreadState[] waitingThreadStates = new DocumentsWriterThreadState[MAX_THREAD_STATE];
+
private int pauseThreads; // Non-zero when we need all threads to
// pause (eg to flush)
boolean flushPending; // True when a thread has decided to flush
boolean bufferIsFull; // True when it's time to write segment
- private int abortCount; // Non-zero while abort is pending or running
+ private boolean aborting; // True if an abort is pending
PrintStream infoStream;
+ int maxFieldLength = IndexWriter.DEFAULT_MAX_FIELD_LENGTH;
+ Similarity similarity;
- boolean hasNorms; // Whether any norms were seen since last flush
-
List newFiles;
+ static class DocState {
+ DocumentsWriter docWriter;
+ Analyzer analyzer;
+ int maxFieldLength;
+ PrintStream infoStream;
+ Similarity similarity;
+ int docID;
+ Document doc;
+ String maxTermPrefix;
+
+ // Only called by asserts
+ public boolean testPoint(String name) {
+ return docWriter.writer.testPoint(name);
+ }
+ }
+
+ static class FlushState {
+ DocumentsWriter docWriter;
+ Directory directory;
+ String segmentName;
+ String docStoreSegmentName;
+ int numDocsInRAM;
+ int numDocsInStore;
+ Collection flushedFiles;
+
+ public String segmentFileName(String ext) {
+ return segmentName + "." + ext;
+ }
+ }
+
+ /** Consumer returns this on each doc. This holds any
+ * state that must be flushed synchronized "in docID
+ * order". We gather these and flush them in order. */
+ abstract static class DocWriter {
+ DocWriter next;
+ int docID;
+ abstract void finish() throws IOException;
+ abstract void abort();
+ abstract long sizeInBytes();
+
+ void setNext(DocWriter next) {
+ this.next = next;
+ }
+ };
+
+ final DocConsumer consumer;
+
// Deletes done after the last flush; these are discarded
// on abort
private BufferedDeletes deletesInRAM = new BufferedDeletes();
@@ -176,7 +201,12 @@
// we are flushing by doc count instead.
private long ramBufferSize = (long) (IndexWriter.DEFAULT_RAM_BUFFER_SIZE_MB*1024*1024);
- // Flush @ this number of docs. If rarmBufferSize is
+ // If we've allocated 5% over our RAM budget, we then
+ // free down to 95%
+ private long freeTrigger = (long) (IndexWriter.DEFAULT_RAM_BUFFER_SIZE_MB*1024*1024*1.05);
+ private long freeLevel = (long) (IndexWriter.DEFAULT_RAM_BUFFER_SIZE_MB*1024*1024*0.95);
+
+ // Flush @ this number of docs. If ramBufferSize is
// non-zero we will flush by RAM usage instead.
private int maxBufferedDocs = IndexWriter.DEFAULT_MAX_BUFFERED_DOCS;
@@ -194,34 +224,58 @@
private boolean closed;
- // Coarse estimates used to measure RAM usage of buffered deletes
- private static int OBJECT_HEADER_BYTES = 8;
-
- BufferedNorms[] norms = new BufferedNorms[0]; // Holds norms until we flush
-
DocumentsWriter(Directory directory, IndexWriter writer) throws IOException {
this.directory = directory;
this.writer = writer;
- flushedDocCount = writer.docCount();
- postingsFreeList = new Posting[0];
+ this.similarity = writer.getSimilarity();
+ flushedDocCount = writer.maxDoc();
+
+ // TODO FI: this should be something the user can pass in
+ // Build up indexing chain:
+ final TermsHashConsumer termVectorsWriter = new TermVectorsTermsWriter(this);
+ final TermsHashConsumer freqProxWriter = new FreqProxTermsWriter();
+
+ final InvertedDocConsumer termsHash = new TermsHash(this, true, freqProxWriter,
+ new TermsHash(this, false, termVectorsWriter, null));
+ final NormsWriter normsWriter = new NormsWriter();
+ final DocInverter docInverter = new DocInverter(termsHash, normsWriter);
+ final StoredFieldsWriter fieldsWriter = new StoredFieldsWriter(this);
+ final DocFieldConsumers docFieldConsumers = new DocFieldConsumers(docInverter, fieldsWriter);
+ consumer = new DocFieldProcessor(this, docFieldConsumers);
}
/** If non-null, various details of indexing are printed
* here. */
- void setInfoStream(PrintStream infoStream) {
+ synchronized void setInfoStream(PrintStream infoStream) {
this.infoStream = infoStream;
+ for(int i=0;i<threadStates.length;i++)
+ threadStates[i].docState.infoStream = infoStream;
}
+ synchronized void setMaxFieldLength(int maxFieldLength) {
+ this.maxFieldLength = maxFieldLength;
+ for(int i=0;i<threadStates.length;i++)
+ threadStates[i].docState.maxFieldLength = maxFieldLength;
+ }
+
+ synchronized void setSimilarity(Similarity similarity) {
+ this.similarity = similarity;
+ for(int i=0;i<threadStates.length;i++)
+ threadStates[i].docState.similarity = similarity;
+ }
+
/** Set how much RAM we can use before flushing. */
- void setRAMBufferSizeMB(double mb) {
+ synchronized void setRAMBufferSizeMB(double mb) {
if (mb == IndexWriter.DISABLE_AUTO_FLUSH) {
ramBufferSize = IndexWriter.DISABLE_AUTO_FLUSH;
} else {
ramBufferSize = (long) (mb*1024*1024);
+ freeTrigger = (long) (1.05 * ramBufferSize);
+ freeLevel = (long) (0.95 * ramBufferSize);
}
}
- double getRAMBufferSizeMB() {
+ synchronized double getRAMBufferSizeMB() {
if (ramBufferSize == IndexWriter.DISABLE_AUTO_FLUSH) {
return ramBufferSize;
} else {
@@ -265,51 +319,40 @@
/** Closes the current open doc stores an returns the doc
* store segment name. This returns null if there are *
* no buffered documents. */
- String closeDocStore() throws IOException {
-
+ synchronized String closeDocStore() throws IOException {
+
assert allThreadsIdle();
- List flushedFiles = files();
-
if (infoStream != null)
- message("closeDocStore: " + flushedFiles.size() + " files to flush to segment " + docStoreSegment + " numDocs=" + numDocsInStore);
+ message("closeDocStore: " + openFiles.size() + " files to flush to segment " + docStoreSegment + " numDocs=" + numDocsInStore);
+
+ boolean success = false;
- if (flushedFiles.size() > 0) {
- files = null;
+ try {
+ initFlushState();
+ closedFiles.clear();
- if (tvx != null) {
- // At least one doc in this run had term vectors enabled
- assert docStoreSegment != null;
- tvx.close();
- tvf.close();
- tvd.close();
- tvx = null;
- assert 4+numDocsInStore*16 == directory.fileLength(docStoreSegment + "." + IndexFileNames.VECTORS_INDEX_EXTENSION):
- "after flush: tvx size mismatch: " + numDocsInStore + " docs vs " + directory.fileLength(docStoreSegment + "." + IndexFileNames.VECTORS_INDEX_EXTENSION) + " length in bytes of " + docStoreSegment + "." + IndexFileNames.VECTORS_INDEX_EXTENSION;
- }
+ consumer.closeDocStore(flushState);
+ assert 0 == openFiles.size();
+ assert closedFiles.size() > 0;
- if (fieldsWriter != null) {
- assert docStoreSegment != null;
- fieldsWriter.close();
- fieldsWriter = null;
- assert 4+numDocsInStore*8 == directory.fileLength(docStoreSegment + "." + IndexFileNames.FIELDS_INDEX_EXTENSION):
- "after flush: fdx size mismatch: " + numDocsInStore + " docs vs " + directory.fileLength(docStoreSegment + "." + IndexFileNames.FIELDS_INDEX_EXTENSION) + " length in bytes of " + docStoreSegment + "." + IndexFileNames.FIELDS_INDEX_EXTENSION;
- }
-
String s = docStoreSegment;
docStoreSegment = null;
docStoreOffset = 0;
numDocsInStore = 0;
+ success = true;
return s;
- } else {
- return null;
+ } finally {
+ if (!success)
+ abort();
}
}
- List files = null; // Cached list of files we've created
- private List abortedFiles = null; // List of files that were written before last abort()
+ private Collection abortedFiles; // List of files that were written before last abort()
- List abortedFiles() {
+ private FlushState flushState;
+
+ Collection abortedFiles() {
return abortedFiles;
}
@@ -317,73 +360,62 @@
writer.message("DW: " + message);
}
- /* Returns list of files in use by this instance,
+ final List openFiles = new ArrayList();
+ final List closedFiles = new ArrayList();
+
+ /* Returns Collection of files in use by this instance,
* including any flushed segments. */
- synchronized List files() {
+ synchronized List openFiles() {
+ return (List) ((ArrayList) openFiles).clone();
+ }
- if (files != null)
- return files;
+ synchronized List closedFiles() {
+ return (List) ((ArrayList) closedFiles).clone();
+ }
- files = new ArrayList();
+ synchronized void addOpenFile(String name) {
+ assert !openFiles.contains(name);
+ openFiles.add(name);
+ }
- // Stored fields:
- if (fieldsWriter != null) {
- assert docStoreSegment != null;
- files.add(docStoreSegment + "." + IndexFileNames.FIELDS_EXTENSION);
- files.add(docStoreSegment + "." + IndexFileNames.FIELDS_INDEX_EXTENSION);
- }
-
- // Vectors:
- if (tvx != null) {
- assert docStoreSegment != null;
- files.add(docStoreSegment + "." + IndexFileNames.VECTORS_INDEX_EXTENSION);
- files.add(docStoreSegment + "." + IndexFileNames.VECTORS_FIELDS_EXTENSION);
- files.add(docStoreSegment + "." + IndexFileNames.VECTORS_DOCUMENTS_EXTENSION);
- }
-
- return files;
+ synchronized void removeOpenFile(String name) {
+ assert openFiles.contains(name);
+ openFiles.remove(name);
+ closedFiles.add(name);
}
synchronized void setAborting() {
- abortCount++;
+ aborting = true;
}
/** Called if we hit an exception at a bad time (when
* updating the index files) and must discard all
* currently buffered docs. This resets our state,
- * discarding any docs added since last flush. If ae is
- * non-null, it contains the root cause exception (which
- * we re-throw after we are done aborting). */
- synchronized void abort(AbortException ae) throws IOException {
+ * discarding any docs added since last flush. */
+ synchronized void abort() throws IOException {
- // Anywhere that throws an AbortException must first
- // mark aborting to make sure while the exception is
- // unwinding the un-synchronized stack, no thread grabs
- // the corrupt ThreadState that hit the aborting
- // exception:
- assert ae == null || abortCount>0;
-
try {
- if (infoStream != null)
- message("docWriter: now abort");
+ message("docWriter: now abort");
// Forcefully remove waiting ThreadStates from line
- for(int i=0;i<numWaiting;i++)
- waitingThreadStates[i].isIdle = true;
- numWaiting = 0;
+ waitQueue.abort();
// Wait for all other threads to finish with DocumentsWriter:
pauseAllThreads();
- assert 0 == numWaiting;
+ assert 0 == waitQueue.numWaiting;
+ waitQueue.waitingBytes = 0;
+
+ openFiles.clear();
+
try {
deletesInRAM.clear();
try {
- abortedFiles = files();
+ abortedFiles = openFiles();
} catch (Throwable t) {
abortedFiles = null;
}
@@ -391,112 +423,43 @@
docStoreSegment = null;
numDocsInStore = 0;
docStoreOffset = 0;
- files = null;
- // Clear vectors & fields from ThreadStates
- for(int i=0;i<threadStates.length;i++) {
- DocumentsWriterThreadState state = threadStates[i];
- state.tvfLocal.reset();
- state.fdtLocal.reset();
- if (state.localFieldsWriter != null) {
- try {
- state.localFieldsWriter.close();
- } catch (Throwable t) {
- }
- state.localFieldsWriter = null;
- }
- }
-
- // Reset vectors writer
- if (tvx != null) {
+ for(int i=0;i<threadStates.length;i++)
try {
- tvx.close();
+ threadStates[i].consumer.abort();
} catch (Throwable t) {
}
- tvx = null;
- }
- if (tvd != null) {
- try {
- tvd.close();
- } catch (Throwable t) {
- }
- tvd = null;
- }
- if (tvf != null) {
- try {
- tvf.close();
- } catch (Throwable t) {
- }
- tvf = null;
- }
- // Reset fields writer
- if (fieldsWriter != null) {
- try {
- fieldsWriter.close();
- } catch (Throwable t) {
- }
- fieldsWriter = null;
+ try {
+ consumer.abort();
+ } catch (Throwable t) {
}
- // Discard pending norms:
- final int numField = fieldInfos.size();
- for (int i=0;i<numField;i++) {
- FieldInfo fi = fieldInfos.fieldInfo(i);
- if (fi.isIndexed && !fi.omitNorms) {
- BufferedNorms n = norms[i];
- if (n != null)
- try {
- n.reset();
- } catch (Throwable t) {
- }
- }
- }
-
// Reset all postings data
- resetPostingsData();
+ doAfterFlush();
} finally {
resumeAllThreads();
}
-
- // If we have a root cause exception, re-throw it now:
- if (ae != null) {
- Throwable t = ae.getCause();
- if (t instanceof IOException)
- throw (IOException) t;
- else if (t instanceof RuntimeException)
- throw (RuntimeException) t;
- else if (t instanceof Error)
- throw (Error) t;
- else
- // Should not get here
- assert false: "unknown exception: " + t;
- }
} finally {
- if (ae != null)
- abortCount--;
+ aborting = false;
notifyAll();
}
}
/** Reset after a flush */
- private void resetPostingsData() throws IOException {
+ private void doAfterFlush() throws IOException {
// All ThreadStates should be idle when we are called
assert allThreadsIdle();
threadBindings.clear();
+ waitQueue.reset();
segment = null;
numDocsInRAM = 0;
nextDocID = 0;
- nextWriteDocID = 0;
- files = null;
- balanceRAM();
bufferIsFull = false;
flushPending = false;
- for(int i=0;i<threadStates.length;i++) {
- threadStates[i].numThreads = 0;
- threadStates[i].resetPostings();
- }
+ for(int i=0;i<threadStates.length;i++)
+ threadStates[i].doAfterFlush();
numBytesUsed = 0;
}
@@ -510,7 +473,8 @@
Thread.currentThread().interrupt();
}
}
- return abortCount > 0;
+ assert aborting || waitQueue.numWaiting == 0;
+ return aborting;
}
synchronized void resumeAllThreads() {
@@ -527,71 +491,96 @@
return true;
}
+ synchronized private void initFlushState() {
+ initSegmentName();
+
+ if (flushState == null) {
+ flushState = new FlushState();
+ flushState.directory = directory;
+ flushState.docWriter = this;
+ }
+
+ flushState.docStoreSegmentName = docStoreSegment;
+ flushState.segmentName = segment;
+ flushState.numDocsInRAM = numDocsInRAM;
+ flushState.numDocsInStore = numDocsInStore;
+ flushState.flushedFiles = new HashSet();
+ }
+
/** Flush all pending docs to a new segment */
synchronized int flush(boolean closeDocStore) throws IOException {
assert allThreadsIdle();
- if (segment == null)
- // In case we are asked to flush an empty segment
- segment = writer.newSegmentName();
+ assert numDocsInRAM > 0;
- newFiles = new ArrayList();
+ assert nextDocID == numDocsInRAM;
+ assert waitQueue.numWaiting == 0;
+ assert waitQueue.waitingBytes == 0;
+ initFlushState();
+
docStoreOffset = numDocsInStore;
- int docCount;
-
- assert numDocsInRAM > 0;
-
if (infoStream != null)
- message("flush postings as segment " + segment + " numDocs=" + numDocsInRAM);
+ message("flush postings as segment " + flushState.segmentName + " numDocs=" + numDocsInRAM);
boolean success = false;
try {
if (closeDocStore) {
- assert docStoreSegment != null;
- assert docStoreSegment.equals(segment);
- newFiles.addAll(files());
+ assert flushState.docStoreSegmentName != null;
+ assert flushState.docStoreSegmentName.equals(flushState.segmentName);
closeDocStore();
}
-
- fieldInfos.write(directory, segment + ".fnm");
- docCount = numDocsInRAM;
+ Collection threads = new HashSet();
+ for(int i=0;i<threadStates.length;i++)
+ threads.add(threadStates[i].consumer);
+ consumer.flush(threads, flushState);
- newFiles.addAll(writeSegment());
+ if (infoStream != null) {
+ final long newSegmentSize = segmentSize(flushState.segmentName);
+ String message = " oldRAMSize=" + numBytesUsed +
+ " newFlushedSize=" + newSegmentSize +
+ " docs/MB=" + nf.format(numDocsInRAM/(newSegmentSize/1024./1024.)) +
+ " new/old=" + nf.format(100.0*newSegmentSize/numBytesUsed) + "%";
+ message(message);
+ }
- flushedDocCount += docCount;
+ flushedDocCount += flushState.numDocsInRAM;
+ doAfterFlush();
+
success = true;
} finally {
if (!success)
- abort(null);
+ abort();
}
- return docCount;
+ assert waitQueue.waitingBytes == 0;
+
+ return flushState.numDocsInRAM;
}
/** Build compound file for the segment we just flushed */
- void createCompoundFile(String segment) throws IOException
- {
+ void createCompoundFile(String segment) throws IOException {
+
CompoundFileWriter cfsWriter = new CompoundFileWriter(directory, segment + "." + IndexFileNames.COMPOUND_FILE_EXTENSION);
- final int size = newFiles.size();
- for(int i=0;i<size;i++)
- cfsWriter.addFile((String) newFiles.get(i));
+ Iterator it = flushState.flushedFiles.iterator();
+ while(it.hasNext())
+ cfsWriter.addFile((String) it.next());
// Perform the merge
cfsWriter.close();
}
/** Set flushPending if it is not already set and returns
- * whether it was set. This is used by IndexWriter to *
+ * whether it was set. This is used by IndexWriter to
* trigger a single flush even when multiple threads are
- * * trying to do so. */
+ * trying to do so. */
synchronized boolean setFlushPending() {
if (flushPending)
return false;
@@ -605,350 +594,22 @@
flushPending = false;
}
- private static final byte defaultNorm = Similarity.encodeNorm(1.0f);
-
- /** Write norms in the "true" segment format. This is
- * called only during commit, to create the .nrm file. */
- void writeNorms(String segmentName, int totalNumDoc) throws IOException {
-
- IndexOutput normsOut = directory.createOutput(segmentName + "." + IndexFileNames.NORMS_EXTENSION);
-
- try {
- normsOut.writeBytes(SegmentMerger.NORMS_HEADER, 0, SegmentMerger.NORMS_HEADER.length);
-
- final int numField = fieldInfos.size();
-
- for (int fieldIdx=0;fieldIdx<numField;fieldIdx++) {
- FieldInfo fi = fieldInfos.fieldInfo(fieldIdx);
- if (fi.isIndexed && !fi.omitNorms) {
- BufferedNorms n = norms[fieldIdx];
- final long v;
- if (n == null)
- v = 0;
- else {
- v = n.out.getFilePointer();
- n.out.writeTo(normsOut);
- n.reset();
- }
- if (v < totalNumDoc)
- fillBytes(normsOut, defaultNorm, (int) (totalNumDoc-v));
- }
- }
- } finally {
- normsOut.close();
- }
- }
-
- private DefaultSkipListWriter skipListWriter = null;
-
- private boolean currentFieldStorePayloads;
-
- /** Creates a segment from all Postings in the Postings
- * hashes across all ThreadStates & FieldDatas. */
- private List writeSegment() throws IOException {
-
- assert allThreadsIdle();
-
- assert nextDocID == numDocsInRAM;
-
- final String segmentName;
-
- segmentName = segment;
-
- TermInfosWriter termsOut = new TermInfosWriter(directory, segmentName, fieldInfos,
- writer.getTermIndexInterval());
-
- IndexOutput freqOut = directory.createOutput(segmentName + ".frq");
- IndexOutput proxOut = directory.createOutput(segmentName + ".prx");
-
- // Gather all FieldData's that have postings, across all
- // ThreadStates
- ArrayList allFields = new ArrayList();
- assert allThreadsIdle();
- for(int i=0;i<threadStates.length;i++) {
- DocumentsWriterThreadState state = threadStates[i];
- state.trimFields();
- final int numFields = state.numAllFieldData;
- for(int j=0;j<numFields;j++) {
- DocumentsWriterFieldData fp = state.allFieldDataArray[j];
- if (fp.numPostings > 0)
- allFields.add(fp);
- }
- }
-
- // Sort by field name
- Collections.sort(allFields);
- final int numAllFields = allFields.size();
-
- skipListWriter = new DefaultSkipListWriter(termsOut.skipInterval,
- termsOut.maxSkipLevels,
- numDocsInRAM, freqOut, proxOut);
-
- int start = 0;
- while(start < numAllFields) {
-
- final String fieldName = ((DocumentsWriterFieldData) allFields.get(start)).fieldInfo.name;
-
- int end = start+1;
- while(end < numAllFields && ((DocumentsWriterFieldData) allFields.get(end)).fieldInfo.name.equals(fieldName))
- end++;
-
- DocumentsWriterFieldData[] fields = new DocumentsWriterFieldData[end-start];
- for(int i=start;i<end;i++)
- fields[i-start] = (DocumentsWriterFieldData) allFields.get(i);
-
- // If this field has postings then add them to the
- // segment
- appendPostings(fields, termsOut, freqOut, proxOut);
-
- for(int i=0;i<fields.length;i++)
- fields[i].resetPostingArrays();
-
- start = end;
- }
-
- freqOut.close();
- proxOut.close();
- termsOut.close();
-
- // Record all files we have flushed
- List flushedFiles = new ArrayList();
- flushedFiles.add(segmentFileName(IndexFileNames.FIELD_INFOS_EXTENSION));
- flushedFiles.add(segmentFileName(IndexFileNames.FREQ_EXTENSION));
- flushedFiles.add(segmentFileName(IndexFileNames.PROX_EXTENSION));
- flushedFiles.add(segmentFileName(IndexFileNames.TERMS_EXTENSION));
- flushedFiles.add(segmentFileName(IndexFileNames.TERMS_INDEX_EXTENSION));
-
- if (hasNorms) {
- writeNorms(segmentName, numDocsInRAM);
- flushedFiles.add(segmentFileName(IndexFileNames.NORMS_EXTENSION));
- }
-
- if (infoStream != null) {
- final long newSegmentSize = segmentSize(segmentName);
- String message = " oldRAMSize=" + numBytesUsed + " newFlushedSize=" + newSegmentSize + " docs/MB=" + nf.format(numDocsInRAM/(newSegmentSize/1024./1024.)) + " new/old=" + nf.format(100.0*newSegmentSize/numBytesUsed) + "%";
- message(message);
- }
-
- resetPostingsData();
-
- // Maybe downsize postingsFreeList array
- if (postingsFreeList.length > 1.5*postingsFreeCount) {
- int newSize = postingsFreeList.length;
- while(newSize > 1.25*postingsFreeCount) {
- newSize = (int) (newSize*0.8);
- }
- Posting[] newArray = new Posting[newSize];
- System.arraycopy(postingsFreeList, 0, newArray, 0, postingsFreeCount);
- postingsFreeList = newArray;
- }
-
- return flushedFiles;
- }
-
synchronized void pushDeletes() {
deletesFlushed.update(deletesInRAM);
}
- /** Returns the name of the file with this extension, on
- * the current segment we are working on. */
- private String segmentFileName(String extension) {
- return segment + "." + extension;
- }
-
- private static int compareText(final char[] text1, int pos1, final char[] text2, int pos2) {
- while(true) {
- final char c1 = text1[pos1++];
- final char c2 = text2[pos2++];
- if (c1 != c2) {
- if (0xffff == c2)
- return 1;
- else if (0xffff == c1)
- return -1;
- else
- return c1-c2;
- } else if (0xffff == c1)
- return 0;
- }
- }
-
- private final TermInfo termInfo = new TermInfo(); // minimize consing
-
- final UnicodeUtil.UTF8Result termsUTF8 = new UnicodeUtil.UTF8Result();
-
- /* Walk through all unique text tokens (Posting
- * instances) found in this field and serialize them
- * into a single RAM segment. */
- void appendPostings(DocumentsWriterFieldData[] fields,
- TermInfosWriter termsOut,
- IndexOutput freqOut,
- IndexOutput proxOut)
- throws CorruptIndexException, IOException {
-
- final int fieldNumber = fields[0].fieldInfo.number;
- int numFields = fields.length;
-
- final DocumentsWriterFieldMergeState[] mergeStates = new DocumentsWriterFieldMergeState[numFields];
-
- for(int i=0;i<numFields;i++) {
- DocumentsWriterFieldMergeState fms = mergeStates[i] = new DocumentsWriterFieldMergeState();
- fms.field = fields[i];
- fms.postings = fms.field.sortPostings();
-
- assert fms.field.fieldInfo == fields[0].fieldInfo;
-
- // Should always be true
- boolean result = fms.nextTerm();
- assert result;
- }
-
- final int skipInterval = termsOut.skipInterval;
- currentFieldStorePayloads = fields[0].fieldInfo.storePayloads;
-
- DocumentsWriterFieldMergeState[] termStates = new DocumentsWriterFieldMergeState[numFields];
-
- while(numFields > 0) {
-
- // Get the next term to merge
- termStates[0] = mergeStates[0];
- int numToMerge = 1;
-
- for(int i=1;i<numFields;i++) {
- final char[] text = mergeStates[i].text;
- final int textOffset = mergeStates[i].textOffset;
- final int cmp = compareText(text, textOffset, termStates[0].text, termStates[0].textOffset);
-
- if (cmp < 0) {
- termStates[0] = mergeStates[i];
- numToMerge = 1;
- } else if (cmp == 0)
- termStates[numToMerge++] = mergeStates[i];
- }
-
- int df = 0;
- int lastPayloadLength = -1;
-
- int lastDoc = 0;
-
- final char[] text = termStates[0].text;
- final int start = termStates[0].textOffset;
-
- long freqPointer = freqOut.getFilePointer();
- long proxPointer = proxOut.getFilePointer();
-
- skipListWriter.resetSkip();
-
- // Now termStates has numToMerge FieldMergeStates
- // which all share the same term. Now we must
- // interleave the docID streams.
- while(numToMerge > 0) {
-
- if ((++df % skipInterval) == 0) {
- skipListWriter.setSkipData(lastDoc, currentFieldStorePayloads, lastPayloadLength);
- skipListWriter.bufferSkip(df);
- }
-
- DocumentsWriterFieldMergeState minState = termStates[0];
- for(int i=1;i<numToMerge;i++)
- if (termStates[i].docID < minState.docID)
- minState = termStates[i];
-
- final int doc = minState.docID;
- final int termDocFreq = minState.termFreq;
-
- assert doc < numDocsInRAM;
- assert doc > lastDoc || df == 1;
-
- final int newDocCode = (doc-lastDoc)<<1;
- lastDoc = doc;
-
- final ByteSliceReader prox = minState.prox;
-
- // Carefully copy over the prox + payload info,
- // changing the format to match Lucene's segment
- // format.
- for(int j=0;j<termDocFreq;j++) {
- final int code = prox.readVInt();
- if (currentFieldStorePayloads) {
- final int payloadLength;
- if ((code & 1) != 0) {
- // This position has a payload
- payloadLength = prox.readVInt();
- } else
- payloadLength = 0;
- if (payloadLength != lastPayloadLength) {
- proxOut.writeVInt(code|1);
- proxOut.writeVInt(payloadLength);
- lastPayloadLength = payloadLength;
- } else
- proxOut.writeVInt(code & (~1));
- if (payloadLength > 0)
- copyBytes(prox, proxOut, payloadLength);
- } else {
- assert 0 == (code & 1);
- proxOut.writeVInt(code>>1);
- }
- }
-
- if (1 == termDocFreq) {
- freqOut.writeVInt(newDocCode|1);
- } else {
- freqOut.writeVInt(newDocCode);
- freqOut.writeVInt(termDocFreq);
- }
-
- if (!minState.nextDoc()) {
-
- // Remove from termStates
- int upto = 0;
- for(int i=0;i<numToMerge;i++)
- if (termStates[i] != minState)
- termStates[upto++] = termStates[i];
- numToMerge--;
- assert upto == numToMerge;
-
- // Advance this state to the next term
-
- if (!minState.nextTerm()) {
- // OK, no more terms, so remove from mergeStates
- // as well
- upto = 0;
- for(int i=0;i<numFields;i++)
- if (mergeStates[i] != minState)
- mergeStates[upto++] = mergeStates[i];
- numFields--;
- assert upto == numFields;
- }
- }
- }
-
- assert df > 0;
-
- // Done merging this term
-
- long skipPointer = skipListWriter.writeSkip(freqOut);
-
- // Write term
- termInfo.set(df, freqPointer, proxPointer, (int) (skipPointer - freqPointer));
-
- // TODO: we could do this incrementally
- UnicodeUtil.UTF16toUTF8(text, start, termsUTF8);
-
- // TODO: we could save O(n) re-scan of the term by
- // computing the shared prefix with the last term
- // while during the UTF8 encoding
- termsOut.add(fieldNumber,
- termsUTF8.result,
- termsUTF8.length,
- termInfo);
- }
- }
-
synchronized void close() {
closed = true;
notifyAll();
}
+ synchronized void initSegmentName() {
+ if (segment == null)
+ segment = writer.newSegmentName();
+ if (docStoreSegment == null)
+ docStoreSegment = segment;
+ }
+
/** Returns a free (idle) ThreadState that may be used for
* indexing this one document. This call also pauses if a
* flush is pending. If delTerm is non-null then we
@@ -961,14 +622,16 @@
// again.
DocumentsWriterThreadState state = (DocumentsWriterThreadState) threadBindings.get(Thread.currentThread());
if (state == null) {
- // First time this thread has called us since last flush
+
+ // First time this thread has called us since last
+ // flush. Find the least loaded thread state:
DocumentsWriterThreadState minThreadState = null;
for(int i=0;i<threadStates.length;i++) {
DocumentsWriterThreadState ts = threadStates[i];
if (minThreadState == null || ts.numThreads < minThreadState.numThreads)
minThreadState = ts;
}
- if (minThreadState != null && (minThreadState.numThreads == 0 || threadStates.length == MAX_THREAD_STATE)) {
+ if (minThreadState != null && (minThreadState.numThreads == 0 || threadStates.length >= MAX_THREAD_STATE)) {
state = minThreadState;
state.numThreads++;
} else {
@@ -987,46 +650,49 @@
// not be paused nor a flush pending:
waitReady(state);
- if (segment == null)
- segment = writer.newSegmentName();
+ // Allocate segment name if this is the first doc since
+ // last flush:
+ initSegmentName();
state.isIdle = false;
+ boolean success = false;
try {
- boolean success = false;
- try {
- state.init(doc, nextDocID);
- if (delTerm != null) {
- addDeleteTerm(delTerm, state.docID);
- state.doFlushAfter = timeToFlushDeletes();
- }
- // Only increment nextDocID & numDocsInRAM on successful init
- nextDocID++;
- numDocsInRAM++;
+ state.docState.docID = nextDocID;
- // We must at this point commit to flushing to ensure we
- // always get N docs when we flush by doc count, even if
- // > 1 thread is adding documents:
- if (!flushPending && maxBufferedDocs != IndexWriter.DISABLE_AUTO_FLUSH
- && numDocsInRAM >= maxBufferedDocs) {
- flushPending = true;
- state.doFlushAfter = true;
- }
+ assert writer.testPoint("DocumentsWriter.ThreadState.init start");
- success = true;
- } finally {
- if (!success) {
- // Forcefully idle this ThreadState:
- state.isIdle = true;
- notifyAll();
- if (state.doFlushAfter) {
- state.doFlushAfter = false;
- flushPending = false;
- }
+ if (delTerm != null) {
+ addDeleteTerm(delTerm, state.docState.docID);
+ state.doFlushAfter = timeToFlushDeletes();
+ }
+
+ assert writer.testPoint("DocumentsWriter.ThreadState.init after delTerm");
+
+ nextDocID++;
+ numDocsInRAM++;
+
+ // We must at this point commit to flushing to ensure we
+ // always get N docs when we flush by doc count, even if
+ // > 1 thread is adding documents:
+ if (!flushPending &&
+ maxBufferedDocs != IndexWriter.DISABLE_AUTO_FLUSH
+ && numDocsInRAM >= maxBufferedDocs) {
+ flushPending = true;
+ state.doFlushAfter = true;
+ }
+
+ success = true;
+ } finally {
+ if (!success) {
+ // Forcefully idle this ThreadState:
+ state.isIdle = true;
+ notifyAll();
+ if (state.doFlushAfter) {
+ state.doFlushAfter = false;
+ flushPending = false;
}
}
- } catch (AbortException ae) {
- abort(ae);
}
return state;
@@ -1049,26 +715,32 @@
// This call is synchronized but fast
final DocumentsWriterThreadState state = getThreadState(doc, delTerm);
+
+ final DocState docState = state.docState;
+ docState.doc = doc;
+ docState.analyzer = analyzer;
+
+ boolean success = false;
try {
- boolean success = false;
- try {
- try {
- // This call is not synchronized and does all the work
- state.processDocument(analyzer);
- } finally {
- // Note that we must call finishDocument even on
- // exception, because for a non-aborting
- // exception, a portion of the document has been
- // indexed (and its ID is marked for deletion), so
- // all index files must be updated to record this
- // document. This call is synchronized but fast.
- finishDocument(state);
- }
- success = true;
- } finally {
- if (!success) {
- synchronized(this) {
+ // This call is not synchronized and does all the
+ // work
+ final DocWriter perDoc = state.consumer.processDocument();
+
+ // This call is synchronized but fast
+ finishDocument(state, perDoc);
+ success = true;
+ } finally {
+ if (!success) {
+ synchronized(this) {
+ state.isIdle = true;
+ notifyAll();
+ if (aborting)
+ abort();
+ else {
+ skipDocWriter.docID = docState.docID;
+ waitQueue.add(skipDocWriter);
+
// If this thread state had decided to flush, we
// must clear it so another thread can flush
if (state.doFlushAfter) {
@@ -1081,12 +753,10 @@
// since likely it was partially added. This
// keeps indexing as "all or none" (atomic) when
// adding a document:
- addDeleteDocID(state.docID);
+ addDeleteDocID(state.docState.docID);
}
}
}
- } catch (AbortException ae) {
- abort(ae);
}
return state.doFlushAfter || timeToFlushDeletes();
@@ -1114,12 +784,14 @@
}
synchronized private void waitReady(DocumentsWriterThreadState state) {
- while(!closed && ((state != null && !state.isIdle) || pauseThreads != 0 || flushPending || abortCount > 0))
+ while(!closed && ((state != null && !state.isIdle) || pauseThreads != 0 || flushPending || aborting)) {
+ // System.out.println("DW.waitReady: closed=" + closed + " state.isIdle=" + state.isIdle + " pauseThreads=" + pauseThreads + " flushPending=" + flushPending + " aborting=" + aborting);
try {
wait();
} catch (InterruptedException e) {
Thread.currentThread().interrupt();
}
+ }
if (closed)
throw new AlreadyClosedException("this IndexWriter is closed");
@@ -1297,67 +969,63 @@
deletesInRAM.queries.put(query, new Integer(flushedDocCount + docID));
}
+ synchronized boolean doBalanceRAM() {
+ return ramBufferSize != IndexWriter.DISABLE_AUTO_FLUSH && !bufferIsFull && (numBytesUsed >= ramBufferSize || numBytesAlloc >= freeTrigger);
+ }
+
/** Does the synchronized work to finish/flush the
- * inverted document. */
- private synchronized void finishDocument(DocumentsWriterThreadState state) throws IOException, AbortException {
- if (abortCount > 0) {
- // Forcefully idle this threadstate -- its state will
- // be reset by abort()
- state.isIdle = true;
- notifyAll();
- return;
- }
+ * inverted document. */
+ private void finishDocument(DocumentsWriterThreadState perThread, DocWriter docWriter) throws IOException {
- if (ramBufferSize != IndexWriter.DISABLE_AUTO_FLUSH
- && numBytesUsed >= ramBufferSize)
+ if (doBalanceRAM())
+ // Must call this w/o holding synchronized(this) else
+ // we'll hit deadlock:
balanceRAM();
- // Now write the indexed document to the real files.
- if (nextWriteDocID == state.docID) {
- // It's my turn, so write everything now:
- nextWriteDocID++;
- state.writeDocument();
- state.isIdle = true;
+ synchronized(this) {
+
+ assert docWriter == null || docWriter.docID == perThread.docState.docID;
+
+ perThread.isIdle = true;
notifyAll();
- // If any states were waiting on me, sweep through and
- // flush those that are enabled by my write.
- if (numWaiting > 0) {
- boolean any = true;
- while(any) {
- any = false;
- for(int i=0;i<numWaiting;) {
- final DocumentsWriterThreadState s = waitingThreadStates[i];
- if (s.docID == nextWriteDocID) {
- s.writeDocument();
- s.isIdle = true;
- nextWriteDocID++;
- any = true;
- if (numWaiting > i+1)
- // Swap in the last waiting state to fill in
- // the hole we just created. It's important
- // to do this as-we-go and not at the end of
- // the loop, because if we hit an aborting
- // exception in one of the s.writeDocument
- // calls (above), it leaves this array in an
- // inconsistent state:
- waitingThreadStates[i] = waitingThreadStates[numWaiting-1];
- numWaiting--;
- } else {
- assert !s.isIdle;
- i++;
- }
- }
- }
+ if (aborting) {
+
+ // We are currently aborting, and another thread is
+ // waiting for me to become idle. We just forcefully
+ // idle this threadState; it will be fully reset by
+ // abort()
+ if (docWriter != null)
+ docWriter.abort();
+
+ return;
}
- } else {
- // Another thread got a docID before me, but, it
- // hasn't finished its processing. So add myself to
- // the line but don't hold up this thread.
- waitingThreadStates[numWaiting++] = state;
+
+ if (docWriter != null)
+ waitQueue.add(docWriter);
+ else {
+ skipDocWriter.docID = perThread.docState.docID;
+ waitQueue.add(skipDocWriter);
+ }
+
+ if (bufferIsFull && !flushPending) {
+ flushPending = true;
+ perThread.doFlushAfter = true;
+ }
}
}
+ private static class SkipDocWriter extends DocWriter {
+ void finish() {
+ }
+ void abort() {
+ }
+ long sizeInBytes() {
+ return 0;
+ }
+ }
+ final SkipDocWriter skipDocWriter = new SkipDocWriter();
+
long getRAMUsed() {
return numBytesUsed;
}
@@ -1367,35 +1035,10 @@
NumberFormat nf = NumberFormat.getInstance();
- /* Used only when writing norms to fill in default norm
- * value into the holes in docID stream for those docs
- * that didn't have this field. */
- static void fillBytes(IndexOutput out, byte b, int numBytes) throws IOException {
- for(int i=0;i<numBytes;i++)
- out.writeByte(b);
- }
-
- final byte[] copyByteBuffer = new byte[4096];
-
- /** Copy numBytes from srcIn to destIn */
- void copyBytes(IndexInput srcIn, IndexOutput destIn, long numBytes) throws IOException {
- // TODO: we could do this more efficiently (save a copy)
- // because it's always from a ByteSliceReader ->
- // IndexOutput
- while(numBytes > 0) {
- final int chunk;
- if (numBytes > 4096)
- chunk = 4096;
- else
- chunk = (int) numBytes;
- srcIn.readBytes(copyByteBuffer, 0, chunk);
- destIn.writeBytes(copyByteBuffer, 0, chunk);
- numBytes -= chunk;
- }
- }
-
- // Used only when infoStream != null
+ // TODO FI: this is not flexible -- we can't hardwire
+ // extensions in here:
private long segmentSize(String segmentName) throws IOException {
+ // Used only when infoStream != null
assert infoStream != null;
long size = directory.fileLength(segmentName + ".tii") +
@@ -1410,62 +1053,12 @@
return size;
}
- final private static int POINTER_NUM_BYTE = 4;
- final private static int INT_NUM_BYTE = 4;
- final private static int CHAR_NUM_BYTE = 2;
+ // Coarse estimates used to measure RAM usage of buffered deletes
+ final static int OBJECT_HEADER_BYTES = 8;
+ final static int POINTER_NUM_BYTE = 4;
+ final static int INT_NUM_BYTE = 4;
+ final static int CHAR_NUM_BYTE = 2;
- // Why + 5*POINTER_NUM_BYTE below?
- // 1: Posting has "vector" field which is a pointer
- // 2: Posting is referenced by postingsFreeList array
- // 3,4,5: Posting is referenced by postings hash, which
- // targets 25-50% fill factor; approximate this
- // as 3X # pointers
- final static int POSTING_NUM_BYTE = OBJECT_HEADER_BYTES + 9*INT_NUM_BYTE + 5*POINTER_NUM_BYTE;
-
- // Holds free pool of Posting instances
- private Posting[] postingsFreeList;
- private int postingsFreeCount;
- private int postingsAllocCount;
-
- /* Allocate more Postings from shared pool */
- synchronized void getPostings(Posting[] postings) {
- numBytesUsed += postings.length * POSTING_NUM_BYTE;
- final int numToCopy;
- if (postingsFreeCount < postings.length)
- numToCopy = postingsFreeCount;
- else
- numToCopy = postings.length;
- final int start = postingsFreeCount-numToCopy;
- System.arraycopy(postingsFreeList, start,
- postings, 0, numToCopy);
- postingsFreeCount -= numToCopy;
-
- // Directly allocate the remainder if any
- if (numToCopy < postings.length) {
- final int extra = postings.length - numToCopy;
- final int newPostingsAllocCount = postingsAllocCount + extra;
- if (newPostingsAllocCount > postingsFreeList.length)
- postingsFreeList = new Posting[(int) (1.25 * newPostingsAllocCount)];
-
- balanceRAM();
- for(int i=numToCopy;i<postings.length;i++) {
- postings[i] = new Posting();
- numBytesAlloc += POSTING_NUM_BYTE;
- postingsAllocCount++;
- }
- }
- assert numBytesUsed <= numBytesAlloc;
- }
-
- synchronized void recyclePostings(Posting[] postings, int numPostings) {
- // Move all Postings from this ThreadState back to our
- // free list. We pre-allocated this array while we were
- // creating Postings to make sure it's large enough
- assert postingsFreeCount + numPostings <= postingsFreeList.length;
- System.arraycopy(postings, 0, postingsFreeList, postingsFreeCount, numPostings);
- postingsFreeCount += numPostings;
- }
-
/* Initial chunks size of the shared byte[] blocks used to
store postings data */
final static int BYTE_BLOCK_SHIFT = 15;
@@ -1483,8 +1076,13 @@
final int size = freeByteBlocks.size();
final byte[] b;
if (0 == size) {
+ // Always record a block allocated, even if
+ // trackAllocations is false. This is necessary
+ // because this block will be shared between
+ // things that don't track allocations (term
+ // vectors) and things that do (freq/prox
+ // postings).
numBytesAlloc += BYTE_BLOCK_SIZE;
- balanceRAM();
b = new byte[BYTE_BLOCK_SIZE];
} else
b = (byte[]) freeByteBlocks.remove(size-1);
@@ -1504,6 +1102,51 @@
}
}
+ /* Initial chunks size of the shared int[] blocks used to
+ store postings data */
+ final static int INT_BLOCK_SHIFT = 13;
+ final static int INT_BLOCK_SIZE = (int) Math.pow(2.0, INT_BLOCK_SHIFT);
+ final static int INT_BLOCK_MASK = INT_BLOCK_SIZE - 1;
+
+ private ArrayList freeIntBlocks = new ArrayList();
+
+ /* Allocate another int[] from the shared pool */
+ synchronized int[] getIntBlock(boolean trackAllocations) {
+ final int size = freeIntBlocks.size();
+ final int[] b;
+ if (0 == size) {
+ // Always record a block allocated, even if
+ // trackAllocations is false. This is necessary
+ // because this block will be shared between
+ // things that don't track allocations (term
+ // vectors) and things that do (freq/prox
+ // postings).
+ numBytesAlloc += INT_BLOCK_SIZE*INT_NUM_BYTE;
+ b = new int[INT_BLOCK_SIZE];
+ } else
+ b = (int[]) freeIntBlocks.remove(size-1);
+ if (trackAllocations)
+ numBytesUsed += INT_BLOCK_SIZE*INT_NUM_BYTE;
+ assert numBytesUsed <= numBytesAlloc;
+ return b;
+ }
+
+ synchronized void bytesAllocated(long numBytes) {
+ numBytesAlloc += numBytes;
+ assert numBytesUsed <= numBytesAlloc;
+ }
+
+ synchronized void bytesUsed(long numBytes) {
+ numBytesUsed += numBytes;
+ assert numBytesUsed <= numBytesAlloc;
+ }
+
+ /* Return int[]s to the pool */
+ synchronized void recycleIntBlocks(int[][] blocks, int start, int end) {
+ for(int i=start;i<end;i++)
+ freeIntBlocks.add(blocks[i]);
+ }
+
ByteBlockAllocator byteBlockAllocator = new ByteBlockAllocator();
/* Initial chunk size of the shared char[] blocks used to
@@ -1522,16 +1165,19 @@
final char[] c;
if (0 == size) {
numBytesAlloc += CHAR_BLOCK_SIZE * CHAR_NUM_BYTE;
- balanceRAM();
c = new char[CHAR_BLOCK_SIZE];
} else
c = (char[]) freeCharBlocks.remove(size-1);
+ // We always track allocations of char blocks, for now,
+ // because nothing that skips allocation tracking
+ // (currently only term vectors) uses its own char
+ // blocks.
numBytesUsed += CHAR_BLOCK_SIZE * CHAR_NUM_BYTE;
assert numBytesUsed <= numBytesAlloc;
return c;
}
- /* Return a char[] to the pool */
+ /* Return char[]s to the pool */
synchronized void recycleCharBlocks(char[][] blocks, int numBlocks) {
for(int i=0;i<numBlocks;i++)
freeCharBlocks.add(blocks[i]);
@@ -1552,16 +1198,8 @@
* the other two. This method just frees allocations from
* the pools once we are over-budget, which balances the
* pools to match the current docs. */
- synchronized void balanceRAM() {
+ void balanceRAM() {
- if (ramBufferSize == IndexWriter.DISABLE_AUTO_FLUSH || bufferIsFull)
- return;
-
- // We free our allocations if we've allocated 5% over
- // our allowed RAM buffer
- final long freeTrigger = (long) (1.05 * ramBufferSize);
- final long freeLevel = (long) (0.95 * ramBufferSize);
-
// We flush when we've used our target usage
final long flushTrigger = (long) ramBufferSize;
@@ -1571,57 +1209,58 @@
" vs trigger=" + toMB(flushTrigger) +
" allocMB=" + toMB(numBytesAlloc) +
" vs trigger=" + toMB(freeTrigger) +
- " postingsFree=" + toMB(postingsFreeCount*POSTING_NUM_BYTE) +
" byteBlockFree=" + toMB(byteBlockAllocator.freeByteBlocks.size()*BYTE_BLOCK_SIZE) +
" charBlockFree=" + toMB(freeCharBlocks.size()*CHAR_BLOCK_SIZE*CHAR_NUM_BYTE));
- // When we've crossed 100% of our target Postings
- // RAM usage, try to free up until we're back down
- // to 95%
final long startBytesAlloc = numBytesAlloc;
- final int postingsFreeChunk = (int) (BYTE_BLOCK_SIZE / POSTING_NUM_BYTE);
-
int iter = 0;
- // We free equally from each pool in 64 KB
+ // We free equally from each pool in 32 KB
// chunks until we are below our threshold
// (freeLevel)
+ boolean any = true;
+
while(numBytesAlloc > freeLevel) {
- if (0 == byteBlockAllocator.freeByteBlocks.size() && 0 == freeCharBlocks.size() && 0 == postingsFreeCount) {
- // Nothing else to free -- must flush now.
- bufferIsFull = true;
- if (infoStream != null)
- message(" nothing to free; now set bufferIsFull");
- break;
- }
+
+ synchronized(this) {
+ if (0 == byteBlockAllocator.freeByteBlocks.size() && 0 == freeCharBlocks.size() && 0 == freeIntBlocks.size() && !any) {
+ // Nothing else to free -- must flush now.
+ bufferIsFull = numBytesUsed > flushTrigger;
+ if (infoStream != null) {
+ if (numBytesUsed > flushTrigger)
+ message(" nothing to free; now set bufferIsFull");
+ else
+ message(" nothing to free");
+ }
+ assert numBytesUsed <= numBytesAlloc;
+ break;
+ }
- if ((0 == iter % 3) && byteBlockAllocator.freeByteBlocks.size() > 0) {
- byteBlockAllocator.freeByteBlocks.remove(byteBlockAllocator.freeByteBlocks.size()-1);
- numBytesAlloc -= BYTE_BLOCK_SIZE;
- }
+ if ((0 == iter % 4) && byteBlockAllocator.freeByteBlocks.size() > 0) {
+ byteBlockAllocator.freeByteBlocks.remove(byteBlockAllocator.freeByteBlocks.size()-1);
+ numBytesAlloc -= BYTE_BLOCK_SIZE;
+ }
- if ((1 == iter % 3) && freeCharBlocks.size() > 0) {
- freeCharBlocks.remove(freeCharBlocks.size()-1);
- numBytesAlloc -= CHAR_BLOCK_SIZE * CHAR_NUM_BYTE;
- }
+ if ((1 == iter % 4) && freeCharBlocks.size() > 0) {
+ freeCharBlocks.remove(freeCharBlocks.size()-1);
+ numBytesAlloc -= CHAR_BLOCK_SIZE * CHAR_NUM_BYTE;
+ }
- if ((2 == iter % 3) && postingsFreeCount > 0) {
- final int numToFree;
- if (postingsFreeCount >= postingsFreeChunk)
- numToFree = postingsFreeChunk;
- else
- numToFree = postingsFreeCount;
- Arrays.fill(postingsFreeList, postingsFreeCount-numToFree, postingsFreeCount, null);
- postingsFreeCount -= numToFree;
- postingsAllocCount -= numToFree;
- numBytesAlloc -= numToFree * POSTING_NUM_BYTE;
+ if ((2 == iter % 4) && freeIntBlocks.size() > 0) {
+ freeIntBlocks.remove(freeIntBlocks.size()-1);
+ numBytesAlloc -= INT_BLOCK_SIZE * INT_NUM_BYTE;
+ }
}
+ if ((3 == iter % 4) && any)
+ // Ask consumer to free any recycled state
+ any = consumer.freeRAM();
+
iter++;
}
-
+
if (infoStream != null)
message(" after free: freedMB=" + nf.format((startBytesAlloc-numBytesAlloc)/1024./1024.) + " usedMB=" + nf.format(numBytesUsed/1024./1024.) + " allocMB=" + nf.format(numBytesAlloc/1024./1024.));
@@ -1631,14 +1270,127 @@
// using, go ahead and flush. This prevents
// over-allocating and then freeing, with every
// flush.
- if (numBytesUsed > flushTrigger) {
- if (infoStream != null)
- message(" RAM: now flush @ usedMB=" + nf.format(numBytesUsed/1024./1024.) +
- " allocMB=" + nf.format(numBytesAlloc/1024./1024.) +
- " triggerMB=" + nf.format(flushTrigger/1024./1024.));
+ synchronized(this) {
- bufferIsFull = true;
+ if (numBytesUsed > flushTrigger) {
+ if (infoStream != null)
+ message(" RAM: now flush @ usedMB=" + nf.format(numBytesUsed/1024./1024.) +
+ " allocMB=" + nf.format(numBytesAlloc/1024./1024.) +
+ " triggerMB=" + nf.format(flushTrigger/1024./1024.));
+
+ bufferIsFull = true;
+ }
}
}
}
+
+ final WaitQueue waitQueue = new WaitQueue();
+
+ private class WaitQueue {
+ DocWriter[] waiting;
+ int nextWriteDocID;
+ int nextWriteLoc;
+ int numWaiting;
+ long waitingBytes;
+
+ public WaitQueue() {
+ waiting = new DocWriter[10];
+ }
+
+ synchronized void reset() {
+ // NOTE: nextWriteLoc doesn't need to be reset
+ assert numWaiting == 0;
+ assert waitingBytes == 0;
+ nextWriteDocID = 0;
+ }
+
+ synchronized void abort() {
+ int count = 0;
+ for(int i=0;i<waiting.length;i++) {
+ final DocWriter doc = waiting[i];
+ if (doc != null) {
+ doc.abort();
+ waiting[i] = null;
+ count++;
+ }
+ }
+ waitingBytes = 0;
+ assert count == numWaiting;
+ numWaiting = 0;
+ }
+
+ private void writeDocument(DocWriter doc) throws IOException {
+ assert doc == skipDocWriter || nextWriteDocID == doc.docID;
+ boolean success = false;
+ try {
+ doc.finish();
+ nextWriteDocID++;
+ numDocsInStore++;
+ nextWriteLoc++;
+ assert nextWriteLoc <= waiting.length;
+ if (nextWriteLoc == waiting.length)
+ nextWriteLoc = 0;
+ success = true;
+ } finally {
+ if (!success)
+ setAborting();
+ }
+ }
+
+ synchronized public void add(DocWriter doc) throws IOException {
+
+ assert doc.docID >= nextWriteDocID;
+
+ if (doc.docID == nextWriteDocID) {
+ //System.out.println("DW.WaitQueue.add: docID=" + doc.docID + ": my turn");
+ writeDocument(doc);
+ while(true) {
+ doc = waiting[nextWriteLoc];
+ if (doc != null) {
+ numWaiting--;
+ waiting[nextWriteLoc] = null;
+ waitingBytes -= doc.sizeInBytes();
+ writeDocument(doc);
+ } else
+ break;
+ }
+ } else {
+
+ //System.out.println("DW.WaitQueue.add: docID=" + doc.docID + ": not my turn: nextWriteDocID=" + nextWriteDocID);
+ // I finished before documents that were added
+ // before me. This can easily happen when I am a
+ // small doc and the docs before me were large, or,
+ // just due to luck in the thread scheduling. Just
+ // add myself to the queue and when that large doc
+ // finishes, it will flush me:
+ int gap = doc.docID - nextWriteDocID;
+ if (gap >= waiting.length) {
+ // Grow queue
+ // nocommit -- must pause when wait queue grows
+ // too large
+ DocWriter[] newArray = new DocWriter[ArrayUtil.getNextSize(gap)];
+ assert nextWriteLoc >= 0;
+ System.arraycopy(waiting, nextWriteLoc, newArray, 0, waiting.length-nextWriteLoc);
+ System.arraycopy(waiting, 0, newArray, waiting.length-nextWriteLoc, nextWriteLoc);
+ nextWriteLoc = 0;
+ waiting = newArray;
+ gap = doc.docID - nextWriteDocID;
+ System.out.println("DocumentsWriter: waitQueue size=" + newArray.length + " bytes=" + waitingBytes);
+ }
+
+ int loc = nextWriteLoc + gap;
+ if (loc >= waiting.length)
+ loc -= waiting.length;
+
+ // We should only wrap one time
+ assert loc < waiting.length;
+
+ // Nobody should be in my spot!
+ assert waiting[loc] == null;
+ waiting[loc] = doc;
+ numWaiting++;
+ waitingBytes += doc.sizeInBytes();
+ }
+ }
+ }
}
Index: src/java/org/apache/lucene/index/IntBlockPool.java
===================================================================
--- src/java/org/apache/lucene/index/IntBlockPool.java (revision 0)
+++ src/java/org/apache/lucene/index/IntBlockPool.java (revision 0)
@@ -0,0 +1,64 @@
+package org.apache.lucene.index;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+final class IntBlockPool {
+
+ public int[][] buffers = new int[10][];
+
+ int bufferUpto = -1; // Which buffer we are upto
+ public int intUpto = DocumentsWriter.INT_BLOCK_SIZE; // Where we are in head buffer
+
+ public int[] buffer; // Current head buffer
+ public int intOffset = -DocumentsWriter.INT_BLOCK_SIZE; // Current head offset
+
+ final private DocumentsWriter docWriter;
+ final boolean trackAllocations;
+
+ public IntBlockPool(DocumentsWriter docWriter, boolean trackAllocations) {
+ this.docWriter = docWriter;
+ this.trackAllocations = trackAllocations;
+ }
+
+ public void reset() {
+ if (bufferUpto > 0) {
+ // Recycle all but the first buffer
+ docWriter.recycleIntBlocks(buffers, 1, 1+bufferUpto);
+
+ // Reuse first buffer
+ bufferUpto = -1;
+ intUpto = DocumentsWriter.INT_BLOCK_SIZE;
+ intOffset = -DocumentsWriter.INT_BLOCK_SIZE;
+ buffer = buffers[0];
+ }
+ }
+
+ public void nextBuffer() {
+ if (1+bufferUpto == buffers.length) {
+ int[][] newBuffers = new int[(int) (buffers.length*1.5)][];
+ System.arraycopy(buffers, 0, newBuffers, 0, buffers.length);
+ buffers = newBuffers;
+ }
+ buffer = buffers[1+bufferUpto] = docWriter.getIntBlock(trackAllocations);
+ bufferUpto++;
+
+ intUpto = 0;
+ intOffset += DocumentsWriter.INT_BLOCK_SIZE;
+ }
+}
+
Property changes on: src/java/org/apache/lucene/index/IntBlockPool.java
___________________________________________________________________
Name: svn:eol-style
+ native
Index: src/java/org/apache/lucene/index/StoredFieldsWriterPerField.java
===================================================================
--- src/java/org/apache/lucene/index/StoredFieldsWriterPerField.java (revision 0)
+++ src/java/org/apache/lucene/index/StoredFieldsWriterPerField.java (revision 0)
@@ -0,0 +1,66 @@
+package org.apache.lucene.index;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+import org.apache.lucene.document.Fieldable;
+
+class StoredFieldsWriterPerField extends DocFieldConsumerPerField {
+
+ final StoredFieldsWriterPerThread perThread;
+ final FieldInfo fieldInfo;
+ final DocumentsWriter.DocState docState;
+
+ public StoredFieldsWriterPerField(StoredFieldsWriterPerThread perThread, FieldInfo fieldInfo) {
+ this.perThread = perThread;
+ this.fieldInfo = fieldInfo;
+ docState = perThread.docState;
+ }
+
+ // Process all occurrences of a single field in one doc;
+ // count is 1 if a given field occurs only once in the
+ // Document, which is the "typical" case
+ public void processFields(Fieldable[] fields, int count) throws IOException {
+
+ final StoredFieldsWriter.PerDoc doc;
+ if (perThread.doc == null) {
+ doc = perThread.doc = perThread.storedFieldsWriter.getPerDoc();
+ doc.docID = docState.docID;
+ perThread.localFieldsWriter.setFieldsStream(doc.fdt);
+ assert doc.numStoredFields == 0;
+ assert 0 == doc.fdt.length();
+ assert 0 == doc.fdt.getFilePointer();
+ } else {
+ doc = perThread.doc;
+ assert doc.docID == docState.docID;
+ }
+
+ for(int i=0;i<count;i++) {
+ final Fieldable field = fields[i];
+ if (field.isStored()) {
+ perThread.localFieldsWriter.writeField(fieldInfo, field);
+ assert docState.testPoint("StoredFieldsWriterPerField.processFields.writeField");
+ doc.numStoredFields++;
+ }
+ }
+ }
+
+ void abort() {
+ }
+}
+
Property changes on: src/java/org/apache/lucene/index/StoredFieldsWriterPerField.java
___________________________________________________________________
Name: svn:eol-style
+ native
Index: src/java/org/apache/lucene/index/TermVectorsTermsWriterPerField.java
===================================================================
--- src/java/org/apache/lucene/index/TermVectorsTermsWriterPerField.java (revision 0)
+++ src/java/org/apache/lucene/index/TermVectorsTermsWriterPerField.java (revision 0)
@@ -0,0 +1,248 @@
+package org.apache.lucene.index;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+import org.apache.lucene.util.UnicodeUtil;
+import org.apache.lucene.analysis.Token;
+import org.apache.lucene.document.Fieldable;
+import org.apache.lucene.store.IndexOutput;
+
+class TermVectorsTermsWriterPerField extends TermsHashConsumerPerField {
+
+ final TermVectorsTermsWriterPerThread perThread;
+ final TermsHashPerField termsHashPerField;
+ final TermVectorsTermsWriter termsWriter;
+ final FieldInfo fieldInfo;
+ final DocumentsWriter.DocState docState;
+ final DocInverter.FieldInvertState fieldState;
+
+ boolean doVectors;
+ boolean doVectorPositions;
+ boolean doVectorOffsets;
+
+ int maxNumPostings;
+
+ public TermVectorsTermsWriterPerField(TermsHashPerField termsHashPerField, TermVectorsTermsWriterPerThread perThread, FieldInfo fieldInfo) {
+ this.termsHashPerField = termsHashPerField;
+ this.perThread = perThread;
+ this.termsWriter = perThread.termsWriter;
+ this.fieldInfo = fieldInfo;
+ docState = termsHashPerField.docState;
+ fieldState = termsHashPerField.fieldState;
+ }
+
+ boolean start(Fieldable[] fields, int count) {
+ doVectors = false;
+ doVectorPositions = false;
+ doVectorOffsets = false;
+
+ for(int i=0;i<count;i++) {
+ Fieldable field = fields[i];
+ if (field.isIndexed() && field.isTermVectorStored()) {
+ doVectors = true;
+ doVectorPositions |= field.isStorePositionWithTermVector();
+ doVectorOffsets |= field.isStoreOffsetWithTermVector();
+ }
+ }
+
+ if (doVectors) {
+ if (perThread.doc == null) {
+ perThread.doc = termsWriter.getPerDoc();
+ perThread.doc.docID = docState.docID;
+ assert perThread.doc.numVectorFields == 0;
+ assert 0 == perThread.doc.tvf.length();
+ assert 0 == perThread.doc.tvf.getFilePointer();
+ } else {
+ assert perThread.doc.docID == docState.docID;
+
+ if (termsHashPerField.numPostings != 0)
+ // Only necessary if previous doc hit a
+ // non-aborting exception while writing vectors in
+ // this field:
+ termsHashPerField.reset();
+ }
+ }
+
+ // TODO: only if needed for performance
+ //perThread.postingsCount = 0;
+
+ return doVectors;
+ }
+
+ void start(Fieldable field) {}
+ void end(Fieldable field) {}
+ public void abort() {}
+
+ /** Called once per field per document if term vectors
+ * are enabled, to write the vectors to
+ * RAMOutputStream, which is then quickly flushed to
+ * * the real term vectors files in the Directory. */
+ final void finish() throws IOException {
+
+ assert docState.testPoint("TermVectorsTermsWriterPerField.finish start");
+
+ final int numPostings = termsHashPerField.numPostings;
+
+ assert numPostings >= 0;
+
+ if (!doVectors || numPostings == 0)
+ return;
+
+ if (numPostings > maxNumPostings)
+ maxNumPostings = numPostings;
+
+ final IndexOutput tvf = perThread.doc.tvf;
+
+ // This is called once, after inverting all occurences
+ // of a given field in the doc. At this point we flush
+ // our hash into the DocWriter.
+
+ assert fieldInfo.storeTermVector;
+ assert perThread.vectorFieldsInOrder(fieldInfo);
+
+ perThread.doc.addField(termsHashPerField.fieldInfo.number);
+
+ final RawPostingList[] postings = termsHashPerField.sortPostings();
+
+ tvf.writeVInt(numPostings);
+ byte bits = 0x0;
+ if (doVectorPositions)
+ bits |= TermVectorsReader.STORE_POSITIONS_WITH_TERMVECTOR;
+ if (doVectorOffsets)
+ bits |= TermVectorsReader.STORE_OFFSET_WITH_TERMVECTOR;
+ tvf.writeByte(bits);
+
+ int encoderUpto = 0;
+ int lastTermBytesCount = 0;
+
+ final ByteSliceReader reader = perThread.vectorSliceReader;
+ final char[][] charBuffers = perThread.termsHashPerThread.charPool.buffers;
+ for(int j=0;j<numPostings;j++) {
+ final TermVectorsTermsWriter.PostingList posting = (TermVectorsTermsWriter.PostingList) postings[j];
+ final int freq = posting.freq;
+
+ final char[] text2 = charBuffers[posting.textStart >> DocumentsWriter.CHAR_BLOCK_SHIFT];
+ final int start2 = posting.textStart & DocumentsWriter.CHAR_BLOCK_MASK;
+
+ // We swap between two encoders to save copying
+ // last Term's byte array
+ final UnicodeUtil.UTF8Result utf8Result = perThread.utf8Results[encoderUpto];
+
+ // TODO: we could do this incrementally
+ UnicodeUtil.UTF16toUTF8(text2, start2, utf8Result);
+ final int termBytesCount = utf8Result.length;
+
+ // TODO: UTF16toUTF8 could tell us this prefix
+ // Compute common prefix between last term and
+ // this term
+ int prefix = 0;
+ if (j > 0) {
+ final byte[] lastTermBytes = perThread.utf8Results[1-encoderUpto].result;
+ final byte[] termBytes = perThread.utf8Results[encoderUpto].result;
+ while(prefix < lastTermBytesCount && prefix < termBytesCount) {
+ if (lastTermBytes[prefix] != termBytes[prefix])
+ break;
+ prefix++;
+ }
+ }
+ encoderUpto = 1-encoderUpto;
+ lastTermBytesCount = termBytesCount;
+
+ final int suffix = termBytesCount - prefix;
+ tvf.writeVInt(prefix);
+ tvf.writeVInt(suffix);
+ tvf.writeBytes(utf8Result.result, prefix, suffix);
+ tvf.writeVInt(freq);
+
+ if (doVectorPositions) {
+ termsHashPerField.initReader(reader, posting, 0);
+ reader.writeTo(tvf);
+ }
+
+ if (doVectorOffsets) {
+ termsHashPerField.initReader(reader, posting, 1);
+ reader.writeTo(tvf);
+ }
+ }
+
+ termsHashPerField.reset();
+ perThread.termsHashPerThread.reset(false);
+ }
+
+ void shrinkHash() {
+ termsHashPerField.shrinkHash(maxNumPostings);
+ maxNumPostings = 0;
+ }
+
+ void newTerm(Token t, RawPostingList p0) {
+
+ assert docState.testPoint("TermVectorsTermsWriterPerField.newTerm start");
+
+ TermVectorsTermsWriter.PostingList p = (TermVectorsTermsWriter.PostingList) p0;
+
+ p.freq = 1;
+
+ if (doVectorOffsets) {
+ final int startOffset = fieldState.offset + t.startOffset();
+ final int endOffset = fieldState.offset + t.endOffset();
+ termsHashPerField.writeVInt(1, startOffset);
+ termsHashPerField.writeVInt(1, endOffset - startOffset);
+ p.lastOffset = endOffset;
+ }
+
+ if (doVectorPositions) {
+ termsHashPerField.writeVInt(0, fieldState.position);
+ p.lastPosition = fieldState.position;
+ }
+
+ // nocommit -- see if this is necessary
+ /*
+ if (perThread.postingsCount == perThread.postings.length) {
+ final int newSize = ArrayUtil.getNextSize(1+perThread.postingsCount);
+ TermVectorsTermsWriter.PostingList[] newArray = new TermVectorsTermsWriter.PostingList[newSize];
+ System.arraycopy(perThread.postings, 0, newArray, 0, perThread.postingsCount);
+ perThread.postings = newArray;
+ }
+ perThread.postings[perThread.postingsCount++] = p;
+ */
+ }
+
+ void addTerm(Token t, RawPostingList p0) {
+
+ assert docState.testPoint("TermVectorsTermsWriterPerField.addTerm start");
+
+ TermVectorsTermsWriter.PostingList p = (TermVectorsTermsWriter.PostingList) p0;
+ p.freq++;
+
+ if (doVectorOffsets) {
+ final int startOffset = fieldState.offset + t.startOffset();
+ final int endOffset = fieldState.offset + t.endOffset();
+ termsHashPerField.writeVInt(1, startOffset - p.lastOffset);
+ termsHashPerField.writeVInt(1, endOffset - startOffset);
+ p.lastOffset = endOffset;
+ }
+
+ if (doVectorPositions) {
+ termsHashPerField.writeVInt(0, fieldState.position - p.lastPosition);
+ p.lastPosition = fieldState.position;
+ }
+ }
+
+ void skippingLongTerm(Token t) {}
+}
Property changes on: src/java/org/apache/lucene/index/TermVectorsTermsWriterPerField.java
___________________________________________________________________
Name: svn:eol-style
+ native
Index: src/java/org/apache/lucene/index/BufferedNorms.java
===================================================================
--- src/java/org/apache/lucene/index/BufferedNorms.java (revision 664673)
+++ src/java/org/apache/lucene/index/BufferedNorms.java (working copy)
@@ -1,60 +0,0 @@
-package org.apache.lucene.index;
-
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-import java.io.IOException;
-import org.apache.lucene.store.RAMOutputStream;
-import org.apache.lucene.search.Similarity;
-
-/* Stores norms, buffered in RAM, until they are flushed
- * to a partial segment. */
-final class BufferedNorms {
-
- RAMOutputStream out;
- int upto;
-
- private static final byte defaultNorm = Similarity.encodeNorm(1.0f);
-
- BufferedNorms() {
- out = new RAMOutputStream();
- }
-
- void add(float norm) throws IOException {
- byte b = Similarity.encodeNorm(norm);
- out.writeByte(b);
- upto++;
- }
-
- void reset() {
- out.reset();
- upto = 0;
- }
-
- void fill(int docID) throws IOException {
- // Must now fill in docs that didn't have this
- // field. Note that this is how norms can consume
- // tremendous storage when the docs have widely
- // varying different fields, because we are not
- // storing the norms sparsely (see LUCENE-830)
- if (upto < docID) {
- DocumentsWriter.fillBytes(out, defaultNorm, docID-upto);
- upto = docID;
- }
- }
-}
-
Index: src/java/org/apache/lucene/index/IndexWriter.java
===================================================================
--- src/java/org/apache/lucene/index/IndexWriter.java (revision 664673)
+++ src/java/org/apache/lucene/index/IndexWriter.java (working copy)
@@ -426,6 +426,7 @@
public void setSimilarity(Similarity similarity) {
ensureOpen();
this.similarity = similarity;
+ docWriter.setSimilarity(similarity);
}
/** Expert: Return the Similarity implementation used by this IndexWriter.
@@ -1130,6 +1131,7 @@
docWriter = new DocumentsWriter(directory, this);
docWriter.setInfoStream(infoStream);
+ docWriter.setMaxFieldLength(maxFieldLength);
// Default deleter (for backwards compatibility) is
// KeepOnlyLastCommitDeleter:
@@ -1267,6 +1269,7 @@
public void setMaxFieldLength(int maxFieldLength) {
ensureOpen();
this.maxFieldLength = maxFieldLength;
+ docWriter.setMaxFieldLength(maxFieldLength);
if (infoStream != null)
message("setMaxFieldLength " + maxFieldLength);
}
@@ -1715,62 +1718,59 @@
*/
private synchronized boolean flushDocStores() throws IOException {
- List files = docWriter.files();
-
boolean useCompoundDocStore = false;
- if (files.size() > 0) {
- String docStoreSegment;
+ String docStoreSegment;
- boolean success = false;
- try {
- docStoreSegment = docWriter.closeDocStore();
- success = true;
- } finally {
- if (!success) {
- if (infoStream != null)
- message("hit exception closing doc store segment");
- docWriter.abort(null);
- }
+ boolean success = false;
+ try {
+ docStoreSegment = docWriter.closeDocStore();
+ success = true;
+ } finally {
+ if (!success) {
+ if (infoStream != null)
+ message("hit exception closing doc store segment");
}
+ }
- useCompoundDocStore = mergePolicy.useCompoundDocStore(segmentInfos);
+ useCompoundDocStore = mergePolicy.useCompoundDocStore(segmentInfos);
- if (useCompoundDocStore && docStoreSegment != null) {
- // Now build compound doc store file
+ if (useCompoundDocStore && docStoreSegment != null) {
+ // Now build compound doc store file
- success = false;
+ success = false;
- final int numSegments = segmentInfos.size();
- final String compoundFileName = docStoreSegment + "." + IndexFileNames.COMPOUND_FILE_STORE_EXTENSION;
+ final int numSegments = segmentInfos.size();
+ final String compoundFileName = docStoreSegment + "." + IndexFileNames.COMPOUND_FILE_STORE_EXTENSION;
- try {
- CompoundFileWriter cfsWriter = new CompoundFileWriter(directory, compoundFileName);
- final int size = files.size();
- for(int i=0;i<size;i++)
- cfsWriter.addFile((String) files.get(i));
+ try {
+ CompoundFileWriter cfsWriter = new CompoundFileWriter(directory, compoundFileName);
+ final Iterator it = docWriter.closedFiles().iterator();
+ while(it.hasNext())
+ cfsWriter.addFile((String) it.next());
- // Perform the merge
- cfsWriter.close();
- success = true;
+ // Perform the merge
+ cfsWriter.close();
+ success = true;
- } finally {
- if (!success) {
- if (infoStream != null)
- message("hit exception building compound file doc store for segment " + docStoreSegment);
- deleter.deleteFile(compoundFileName);
- }
+ } finally {
+ if (!success) {
+ if (infoStream != null)
+ message("hit exception building compound file doc store for segment " + docStoreSegment);
+ deleter.deleteFile(compoundFileName);
}
+ }
- for(int i=0;i<numSegments;i++) {
- SegmentInfo si = segmentInfos.info(i);
- if (si.getDocStoreOffset() != -1 &&
- si.getDocStoreSegment().equals(docStoreSegment))
- si.setDocStoreIsCompoundFile(true);
- }
-
- checkpoint();
+ for(int i=0;i<numSegments;i++) {
+ SegmentInfo si = segmentInfos.info(i);
+ if (si.getDocStoreOffset() != -1 &&
+ si.getDocStoreSegment().equals(docStoreSegment))
+ si.setDocStoreIsCompoundFile(true);
}
+
+ checkpoint();
+
+ deleter.deleteNewFiles(docWriter.closedFiles());
}
return useCompoundDocStore;
@@ -1947,7 +1947,7 @@
// If docWriter has some aborted files that were
// never incref'd, then we clean them up here
if (docWriter != null) {
- final List files = docWriter.abortedFiles();
+ final Collection files = docWriter.abortedFiles();
if (files != null)
deleter.deleteNewFiles(files);
}
@@ -2076,7 +2076,7 @@
synchronized (this) {
// If docWriter has some aborted files that were
// never incref'd, then we clean them up here
- final List files = docWriter.abortedFiles();
+ final Collection files = docWriter.abortedFiles();
if (files != null)
deleter.deleteNewFiles(files);
}
@@ -2651,7 +2651,7 @@
segmentInfos.clear();
segmentInfos.addAll(rollbackSegmentInfos);
- docWriter.abort(null);
+ docWriter.abort();
// Ask deleter to locate unreferenced files & remove
// them:
@@ -3338,7 +3338,6 @@
if (!success) {
if (infoStream != null)
message("hit exception flushing segment " + segment);
- docWriter.abort(null);
deleter.refresh(segment);
}
}
@@ -3860,7 +3859,7 @@
// newly flushed doc store files then we should not
// make compound file out of them...
if (infoStream != null)
- message("flush at merge");
+ message("now flush at merge");
flush(false, true, false);
}
Index: src/java/org/apache/lucene/index/PostingVector.java
===================================================================
--- src/java/org/apache/lucene/index/PostingVector.java (revision 664673)
+++ src/java/org/apache/lucene/index/PostingVector.java (working copy)
@@ -1,30 +0,0 @@
-package org.apache.lucene.index;
-
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-/* Used by DocumentsWriter to track data for term vectors.
- * One of these exists per unique term seen in each field in
- * the document. */
-class PostingVector {
- Posting p; // Corresponding Posting instance for this term
- int lastOffset; // Last offset we saw
- int offsetStart; // Address of first slice for offsets
- int offsetUpto; // Next write address for offsets
- int posStart; // Address of first slice for positions
- int posUpto; // Next write address for positions
-}
Index: src/java/org/apache/lucene/index/InvertedDocEndConsumerPerThread.java
===================================================================
--- src/java/org/apache/lucene/index/InvertedDocEndConsumerPerThread.java (revision 0)
+++ src/java/org/apache/lucene/index/InvertedDocEndConsumerPerThread.java (revision 0)
@@ -0,0 +1,25 @@
+package org.apache.lucene.index;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+abstract class InvertedDocEndConsumerPerThread {
+ abstract void startDocument();
+ abstract InvertedDocEndConsumerPerField addField(DocInverterPerField docInverterPerField, FieldInfo fieldInfo);
+ abstract void finishDocument();
+ abstract void abort();
+}
Property changes on: src/java/org/apache/lucene/index/InvertedDocEndConsumerPerThread.java
___________________________________________________________________
Name: svn:eol-style
+ native
Index: src/java/org/apache/lucene/index/IndexFileDeleter.java
===================================================================
--- src/java/org/apache/lucene/index/IndexFileDeleter.java (revision 664673)
+++ src/java/org/apache/lucene/index/IndexFileDeleter.java (working copy)
@@ -403,7 +403,7 @@
final List docWriterFiles;
if (docWriter != null) {
- docWriterFiles = docWriter.files();
+ docWriterFiles = docWriter.openFiles();
if (docWriterFiles != null)
// We must incRef thes files before decRef'ing
// last files to make sure we don't accidentally
@@ -510,11 +510,13 @@
/** Delets the specified files, but only if they are new
* (have not yet been incref'd). */
- void deleteNewFiles(List files) throws IOException {
- final int size = files.size();
- for(int i=0;i<size;i++)
- if (!refCounts.containsKey(files.get(i)))
- deleteFile((String) files.get(i));
+ void deleteNewFiles(Collection files) throws IOException {
+ final Iterator it = files.iterator();
+ while(it.hasNext()) {
+ final String fileName = (String) it.next();
+ if (!refCounts.containsKey(fileName))
+ deleteFile(fileName);
+ }
}
void deleteFile(String fileName)
Index: src/java/org/apache/lucene/index/FreqProxTermsWriterPerThread.java
===================================================================
--- src/java/org/apache/lucene/index/FreqProxTermsWriterPerThread.java (revision 0)
+++ src/java/org/apache/lucene/index/FreqProxTermsWriterPerThread.java (revision 0)
@@ -0,0 +1,41 @@
+package org.apache.lucene.index;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+final class FreqProxTermsWriterPerThread extends TermsHashConsumerPerThread {
+ final TermsHashPerThread termsHashPerThread;
+ final DocumentsWriter.DocState docState;
+
+ public FreqProxTermsWriterPerThread(TermsHashPerThread perThread) {
+ docState = perThread.docState;
+ termsHashPerThread = perThread;
+ }
+
+ public TermsHashConsumerPerField addField(TermsHashPerField termsHashPerField, FieldInfo fieldInfo) {
+ return new FreqProxTermsWriterPerField(termsHashPerField, this, fieldInfo);
+ }
+
+ void startDocument() {
+ }
+
+ DocumentsWriter.DocWriter finishDocument() {
+ return null;
+ }
+
+ public void abort() {}
+}
Property changes on: src/java/org/apache/lucene/index/FreqProxTermsWriterPerThread.java
___________________________________________________________________
Name: svn:eol-style
+ native
Index: src/java/org/apache/lucene/index/DocConsumer.java
===================================================================
--- src/java/org/apache/lucene/index/DocConsumer.java (revision 0)
+++ src/java/org/apache/lucene/index/DocConsumer.java (revision 0)
@@ -0,0 +1,29 @@
+package org.apache.lucene.index;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+import java.util.Collection;
+
+abstract class DocConsumer {
+ abstract DocConsumerPerThread addThread(DocumentsWriterThreadState perThread) throws IOException;
+ abstract void flush(final Collection threads, final DocumentsWriter.FlushState state) throws IOException;
+ abstract void closeDocStore(final DocumentsWriter.FlushState state) throws IOException;
+ abstract void abort();
+ abstract boolean freeRAM();
+}
Property changes on: src/java/org/apache/lucene/index/DocConsumer.java
___________________________________________________________________
Name: svn:eol-style
+ native
Index: src/java/org/apache/lucene/index/InvertedDocConsumerPerThread.java
===================================================================
--- src/java/org/apache/lucene/index/InvertedDocConsumerPerThread.java (revision 0)
+++ src/java/org/apache/lucene/index/InvertedDocConsumerPerThread.java (revision 0)
@@ -0,0 +1,27 @@
+package org.apache.lucene.index;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+
+abstract class InvertedDocConsumerPerThread {
+ abstract void startDocument() throws IOException;
+ abstract InvertedDocConsumerPerField addField(DocInverterPerField docInverterPerField, FieldInfo fieldInfo);
+ abstract DocumentsWriter.DocWriter finishDocument() throws IOException;
+ abstract void abort();
+}
Property changes on: src/java/org/apache/lucene/index/InvertedDocConsumerPerThread.java
___________________________________________________________________
Name: svn:eol-style
+ native
Index: src/java/org/apache/lucene/index/DocInverterPerThread.java
===================================================================
--- src/java/org/apache/lucene/index/DocInverterPerThread.java (revision 0)
+++ src/java/org/apache/lucene/index/DocInverterPerThread.java (revision 0)
@@ -0,0 +1,70 @@
+package org.apache.lucene.index;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+
+import org.apache.lucene.analysis.Token;
+
+/** This is a DocFieldConsumer that inverts each field,
+ * separately, from a Document, and accepts a
+ * InvertedTermsConsumer to process those terms. */
+
+class DocInverterPerThread extends DocFieldConsumerPerThread {
+ final DocInverter docInverter;
+ final InvertedDocConsumerPerThread consumer;
+ final InvertedDocEndConsumerPerThread endConsumer;
+ final Token localToken = new Token();
+ final DocumentsWriter.DocState docState;
+
+ final DocInverter.FieldInvertState fieldState = new DocInverter.FieldInvertState();
+
+ // Used to read a string value for a field
+ final ReusableStringReader stringReader = new ReusableStringReader();
+
+ public DocInverterPerThread(DocFieldProcessorPerThread docFieldProcessorPerThread, DocInverter docInverter) {
+ this.docInverter = docInverter;
+ docState = docFieldProcessorPerThread.docState;
+ consumer = docInverter.consumer.addThread(this);
+ endConsumer = docInverter.endConsumer.addThread(this);
+ }
+
+ public void startDocument() throws IOException {
+ consumer.startDocument();
+ endConsumer.startDocument();
+ }
+
+ public DocumentsWriter.DocWriter finishDocument() throws IOException {
+ // TODO: allow endConsumer.finishDocument to also return
+ // a DocWriter
+ endConsumer.finishDocument();
+ return consumer.finishDocument();
+ }
+
+ void abort() {
+ try {
+ consumer.abort();
+ } finally {
+ endConsumer.abort();
+ }
+ }
+
+ public DocFieldConsumerPerField addField(FieldInfo fi) {
+ return new DocInverterPerField(this, fi);
+ }
+}
Property changes on: src/java/org/apache/lucene/index/DocInverterPerThread.java
___________________________________________________________________
Name: svn:eol-style
+ native
Index: src/java/org/apache/lucene/index/DocInverter.java
===================================================================
--- src/java/org/apache/lucene/index/DocInverter.java (revision 0)
+++ src/java/org/apache/lucene/index/DocInverter.java (revision 0)
@@ -0,0 +1,109 @@
+package org.apache.lucene.index;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.util.Map;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Collection;
+import java.util.Iterator;
+import java.io.IOException;
+
+/** This is a DocFieldConsumer that inverts each field,
+ * separately, from a Document, and accepts a
+ * InvertedTermsConsumer to process those terms. */
+
+final class DocInverter extends DocFieldConsumer {
+
+ final InvertedDocConsumer consumer;
+ final InvertedDocEndConsumer endConsumer;
+
+ public DocInverter(InvertedDocConsumer consumer, InvertedDocEndConsumer endConsumer) {
+ this.consumer = consumer;
+ this.endConsumer = endConsumer;
+ }
+
+ void setFieldInfos(FieldInfos fieldInfos) {
+ super.setFieldInfos(fieldInfos);
+ consumer.setFieldInfos(fieldInfos);
+ endConsumer.setFieldInfos(fieldInfos);
+ }
+
+ void flush(Map threadsAndFields, DocumentsWriter.FlushState state) throws IOException {
+
+ Map childThreadsAndFields = new HashMap();
+ Map endChildThreadsAndFields = new HashMap();
+
+ Iterator it = threadsAndFields.entrySet().iterator();
+ while(it.hasNext()) {
+
+ Map.Entry entry = (Map.Entry) it.next();
+
+ DocInverterPerThread perThread = (DocInverterPerThread) entry.getKey();
+
+ Collection fields = (Collection) entry.getValue();
+
+ Iterator fieldsIt = fields.iterator();
+ Collection childFields = new HashSet();
+ Collection endChildFields = new HashSet();
+ while(fieldsIt.hasNext()) {
+ DocInverterPerField perField = (DocInverterPerField) fieldsIt.next();
+ childFields.add(perField.consumer);
+ endChildFields.add(perField.endConsumer);
+ }
+
+ childThreadsAndFields.put(perThread.consumer, childFields);
+ endChildThreadsAndFields.put(perThread.endConsumer, endChildFields);
+ }
+
+ consumer.flush(childThreadsAndFields, state);
+ endConsumer.flush(endChildThreadsAndFields, state);
+ }
+
+ public void closeDocStore(DocumentsWriter.FlushState state) throws IOException {
+ consumer.closeDocStore(state);
+ endConsumer.closeDocStore(state);
+ }
+
+ void abort() {
+ consumer.abort();
+ endConsumer.abort();
+ }
+
+ public boolean freeRAM() {
+ return consumer.freeRAM();
+ }
+
+ public DocFieldConsumerPerThread addThread(DocFieldProcessorPerThread docFieldProcessorPerThread) {
+ return new DocInverterPerThread(docFieldProcessorPerThread, this);
+ }
+
+ static class FieldInvertState {
+ int position;
+ int length;
+ int offset;
+ float boost;
+
+ void reset(float docBoost) {
+ position = 0;
+ length = 0;
+ offset = 0;
+ boost = docBoost;
+ }
+ }
+}
Property changes on: src/java/org/apache/lucene/index/DocInverter.java
___________________________________________________________________
Name: svn:eol-style
+ native
Index: src/java/org/apache/lucene/index/TermVectorsTermsWriterPerThread.java
===================================================================
--- src/java/org/apache/lucene/index/TermVectorsTermsWriterPerThread.java (revision 0)
+++ src/java/org/apache/lucene/index/TermVectorsTermsWriterPerThread.java (revision 0)
@@ -0,0 +1,92 @@
+package org.apache.lucene.index;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import org.apache.lucene.util.UnicodeUtil;
+
+final class TermVectorsTermsWriterPerThread extends TermsHashConsumerPerThread {
+
+ final TermVectorsTermsWriter termsWriter;
+ final TermsHashPerThread termsHashPerThread;
+ final DocumentsWriter.DocState docState;
+
+ TermVectorsTermsWriter.PerDoc doc;
+
+ /*
+ TermVectorsTermsWriter.PostingList[] postings = new TermVectorsTermsWriter.PostingList[1];
+ int postingsCount;
+ */
+
+ public TermVectorsTermsWriterPerThread(TermsHashPerThread termsHashPerThread, TermVectorsTermsWriter termsWriter) {
+ this.termsWriter = termsWriter;
+ this.termsHashPerThread = termsHashPerThread;
+ docState = termsHashPerThread.docState;
+ }
+
+ // Used by perField when serializing the term vectors
+ final ByteSliceReader vectorSliceReader = new ByteSliceReader();
+
+ final UnicodeUtil.UTF8Result utf8Results[] = {new UnicodeUtil.UTF8Result(),
+ new UnicodeUtil.UTF8Result()};
+
+ public void startDocument() {
+ assert clearLastVectorFieldName();
+ if (doc != null) {
+ doc.reset();
+ doc.docID = docState.docID;
+ }
+ }
+
+ public DocumentsWriter.DocWriter finishDocument() {
+ try {
+ return doc;
+ } finally {
+ doc = null;
+ }
+ }
+
+ public TermsHashConsumerPerField addField(TermsHashPerField termsHashPerField, FieldInfo fieldInfo) {
+ return new TermVectorsTermsWriterPerField(termsHashPerField, this, fieldInfo);
+ }
+
+ public void abort() {
+ if (doc != null) {
+ doc.abort();
+ doc = null;
+ }
+ }
+
+ // Called only by assert
+ final boolean clearLastVectorFieldName() {
+ lastVectorFieldName = null;
+ return true;
+ }
+
+ // Called only by assert
+ String lastVectorFieldName;
+ final boolean vectorFieldsInOrder(FieldInfo fi) {
+ try {
+ if (lastVectorFieldName != null)
+ return lastVectorFieldName.compareTo(fi.name) < 0;
+ else
+ return true;
+ } finally {
+ lastVectorFieldName = fi.name;
+ }
+ }
+}
Property changes on: src/java/org/apache/lucene/index/TermVectorsTermsWriterPerThread.java
___________________________________________________________________
Name: svn:eol-style
+ native
Index: src/java/org/apache/lucene/index/NormsWriter.java
===================================================================
--- src/java/org/apache/lucene/index/NormsWriter.java (revision 0)
+++ src/java/org/apache/lucene/index/NormsWriter.java (revision 0)
@@ -0,0 +1,175 @@
+package org.apache.lucene.index;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.List;
+import java.util.ArrayList;
+
+import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.search.Similarity;
+
+// TODO FI: norms could actually be stored as doc store
+
+/** Writes norms. Each thread X field accumulates the norms
+ * for the doc/fields it saw, then the flush method below
+ * merges all of these together into a single _X.nrm file.
+ */
+
+class NormsWriter extends InvertedDocEndConsumer {
+
+ private static final byte defaultNorm = Similarity.encodeNorm(1.0f);
+ private FieldInfos fieldInfos;
+ public InvertedDocEndConsumerPerThread addThread(DocInverterPerThread docInverterPerThread) {
+ return new NormsWriterPerThread(docInverterPerThread, this);
+ }
+
+ public void abort() {}
+
+ // We only write the _X.nrm file at flush
+ void files(Collection files) {}
+
+ void setFieldInfos(FieldInfos fieldInfos) {
+ this.fieldInfos = fieldInfos;
+ }
+
+ /** Produce _X.nrm if any document had a field with norms
+ * not disabled */
+ public void flush(Map threadsAndFields, DocumentsWriter.FlushState state) throws IOException {
+
+ final Map byField = new HashMap();
+
+ // Typically, each thread will have encountered the same
+ // field. So first we collate by field, ie, all
+ // per-thread field instances that correspond to the
+ // same FieldInfo
+ final Iterator it = threadsAndFields.entrySet().iterator();
+ while(it.hasNext()) {
+ Map.Entry entry = (Map.Entry) it.next();
+
+ Collection fields = (Collection) entry.getValue();
+ Iterator fieldsIt = fields.iterator();
+
+ while(fieldsIt.hasNext()) {
+ NormsWriterPerField perField = (NormsWriterPerField) fieldsIt.next();
+
+ if (perField.upto > 0) {
+ // It has some norms
+ List l = (List) byField.get(perField.fieldInfo);
+ if (l == null) {
+ l = new ArrayList();
+ byField.put(perField.fieldInfo, l);
+ }
+ l.add(perField);
+ } else
+ // Remove this field since we haven't seen it
+ // since the previous flush
+ fieldsIt.remove();
+ }
+ }
+
+ final String normsFileName = state.segmentName + "." + IndexFileNames.NORMS_EXTENSION;
+ state.flushedFiles.add(normsFileName);
+ IndexOutput normsOut = state.directory.createOutput(normsFileName);
+
+ try {
+ normsOut.writeBytes(SegmentMerger.NORMS_HEADER, 0, SegmentMerger.NORMS_HEADER.length);
+
+ final int numField = fieldInfos.size();
+
+ int normCount = 0;
+
+ for(int fieldNumber=0;fieldNumber<numField;fieldNumber++) {
+
+ final FieldInfo fieldInfo = fieldInfos.fieldInfo(fieldNumber);
+
+ List toMerge = (List) byField.get(fieldInfo);
+ int upto = 0;
+ if (toMerge != null) {
+
+ final int numFields = toMerge.size();
+
+ normCount++;
+
+ final NormsWriterPerField[] fields = new NormsWriterPerField[numFields];
+ int[] uptos = new int[numFields];
+
+ for(int j=0;j<numFields;j++)
+ fields[j] = (NormsWriterPerField) toMerge.get(j);
+
+ int numLeft = numFields;
+
+ while(numLeft > 0) {
+
+ assert uptos[0] < fields[0].docIDs.length : " uptos[0]=" + uptos[0] + " len=" + (fields[0].docIDs.length);
+
+ int minLoc = 0;
+ int minDocID = fields[0].docIDs[uptos[0]];
+
+ for(int j=1;j<numLeft;j++) {
+ final int docID = fields[j].docIDs[uptos[j]];
+ if (docID < minDocID) {
+ minDocID = docID;
+ minLoc = j;
+ }
+ }
+
+ assert minDocID < state.numDocsInRAM;
+
+ // Fill hole
+ for(;upto<minDocID;upto++)
+ normsOut.writeByte(defaultNorm);
+
+ normsOut.writeByte(fields[minLoc].norms[uptos[minLoc]]);
+ (uptos[minLoc])++;
+ upto++;
+
+ if (uptos[minLoc] == fields[minLoc].upto) {
+ fields[minLoc].reset();
+ if (minLoc != numLeft-1) {
+ fields[minLoc] = fields[numLeft-1];
+ uptos[minLoc] = uptos[numLeft-1];
+ }
+ numLeft--;
+ }
+ }
+
+ // Fill final hole with defaultNorm
+ for(;upto<state.numDocsInRAM;upto++)
+ normsOut.writeByte(defaultNorm);
+ } else if (fieldInfo.isIndexed && !fieldInfo.omitNorms) {
+ normCount++;
+ // Fill entire field with default norm:
+ for(;upto<state.numDocsInRAM;upto++)
+ normsOut.writeByte(defaultNorm);
+ }
+
+ assert 4+normCount*state.numDocsInRAM == normsOut.getFilePointer() : ".nrm file size mismatch: expected=" + (4+normCount*state.numDocsInRAM) + " actual=" + normsOut.getFilePointer();
+ }
+
+ } finally {
+ normsOut.close();
+ }
+ }
+
+ void closeDocStore(DocumentsWriter.FlushState state) {}
+}
Property changes on: src/java/org/apache/lucene/index/NormsWriter.java
___________________________________________________________________
Name: svn:eol-style
+ native
Index: src/java/org/apache/lucene/store/RAMOutputStream.java
===================================================================
--- src/java/org/apache/lucene/store/RAMOutputStream.java (revision 664673)
+++ src/java/org/apache/lucene/store/RAMOutputStream.java (working copy)
@@ -149,4 +149,9 @@
public long getFilePointer() {
return currentBufferIndex < 0 ? 0 : bufferStart + bufferPosition;
}
+
+ /** Returns byte usage of all buffers. */
+ public long sizeInBytes() {
+ return file.numBuffers() * BUFFER_SIZE;
+ }
}
Index: src/java/org/apache/lucene/util/ArrayUtil.java
===================================================================
--- src/java/org/apache/lucene/util/ArrayUtil.java (revision 0)
+++ src/java/org/apache/lucene/util/ArrayUtil.java (revision 0)
@@ -0,0 +1,109 @@
+package org.apache.lucene.util;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+public final class ArrayUtil {
+
+ public static int getNextSize(int targetSize) {
+ /* This over-allocates proportional to the list size, making room
+ * for additional growth. The over-allocation is mild, but is
+ * enough to give linear-time amortized behavior over a long
+ * sequence of appends() in the presence of a poorly-performing
+ * system realloc().
+ * The growth pattern is: 0, 4, 8, 16, 25, 35, 46, 58, 72, 88, ...
+ */
+ return (targetSize >> 3) + (targetSize < 9 ? 3 : 6) + targetSize;
+ }
+
+ public static int getShrinkSize(int currentSize, int targetSize) {
+ final int newSize = getNextSize(targetSize);
+ if (newSize < currentSize && currentSize > newSize*1.25)
+ return newSize;
+ else
+ return currentSize;
+ }
+
+ public static int[] grow(int[] array, int minSize) {
+ if (array.length < minSize) {
+ int[] newArray = new int[getNextSize(minSize)];
+ System.arraycopy(array, 0, newArray, 0, array.length);
+ return newArray;
+ } else
+ return array;
+ }
+
+ public static int[] grow(int[] array) {
+ return grow(array, 1+array.length);
+ }
+
+ public static int[] shrink(int[] array, int targetSize) {
+ final int newSize = getShrinkSize(array.length, targetSize);
+ if (newSize != array.length) {
+ int[] newArray = new int[newSize];
+ System.arraycopy(array, 0, newArray, 0, newSize);
+ return newArray;
+ } else
+ return array;
+ }
+
+ public static long[] grow(long[] array, int minSize) {
+ if (array.length < minSize) {
+ long[] newArray = new long[getNextSize(minSize)];
+ System.arraycopy(array, 0, newArray, 0, array.length);
+ return newArray;
+ } else
+ return array;
+ }
+
+ public static long[] grow(long[] array) {
+ return grow(array, 1+array.length);
+ }
+
+ public static long[] shrink(long[] array, int targetSize) {
+ final int newSize = getShrinkSize(array.length, targetSize);
+ if (newSize != array.length) {
+ long[] newArray = new long[newSize];
+ System.arraycopy(array, 0, newArray, 0, newSize);
+ return newArray;
+ } else
+ return array;
+ }
+
+ public static byte[] grow(byte[] array, int minSize) {
+ if (array.length < minSize) {
+ byte[] newArray = new byte[getNextSize(minSize)];
+ System.arraycopy(array, 0, newArray, 0, array.length);
+ return newArray;
+ } else
+ return array;
+ }
+
+ public static byte[] grow(byte[] array) {
+ return grow(array, 1+array.length);
+ }
+
+ public static byte[] shrink(byte[] array, int targetSize) {
+ final int newSize = getShrinkSize(array.length, targetSize);
+ if (newSize != array.length) {
+ byte[] newArray = new byte[newSize];
+ System.arraycopy(array, 0, newArray, 0, newSize);
+ return newArray;
+ } else
+ return array;
+ }
+}
Index: LICENSE.txt
===================================================================
--- LICENSE.txt (revision 664673)
+++ LICENSE.txt (working copy)
@@ -229,3 +229,12 @@
* for internal or external distribution as long as this notice
* remains attached.
*/
+
+
+Some code in src/java/org/apache/lucene/util/ArrayUtil.java was
+derived from Python sources available at http://www.python.org. Full
+license is here:
+
+ http://www.python.org/download/releases/2.4.2/license/
+
+