1 package org.apache.lucene.store.instantiated;
4 * Copyright 2006 The Apache Software Foundation
6 * Licensed under the Apache License, Version 2.0 (the "License");
7 * you may not use this file except in compliance with the License.
8 * You may obtain a copy of the License at
10 * http://www.apache.org/licenses/LICENSE-2.0
12 * Unless required by applicable law or agreed to in writing, software
13 * distributed under the License is distributed on an "AS IS" BASIS,
14 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 * See the License for the specific language governing permissions and
16 * limitations under the License.
19 import java.io.IOException;
20 import java.io.PrintStream;
21 import java.io.StringReader;
22 import java.io.Closeable;
23 import java.util.ArrayList;
24 import java.util.Arrays;
25 import java.util.Collections;
26 import java.util.Comparator;
27 import java.util.HashMap;
28 import java.util.HashSet;
29 import java.util.Iterator;
30 import java.util.LinkedHashMap;
31 import java.util.LinkedList;
32 import java.util.List;
36 import org.apache.lucene.analysis.Analyzer;
37 import org.apache.lucene.analysis.Token;
38 import org.apache.lucene.analysis.TokenStream;
39 import org.apache.lucene.document.Document;
40 import org.apache.lucene.document.Fieldable;
41 import org.apache.lucene.index.FieldInvertState;
42 import org.apache.lucene.index.IndexReader;
43 import org.apache.lucene.index.IndexWriter;
44 import org.apache.lucene.index.Term;
45 import org.apache.lucene.index.TermVectorOffsetInfo;
46 import org.apache.lucene.search.Similarity;
47 import org.apache.lucene.util.StringHelper;
48 import org.apache.lucene.util.ArrayUtil;
49 import org.apache.lucene.util.CollectionUtil;
50 import org.apache.lucene.util.AttributeImpl;
51 import org.apache.lucene.util.BitVector;
54 * This class, similar to {@link org.apache.lucene.index.IndexWriter}, has no locking mechanism.
56 * {@link org.apache.lucene.store.instantiated.InstantiatedIndexReader} is navigating
57 * the same instances in memory as this writer is updating so searchers active while
58 * you are committing are bound to throw exceptions.
60 * Consider using InstantiatedIndex as if it was immutable.
62 * @see org.apache.lucene.index.IndexWriter
64 public class InstantiatedIndexWriter implements Closeable {
66 private PrintStream infoStream = null;
68 private int maxFieldLength = IndexWriter.DEFAULT_MAX_FIELD_LENGTH;
70 private final InstantiatedIndex index;
71 private final Analyzer analyzer;
73 private Similarity similarity = Similarity.getDefault(); // how to normalize;
75 private transient Set<String> fieldNameBuffer;
77 * linked to ensure chronological order
79 private Map<InstantiatedDocument, Map<FieldSetting, Map<String /*text*/, TermDocumentInformationFactory>>> termDocumentInformationFactoryByDocument = new LinkedHashMap<InstantiatedDocument, Map<FieldSetting, Map<String /*text*/, TermDocumentInformationFactory>>>(2000);
81 private Set<InstantiatedDocument> unflushedDocuments = new HashSet<InstantiatedDocument>();
83 public InstantiatedIndexWriter(InstantiatedIndex index) throws IOException {
87 public InstantiatedIndexWriter(InstantiatedIndex index, Analyzer analyzer) throws IOException {
88 this(index, analyzer, false);
91 public InstantiatedIndexWriter(InstantiatedIndex index, Analyzer analyzer, boolean create) throws IOException {
93 this.analyzer = analyzer;
94 fieldNameBuffer = new HashSet<String>();
96 this.index.initialize();
100 private int mergeFactor = 2500;
103 * The sweetspot for this implementation is somewhere around 2500 at 2K text large documents.
107 * ------------> Report sum by Prefix (MAddDocs) and Round (8 about 8 out of 160153)
108 * Operation round mrg buf cmpnd runCnt recsPerRun rec/s elapsedSec avgUsedMem avgTotalMem
109 * MAddDocs_20000 0 10 10 true 1 20000 81,4 245,68 200 325 152 268 156 928
110 * MAddDocs_20000 - 1 1000 10 true - - 1 - - 20000 - - 494,1 - - 40,47 - 247 119 072 - 347 025 408
111 * MAddDocs_20000 2 10 100 true 1 20000 104,8 190,81 233 895 552 363 720 704
112 * MAddDocs_20000 - 3 2000 100 true - - 1 - - 20000 - - 527,2 - - 37,94 - 266 136 448 - 378 273 792
113 * MAddDocs_20000 4 10 10 false 1 20000 103,2 193,75 222 089 792 378 273 792
114 * MAddDocs_20000 - 5 3000 10 false - - 1 - - 20000 - - 545,2 - - 36,69 - 237 917 152 - 378 273 792
115 * MAddDocs_20000 6 10 100 false 1 20000 102,7 194,67 237 018 976 378 273 792
116 * MAddDocs_20000 - 7 4000 100 false - - 1 - - 20000 - - 535,8 - - 37,33 - 309 680 640 - 501 968 896
119 * @see org.apache.lucene.index.LogMergePolicy#setMergeFactor(int)
121 public void setMergeFactor(int mergeFactor) {
122 this.mergeFactor = mergeFactor;
126 * @see org.apache.lucene.index.LogMergePolicy#getMergeFactor()
128 public int getMergeFactor() {
134 * If non-null, information about merges and a message when
135 * maxFieldLength is reached could be printed to this -- currently
136 * not yet implemented.
138 public void setInfoStream(PrintStream infoStream) {
139 this.infoStream = infoStream;
143 public void abort() throws IOException {
148 public void addIndexes(IndexReader[] readers) {
149 throw new RuntimeException("Not implemented");
153 public PrintStream getInfoStream() {
159 * Flushes all changes to an index and closes all associated files.
161 public void close() throws IOException {
166 * Returns the number of documents currently in this index.
168 public int docCount() {
169 // todo: not certain. see http://www.nabble.com/IndexWriter.docCount-tf3128882.html#a8669483
170 return index.getDocumentsByNumber().length /* - index.getDeletedDocuments().size() */ + unflushedDocuments.size();
174 * Locks the index and commits the buffered documents.
176 public void commit() throws IOException {
178 // todo write lock, unless held by caller
180 boolean orderedTermsDirty = false;
181 Set<InstantiatedTerm> dirtyTerms = new HashSet<InstantiatedTerm>(1000);
183 Map<String, FieldSetting> fieldSettingsByFieldName = new HashMap<String, FieldSetting>();
184 for (String fieldName : fieldNameBuffer) {
185 fieldSettingsByFieldName.put(fieldName, new FieldSetting(fieldName));
188 InstantiatedDocument[] documentsByNumber = new InstantiatedDocument[index.getDocumentsByNumber().length + termDocumentInformationFactoryByDocument.size()];
189 System.arraycopy(index.getDocumentsByNumber(), 0, documentsByNumber, 0, index.getDocumentsByNumber().length);
190 int documentNumber = index.getDocumentsByNumber().length;
192 List<InstantiatedTerm> orderedTerms = new ArrayList<InstantiatedTerm>(index.getOrderedTerms().length + 5000);
193 for (InstantiatedTerm instantiatedTerm : index.getOrderedTerms()) {
194 orderedTerms.add(instantiatedTerm);
197 // update norm array with fake values for new documents
198 Map<String, byte[]> normsByFieldNameAndDocumentNumber = new HashMap<String, byte[]>(index.getTermsByFieldAndText().size());
199 Set<String> fieldNames = new HashSet<String>(20);
200 fieldNames.addAll(index.getNormsByFieldNameAndDocumentNumber().keySet());
201 fieldNames.addAll(fieldNameBuffer);
202 for (String field : index.getTermsByFieldAndText().keySet()) {
203 byte[] norms = new byte[index.getDocumentsByNumber().length + termDocumentInformationFactoryByDocument.size()];
204 byte[] oldNorms = index.getNormsByFieldNameAndDocumentNumber().get(field);
205 if (oldNorms != null) {
206 System.arraycopy(oldNorms, 0, norms, 0, oldNorms.length);
207 Arrays.fill(norms, oldNorms.length, norms.length, similarity.encodeNormValue(1.0f));
209 Arrays.fill(norms, 0, norms.length, similarity.encodeNormValue(1.0f));
211 normsByFieldNameAndDocumentNumber.put(field, norms);
212 fieldNames.remove(field);
214 for (String field : fieldNames) {
215 //System.out.println(field);
216 byte[] norms = new byte[index.getDocumentsByNumber().length + termDocumentInformationFactoryByDocument.size()];
217 Arrays.fill(norms, 0, norms.length, similarity.encodeNormValue(1.0f));
218 normsByFieldNameAndDocumentNumber.put(field, norms);
221 index.setNormsByFieldNameAndDocumentNumber(normsByFieldNameAndDocumentNumber);
223 for (Map.Entry<InstantiatedDocument, Map<FieldSetting, Map<String /*text*/, TermDocumentInformationFactory>>> eDocumentTermDocInfoByTermTextAndField : termDocumentInformationFactoryByDocument.entrySet()) {
225 InstantiatedDocument document = eDocumentTermDocInfoByTermTextAndField.getKey();
227 // assign document number
228 document.setDocumentNumber(documentNumber++);
229 documentsByNumber[document.getDocumentNumber()] = document;
231 // set norms, prepare document and create optimized size collections.
233 int numFieldsWithTermVectorsInDocument = 0;
234 int termsInDocument = 0;
235 for (Map.Entry<FieldSetting, Map<String /*text*/, TermDocumentInformationFactory>> eFieldTermDocInfoFactoriesByTermText : eDocumentTermDocInfoByTermTextAndField.getValue().entrySet()) {
236 if (eFieldTermDocInfoFactoriesByTermText.getKey().storeTermVector) {
237 numFieldsWithTermVectorsInDocument += eFieldTermDocInfoFactoriesByTermText.getValue().size();
239 termsInDocument += eFieldTermDocInfoFactoriesByTermText.getValue().size();
241 if (eFieldTermDocInfoFactoriesByTermText.getKey().indexed && !eFieldTermDocInfoFactoriesByTermText.getKey().omitNorms) {
242 final FieldInvertState invertState = new FieldInvertState();
243 invertState.setBoost(eFieldTermDocInfoFactoriesByTermText.getKey().boost * document.getDocument().getBoost());
244 invertState.setLength(eFieldTermDocInfoFactoriesByTermText.getKey().fieldLength);
245 final float norm = similarity.computeNorm(eFieldTermDocInfoFactoriesByTermText.getKey().fieldName, invertState);
246 normsByFieldNameAndDocumentNumber.get(eFieldTermDocInfoFactoriesByTermText.getKey().fieldName)[document.getDocumentNumber()] = similarity.encodeNormValue(norm);
248 System.currentTimeMillis();
253 /** used for term vectors only, i think.. */
254 Map<InstantiatedTerm, InstantiatedTermDocumentInformation> informationByTermOfCurrentDocument = new HashMap<InstantiatedTerm, InstantiatedTermDocumentInformation>(termsInDocument);
257 Map<String, FieldSetting> documentFieldSettingsByFieldName = new HashMap<String, FieldSetting>(eDocumentTermDocInfoByTermTextAndField.getValue().size());
260 for (Map.Entry<FieldSetting, Map<String /*text*/, TermDocumentInformationFactory>> eFieldSetting_TermDocInfoFactoriesByTermText : eDocumentTermDocInfoByTermTextAndField.getValue().entrySet()) {
261 documentFieldSettingsByFieldName.put(eFieldSetting_TermDocInfoFactoriesByTermText.getKey().fieldName, eFieldSetting_TermDocInfoFactoriesByTermText.getKey());
263 // find or create term
264 for (Map.Entry<String /*text*/, TermDocumentInformationFactory> eTermText_TermDocInfoFactory : eFieldSetting_TermDocInfoFactoriesByTermText.getValue().entrySet()) {
267 InstantiatedTerm term;
268 Map<String, InstantiatedTerm> termsByText = index.getTermsByFieldAndText().get(eFieldSetting_TermDocInfoFactoriesByTermText.getKey().fieldName);
269 if (termsByText == null) {
270 termsByText = new HashMap<String, InstantiatedTerm>(1000);
271 index.getTermsByFieldAndText().put(eFieldSetting_TermDocInfoFactoriesByTermText.getKey().fieldName, termsByText);
272 term = new InstantiatedTerm(eFieldSetting_TermDocInfoFactoriesByTermText.getKey().fieldName, eTermText_TermDocInfoFactory.getKey());
273 termsByText.put(eTermText_TermDocInfoFactory.getKey(), term);
274 int pos = Collections.binarySearch(orderedTerms, term, InstantiatedTerm.comparator);
276 orderedTerms.add(pos, term);
277 orderedTermsDirty = true;
279 term = termsByText.get(eTermText_TermDocInfoFactory.getKey());
281 term = new InstantiatedTerm(eFieldSetting_TermDocInfoFactoriesByTermText.getKey().fieldName, eTermText_TermDocInfoFactory.getKey());
282 termsByText.put(eTermText_TermDocInfoFactory.getKey(), term);
283 int pos = Collections.binarySearch(orderedTerms, term, InstantiatedTerm.comparator);
285 orderedTerms.add(pos, term);
286 orderedTermsDirty = true;
290 // create association term document information
292 // [Term]-- {0..*} | {0..* ordered} --(field)[Document]
295 // [TermDocumentInformation]
297 int[] positions = new int[eTermText_TermDocInfoFactory.getValue().termPositions.size()];
298 for (int i = 0; i < positions.length; i++) {
299 positions[i] = eTermText_TermDocInfoFactory.getValue().termPositions.get(i);
302 byte[][] payloads = new byte[eTermText_TermDocInfoFactory.getValue().payloads.size()][];
303 for (int i = 0; i < payloads.length; i++) {
304 payloads[i] = eTermText_TermDocInfoFactory.getValue().payloads.get(i);
309 InstantiatedTermDocumentInformation info = new InstantiatedTermDocumentInformation(term, document, /*eTermText_TermDocInfoFactory.getValue().termFrequency,*/ positions, payloads);
311 // todo optimize, this should be cached and updated to array in batches rather than appending the array once for every position!
312 InstantiatedTermDocumentInformation[] associatedDocuments;
313 if (term.getAssociatedDocuments() != null) {
314 associatedDocuments = new InstantiatedTermDocumentInformation[term.getAssociatedDocuments().length + 1];
315 System.arraycopy(term.getAssociatedDocuments(), 0, associatedDocuments, 0, term.getAssociatedDocuments().length);
317 associatedDocuments = new InstantiatedTermDocumentInformation[1];
319 associatedDocuments[associatedDocuments.length - 1] = info;
320 term.setAssociatedDocuments(associatedDocuments);
322 // todo optimize, only if term vector?
323 informationByTermOfCurrentDocument.put(term, info);
326 dirtyTerms.add(term);
329 // term vector offsets
330 if (eFieldSetting_TermDocInfoFactoriesByTermText.getKey().storeOffsetWithTermVector) {
331 for (Map.Entry<InstantiatedTerm, InstantiatedTermDocumentInformation> e : informationByTermOfCurrentDocument.entrySet()) {
332 if (eFieldSetting_TermDocInfoFactoriesByTermText.getKey().fieldName.equals(e.getKey().field())) {
333 TermDocumentInformationFactory factory = eFieldSetting_TermDocInfoFactoriesByTermText.getValue().get(e.getKey().text());
334 e.getValue().setTermOffsets(factory.termOffsets.toArray(new TermVectorOffsetInfo[factory.termOffsets.size()]));
340 Map<String, List<InstantiatedTermDocumentInformation>> termDocumentInformationsByField = new HashMap<String, List<InstantiatedTermDocumentInformation>>();
341 for (Map.Entry<InstantiatedTerm, InstantiatedTermDocumentInformation> eTerm_TermDocumentInformation : informationByTermOfCurrentDocument.entrySet()) {
342 List<InstantiatedTermDocumentInformation> termDocumentInformations = termDocumentInformationsByField.get(eTerm_TermDocumentInformation.getKey().field());
343 if (termDocumentInformations == null) {
344 termDocumentInformations = new ArrayList<InstantiatedTermDocumentInformation>();
345 termDocumentInformationsByField.put(eTerm_TermDocumentInformation.getKey().field(), termDocumentInformations);
347 termDocumentInformations.add(eTerm_TermDocumentInformation.getValue());
350 for (Map.Entry<String, List<InstantiatedTermDocumentInformation>> eField_TermDocInfos : termDocumentInformationsByField.entrySet()) {
352 CollectionUtil.quickSort(eField_TermDocInfos.getValue(), tdComp);
355 if (documentFieldSettingsByFieldName.get(eField_TermDocInfos.getKey()).storeTermVector) {
356 if (document.getVectorSpace() == null) {
357 document.setVectorSpace(new HashMap<String, List<InstantiatedTermDocumentInformation>>(documentFieldSettingsByFieldName.size()));
359 document.getVectorSpace().put(eField_TermDocInfos.getKey(), eField_TermDocInfos.getValue());
363 fieldSettingsByFieldName.putAll(documentFieldSettingsByFieldName);
366 // order document informations in dirty terms
367 for (InstantiatedTerm term : dirtyTerms) {
368 // todo optimize, i believe this is useless, that the natural order is document number?
369 ArrayUtil.mergeSort(term.getAssociatedDocuments(), InstantiatedTermDocumentInformation.documentNumberComparator);
371 // // update association class reference for speedy skipTo()
372 // for (int i = 0; i < term.getAssociatedDocuments().length; i++) {
373 // term.getAssociatedDocuments()[i].setIndexFromTerm(i);
379 index.setDocumentsByNumber(documentsByNumber);
380 index.setOrderedTerms(orderedTerms.toArray(new InstantiatedTerm[orderedTerms.size()]));
382 for (FieldSetting fieldSetting : fieldSettingsByFieldName.values()) {
383 index.getFieldSettings().merge(fieldSetting);
386 if (orderedTermsDirty) {
387 // todo optimize, only update from start position
388 for (int i = 0; i < index.getOrderedTerms().length; i++) {
389 index.getOrderedTerms()[i].setTermIndex(i);
394 // remove deleted documents
395 IndexReader indexDeleter = index.indexReaderFactory();
396 if (unflushedDeletions.size() > 0) {
397 for (Term term : unflushedDeletions) {
398 indexDeleter.deleteDocuments(term);
400 unflushedDeletions.clear();
404 // all done, clear buffers
405 unflushedDocuments.clear();
406 termDocumentInformationFactoryByDocument.clear();
407 fieldNameBuffer.clear();
410 // update deleted documents bitset
411 if (index.getDeletedDocuments() != null) {
412 BitVector deletedDocuments = new BitVector(index.getDocumentsByNumber().length);
413 for (int i = 0; i < index.getDeletedDocuments().size(); i++) {
414 if (index.getDeletedDocuments().get(i)) {
415 deletedDocuments.set(i);
418 index.setDeletedDocuments(deletedDocuments);
421 index.setVersion(System.currentTimeMillis());
425 indexDeleter.close();
429 private static final Comparator<InstantiatedTermDocumentInformation> tdComp = new Comparator<InstantiatedTermDocumentInformation>() {
430 public int compare(InstantiatedTermDocumentInformation instantiatedTermDocumentInformation, InstantiatedTermDocumentInformation instantiatedTermDocumentInformation1) {
431 return instantiatedTermDocumentInformation.getTerm().getTerm().compareTo(instantiatedTermDocumentInformation1.getTerm().getTerm());
436 * Adds a document to this index. If the document contains more than
437 * {@link #setMaxFieldLength(int)} terms for a given field, the remainder are
440 public void addDocument(Document doc) throws IOException {
441 addDocument(doc, getAnalyzer());
445 * Adds a document to this index, using the provided analyzer instead of the
446 * value of {@link #getAnalyzer()}. If the document contains more than
447 * {@link #setMaxFieldLength(int)} terms for a given field, the remainder are
452 * @throws IOException
454 public void addDocument(Document doc, Analyzer analyzer) throws IOException {
455 addDocument(new InstantiatedDocument(doc), analyzer);
459 * Tokenizes a document and adds it to the buffer.
460 * Try to do all calculations in this method rather than in commit, as this is a non locking method.
461 * Remember, this index implementation expects unlimited memory for maximum speed.
465 * @throws IOException
467 protected void addDocument(InstantiatedDocument document, Analyzer analyzer) throws IOException {
469 if (document.getDocumentNumber() != null) {
470 throw new RuntimeException("Document number already set! Are you trying to add a document that already is bound to this or another index?");
475 // normalize settings per field name in document
477 Map<String /* field name */, FieldSetting> fieldSettingsByFieldName = new HashMap<String, FieldSetting>();
478 for (Fieldable field : document.getDocument().getFields()) {
479 FieldSetting fieldSetting = fieldSettingsByFieldName.get(field.name());
480 if (fieldSetting == null) {
481 fieldSetting = new FieldSetting();
482 fieldSetting.fieldName = StringHelper.intern(field.name());
483 fieldSettingsByFieldName.put(fieldSetting.fieldName, fieldSetting);
484 fieldNameBuffer.add(fieldSetting.fieldName);
487 // todo: fixme: multiple fields with the same name does not mean field boost += more boost.
488 fieldSetting.boost *= field.getBoost();
489 //fieldSettings.dimensions++;
492 // once fieldSettings, always fieldSettings.
493 if (field.getOmitNorms()) {
494 fieldSetting.omitNorms = true;
496 if (field.isIndexed() ) {
497 fieldSetting.indexed = true;
499 if (field.isTokenized()) {
500 fieldSetting.tokenized = true;
502 if (field.isStored()) {
503 fieldSetting.stored = true;
505 if (field.isBinary()) {
506 fieldSetting.isBinary = true;
508 if (field.isTermVectorStored()) {
509 fieldSetting.storeTermVector = true;
511 if (field.isStorePositionWithTermVector()) {
512 fieldSetting.storePositionWithTermVector = true;
514 if (field.isStoreOffsetWithTermVector()) {
515 fieldSetting.storeOffsetWithTermVector = true;
519 Map<Fieldable, LinkedList<Token>> tokensByField = new LinkedHashMap<Fieldable, LinkedList<Token>>(20);
521 // tokenize indexed fields.
522 for (Iterator<Fieldable> it = document.getDocument().getFields().iterator(); it.hasNext();) {
524 Fieldable field = it.next();
526 FieldSetting fieldSetting = fieldSettingsByFieldName.get(field.name());
528 if (field.isIndexed()) {
530 LinkedList<Token> tokens = new LinkedList<Token>();
531 tokensByField.put(field, tokens);
533 if (field.isTokenized()) {
534 final TokenStream tokenStream;
535 // todo readerValue(), binaryValue()
536 if (field.tokenStreamValue() != null) {
537 tokenStream = field.tokenStreamValue();
539 tokenStream = analyzer.reusableTokenStream(field.name(), new StringReader(field.stringValue()));
542 // reset the TokenStream to the first token
545 while (tokenStream.incrementToken()) {
546 // TODO: this is a simple workaround to still work with tokens, not very effective, but as far as I know, this writer should get removed soon:
547 final Token token = new Token();
548 for (Iterator<AttributeImpl> atts = tokenStream.getAttributeImplsIterator(); atts.hasNext();) {
549 final AttributeImpl att = atts.next();
552 } catch (Exception e) {
553 // ignore unsupported attributes,
554 // this may fail to copy some attributes, if a special combined AttributeImpl is used, that
555 // implements basic attributes supported by Token and also other customized ones in one class.
558 tokens.add(token); // the vector will be built on commit.
559 fieldSetting.fieldLength++;
560 if (fieldSetting.fieldLength > maxFieldLength) {
568 String fieldVal = field.stringValue();
569 Token token = new Token(0, fieldVal.length(), "untokenized");
570 token.setEmpty().append(fieldVal);
572 fieldSetting.fieldLength++;
576 if (!field.isStored()) {
582 Map<FieldSetting, Map<String /*text*/, TermDocumentInformationFactory>> termDocumentInformationFactoryByTermTextAndFieldSetting = new HashMap<FieldSetting, Map<String /*text*/, TermDocumentInformationFactory>>();
583 termDocumentInformationFactoryByDocument.put(document, termDocumentInformationFactoryByTermTextAndFieldSetting);
585 // build term vector, term positions and term offsets
586 for (Map.Entry<Fieldable, LinkedList<Token>> eField_Tokens : tokensByField.entrySet()) {
587 FieldSetting fieldSetting = fieldSettingsByFieldName.get(eField_Tokens.getKey().name());
589 Map<String, TermDocumentInformationFactory> termDocumentInformationFactoryByTermText = termDocumentInformationFactoryByTermTextAndFieldSetting.get(fieldSettingsByFieldName.get(eField_Tokens.getKey().name()));
590 if (termDocumentInformationFactoryByTermText == null) {
591 termDocumentInformationFactoryByTermText = new HashMap<String /*text*/, TermDocumentInformationFactory>();
592 termDocumentInformationFactoryByTermTextAndFieldSetting.put(fieldSettingsByFieldName.get(eField_Tokens.getKey().name()), termDocumentInformationFactoryByTermText);
597 // for each new field, move positions a bunch.
598 if (fieldSetting.position > 0) {
599 // todo what if no analyzer set, multiple fields with same name and index without tokenization?
600 fieldSetting.position += analyzer.getPositionIncrementGap(fieldSetting.fieldName);
603 for (Token token : eField_Tokens.getValue()) {
605 TermDocumentInformationFactory termDocumentInformationFactory = termDocumentInformationFactoryByTermText.get(token.toString());
606 if (termDocumentInformationFactory == null) {
607 termDocumentInformationFactory = new TermDocumentInformationFactory();
608 termDocumentInformationFactoryByTermText.put(token.toString(), termDocumentInformationFactory);
610 //termDocumentInformationFactory.termFrequency++;
612 fieldSetting.position += (token.getPositionIncrement() - 1);
613 termDocumentInformationFactory.termPositions.add(fieldSetting.position++);
615 if (token.getPayload() != null && token.getPayload().length() > 0) {
616 termDocumentInformationFactory.payloads.add(token.getPayload().toByteArray());
617 fieldSetting.storePayloads = true;
619 termDocumentInformationFactory.payloads.add(null);
622 if (eField_Tokens.getKey().isStoreOffsetWithTermVector()) {
624 termDocumentInformationFactory.termOffsets.add(new TermVectorOffsetInfo(fieldSetting.offset + token.startOffset(), fieldSetting.offset + token.endOffset()));
625 lastOffset = fieldSetting.offset + token.endOffset();
631 if (eField_Tokens.getKey().isStoreOffsetWithTermVector()) {
632 fieldSetting.offset = lastOffset + 1;
638 unflushedDocuments.add(document);
640 // if too many documents in buffer, commit.
641 if (unflushedDocuments.size() >= getMergeFactor()) {
645 // todo: unlock write lock
650 private Set<Term> unflushedDeletions = new HashSet<Term>();
652 public void deleteDocuments(Term term) throws IOException {
653 unflushedDeletions.add(term);
656 public void deleteDocuments(Term[] terms) throws IOException {
657 for (Term term : terms) {
658 deleteDocuments(term);
662 public void updateDocument(Term term, Document doc) throws IOException {
663 updateDocument(term, doc, getAnalyzer());
666 public void updateDocument(Term term, Document doc, Analyzer analyzer) throws IOException {
667 deleteDocuments(term);
668 addDocument(doc, analyzer);
671 public int getMaxFieldLength() {
672 return maxFieldLength;
675 public void setMaxFieldLength(int maxFieldLength) {
676 this.maxFieldLength = maxFieldLength;
679 public Similarity getSimilarity() {
683 public void setSimilarity(Similarity similarity) {
684 this.similarity = similarity;
687 public Analyzer getAnalyzer() {
691 private class TermDocumentInformationFactory {
692 private LinkedList<byte[]> payloads = new LinkedList<byte[]>();
693 private LinkedList<Integer> termPositions = new LinkedList<Integer>();
694 private LinkedList<TermVectorOffsetInfo> termOffsets = new LinkedList<TermVectorOffsetInfo>();
698 static class FieldSetting extends org.apache.lucene.store.instantiated.FieldSetting {
705 boolean omitNorms = false;
706 boolean isBinary = false;
708 private FieldSetting() {
711 private FieldSetting(String fieldName) {