You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by mi...@apache.org on 2015/01/02 13:02:33 UTC
svn commit: r1649007 [3/10] - in /lucene/dev/branches/lucene6005/lucene:
analysis/common/src/test/org/apache/lucene/analysis/core/
analysis/common/src/test/org/apache/lucene/collation/
benchmark/src/java/org/apache/lucene/benchmark/byTask/tasks/ benchm...
Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/document/FilteringTokenFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/document/FilteringTokenFilter.java?rev=1649007&r1=1649006&r2=1649007&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/document/FilteringTokenFilter.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/document/FilteringTokenFilter.java Fri Jan 2 12:02:31 2015
@@ -23,7 +23,7 @@ import org.apache.lucene.analysis.TokenF
import org.apache.lucene.analysis.TokenStream;
import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
-// nocommit forked from analysis module
+// TODO: forked from analysis module; can we share?
/**
* Abstract base class for TokenFilters that may remove tokens.
Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/document/LengthFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/document/LengthFilter.java?rev=1649007&r1=1649006&r2=1649007&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/document/LengthFilter.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/document/LengthFilter.java Fri Jan 2 12:02:31 2015
@@ -17,7 +17,7 @@ package org.apache.lucene.document;
* limitations under the License.
*/
-// nocommit forked from analysis module
+// TODO: forked from analysis module; can we share?
import org.apache.lucene.analysis.TokenStream;
import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/document/LimitTokenCountFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/document/LimitTokenCountFilter.java?rev=1649007&r1=1649006&r2=1649007&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/document/LimitTokenCountFilter.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/document/LimitTokenCountFilter.java Fri Jan 2 12:02:31 2015
@@ -22,7 +22,7 @@ import java.io.IOException;
import org.apache.lucene.analysis.TokenFilter;
import org.apache.lucene.analysis.TokenStream;
-// nocommit forked from analysis module
+// TODO: forked from analysis module; can we share?
/**
* This TokenFilter limits the number of tokens while indexing. It is
Copied: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/document/SingleTokenTokenizer.java (from r1648667, lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/document/CoreKeywordTokenizer.java)
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/document/SingleTokenTokenizer.java?p2=lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/document/SingleTokenTokenizer.java&p1=lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/document/CoreKeywordTokenizer.java&r1=1648667&r2=1649007&rev=1649007&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/document/CoreKeywordTokenizer.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/document/SingleTokenTokenizer.java Fri Jan 2 12:02:31 2015
@@ -23,10 +23,9 @@ import org.apache.lucene.analysis.Tokeni
import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
-// nocommit ... this is privte fork of KeywordTokenizer ... should we move it to ../analysis and make it public? same deal with
-// LengthFilter and LimitTokenCountFilter:
+// TODO: forked from analysis module; can we share?
-final class CoreKeywordTokenizer extends Tokenizer {
+final class SingleTokenTokenizer extends Tokenizer {
/** Default read buffer size */
public static final int DEFAULT_BUFFER_SIZE = 256;
@@ -35,11 +34,11 @@ final class CoreKeywordTokenizer extends
private final CharTermAttribute termAtt = addAttribute(CharTermAttribute.class);
private final OffsetAttribute offsetAtt = addAttribute(OffsetAttribute.class);
- public CoreKeywordTokenizer() {
+ public SingleTokenTokenizer() {
this(DEFAULT_BUFFER_SIZE);
}
- CoreKeywordTokenizer(int bufferSize) {
+ SingleTokenTokenizer(int bufferSize) {
if (bufferSize <= 0) {
throw new IllegalArgumentException("bufferSize must be > 0");
}
Added: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/AlwaysForceMergePolicy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/AlwaysForceMergePolicy.java?rev=1649007&view=auto
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/AlwaysForceMergePolicy.java (added)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/AlwaysForceMergePolicy.java Fri Jan 2 12:02:31 2015
@@ -0,0 +1,84 @@
+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.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.HashMap;
+
+/** Wraps another {@link MergePolicy}, except when forceMerge is requested, if the wrapped merge
+ * policy woulid do nothing, this one will always merge the one segment. */
+public class AlwaysForceMergePolicy extends MergePolicy {
+
+ /** Wrapped {@link MergePolicy}. */
+ protected final MergePolicy in;
+
+ /** Wrap the given {@link MergePolicy} and intercept forceMerge requests to
+ * only upgrade segments written with previous Lucene versions. */
+ public AlwaysForceMergePolicy(MergePolicy in) {
+ this.in = in;
+ }
+
+ private boolean didForceMerge;
+
+ @Override
+ public MergeSpecification findMerges(MergeTrigger mergeTrigger, SegmentInfos segmentInfos, IndexWriter writer) throws IOException {
+ return in.findMerges(null, segmentInfos, writer);
+ }
+
+ /** Call this to "force" a force merge again. */
+ public void reset() {
+ didForceMerge = false;
+ }
+
+ @Override
+ public MergeSpecification findForcedMerges(SegmentInfos segmentInfos, int maxSegmentCount, Map<SegmentCommitInfo,Boolean> segmentsToMerge, IndexWriter writer) throws IOException {
+ MergeSpecification spec = in.findForcedMerges(segmentInfos, maxSegmentCount, segmentsToMerge, writer);
+ if (spec == null && didForceMerge == false) {
+ didForceMerge = true;
+ List<SegmentCommitInfo> infos = new ArrayList<>();
+ for(SegmentCommitInfo info : segmentInfos) {
+ infos.add(info);
+ }
+ if (infos.isEmpty()) {
+ spec = null;
+ } else {
+ spec = new MergeSpecification();
+ spec.add(new OneMerge(infos));
+ }
+ }
+ return spec;
+ }
+
+ @Override
+ public MergeSpecification findForcedDeletesMerges(SegmentInfos segmentInfos, IndexWriter writer) throws IOException {
+ return in.findForcedDeletesMerges(segmentInfos, writer);
+ }
+
+ @Override
+ public boolean useCompoundFile(SegmentInfos segments, SegmentCommitInfo newSegment, IndexWriter writer) throws IOException {
+ return in.useCompoundFile(segments, newSegment, writer);
+ }
+
+ @Override
+ public String toString() {
+ return "[" + getClass().getSimpleName() + "->" + in + "]";
+ }
+}
Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java?rev=1649007&r1=1649006&r2=1649007&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java Fri Jan 2 12:02:31 2015
@@ -144,6 +144,9 @@ public class CheckIndex implements Close
/** Holds the userData of the last commit in the index */
public Map<String, String> userData;
+ /** How many docs violate unique field constraints and would be deleted with -exorcise. */
+ public int nonUniqueCount;
+
/** Holds the status of each segment in the index.
* See {@link #segmentInfos}.
*
@@ -477,11 +480,8 @@ public class CheckIndex implements Close
return result;
}
- // nocommit TestIndexWriterReader.testAddIndexesAndDoDeletesThreads one time hit EOFE in here:
FieldTypes fieldTypes = FieldTypes.getFieldTypes(sis.getUserData(), null, null);
- // nocommit verify unique atom type is in fact unique
-
// find the oldest and newest segment versions
Version oldest = null;
Version newest = null;
@@ -541,9 +541,10 @@ public class CheckIndex implements Close
result.numSegments = numSegments;
result.userData = sis.getUserData();
String userDataString;
- if (sis.getUserData().size() > 0) {
- // nocommit don't print fieldTypes string? it's huge and ugly?
- userDataString = " userData=" + sis.getUserData();
+ Map<String,String> userData = new HashMap<String,String>(sis.getUserData());
+ userData.remove(FieldTypes.FIELD_TYPES_KEY);
+ if (userData.isEmpty() == false) {
+ userDataString = " userData=" + userData;
} else {
userDataString = "";
}
@@ -580,9 +581,14 @@ public class CheckIndex implements Close
return result;
}
-
result.newSegments = sis.clone();
result.newSegments.clear();
+
+ // Carry over field types:
+ userData = sis.getUserData();
+ if (userData.containsKey(FieldTypes.FIELD_TYPES_KEY)) {
+ result.newSegments.getUserData().put(FieldTypes.FIELD_TYPES_KEY, userData.get(FieldTypes.FIELD_TYPES_KEY));
+ }
result.maxSegmentName = -1;
IndexReader[] segmentReaders = new IndexReader[numSegments];
@@ -744,6 +750,7 @@ public class CheckIndex implements Close
try {
int nonUniqueCount = 0;
String nonUniqueMessage = null;
+
for(String fieldName : fieldTypes.getFieldNames()) {
if (fieldTypes.getIsUnique(fieldName)) {
Terms terms = MultiFields.getTerms(topReader, fieldName);
@@ -759,7 +766,6 @@ public class CheckIndex implements Close
int docID2 = docsEnum.nextDoc();
if (docID2 != DocsEnum.NO_MORE_DOCS) {
if (nonUniqueCount == 0) {
- // nocommit should "isUnique" be in low schema?
// nocommit have -fix delete the offenders:
nonUniqueMessage = "field=\"" + fieldName + "\" is supposed to be unique, but isn't: e.g. term=" + termsEnum.term() + " matches both docID=" + docID + " and docID=" + docID2;
if (failFast) {
@@ -781,21 +787,35 @@ public class CheckIndex implements Close
if (nonUniqueCount != 0) {
nonUniqueMessage += "; total " + nonUniqueCount + " non-unique documents would be deleted";
+ result.nonUniqueCount = nonUniqueCount;
msg(infoStream, "FAILED");
msg(infoStream, nonUniqueMessage);
- throw new RuntimeException(nonUniqueMessage);
}
} finally {
topReader.close();
}
}
- if (0 == result.numBadSegments) {
+ if (0 == result.numBadSegments && 0 == result.nonUniqueCount) {
result.clean = true;
- } else
- msg(infoStream, "WARNING: " + result.numBadSegments + " broken segments (containing " + result.totLoseDocCount + " documents) detected");
+ } else {
+ StringBuilder whatsWrong = new StringBuilder();
+ if (result.numBadSegments != 0) {
+ whatsWrong.append(result.numBadSegments + " broken segments (containing " + result.totLoseDocCount + " documents)");
+ }
+ if (result.nonUniqueCount != 0) {
+ if (whatsWrong.length() != 0) {
+ whatsWrong.append(", and up to ");
+ }
+ whatsWrong.append(result.nonUniqueCount + " non-unique documents");
+ }
+
+ msg(infoStream, "WARNING: " + whatsWrong.toString());
+ }
+
+ result.validCounter = result.maxSegmentName < sis.counter;
- if ( ! (result.validCounter = (result.maxSegmentName < sis.counter))) {
+ if (result.validCounter == false) {
result.clean = false;
result.newSegments.counter = result.maxSegmentName + 1;
msg(infoStream, "ERROR: Next segment name counter " + sis.counter + " is not greater than max segment name " + result.maxSegmentName);
@@ -2285,6 +2305,59 @@ public class CheckIndex implements Close
throw new IllegalArgumentException("can only exorcise an index that was fully checked (this status checked a subset of segments)");
result.newSegments.changed();
result.newSegments.commit(result.dir);
+
+ if (result.nonUniqueCount != 0) {
+ // Open an IndexWriter to delete all non-unique documents:
+
+ // TODO: messy that we drop & then reacquire write lock; can we xfer to writer somehow? Or maybe make a filter dir w/ NoLockFactory?
+ IOUtils.close(writeLock);
+ IndexWriter w = new IndexWriter(dir, new IndexWriterConfig(null));
+ IndexReader r = null;
+ boolean success = false;
+ try {
+ int delCount = 0;
+ FieldTypes fieldTypes = w.getFieldTypes();
+ r = w.getReader();
+ for (String fieldName : fieldTypes.getFieldNames()) {
+ if (fieldTypes.getIsUnique(fieldName)) {
+ Terms terms = MultiFields.getTerms(r, fieldName);
+ if (terms != null) {
+ Bits liveDocs = MultiFields.getLiveDocs(r);
+ TermsEnum termsEnum = terms.iterator(null);
+ DocsEnum docsEnum = null;
+ while (termsEnum.next() != null) {
+ docsEnum = termsEnum.docs(liveDocs, docsEnum, DocsEnum.FLAG_NONE);
+ int docID = docsEnum.nextDoc();
+ if (docID != DocsEnum.NO_MORE_DOCS) {
+ // Delete all but the first document:
+ while ((docID = docsEnum.nextDoc()) != DocsEnum.NO_MORE_DOCS) {
+ delCount++;
+ if (w.tryDeleteDocument(r, docID) == false) {
+ throw new RuntimeException("failed to tryDeleteDocument " + docID);
+ }
+ }
+ }
+ }
+ }
+ }
+ }
+ if (delCount != result.nonUniqueCount) {
+ throw new RuntimeException("exorcise attempted to delete the wrong number (" + delCount + ") of documents vs expected " + result.nonUniqueCount);
+ }
+ success = true;
+ } finally {
+ if (success == false) {
+ IOUtils.closeWhileHandlingException(r);
+ w.rollback();
+ } else {
+ IOUtils.close(w, r);
+ }
+ // re-obtain write lock
+ if (!writeLock.obtain(IndexWriterConfig.WRITE_LOCK_TIMEOUT)) {
+ throw new LockObtainFailedException("Index locked for write: " + writeLock);
+ }
+ }
+ }
}
private static boolean assertsOn;
Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/DefaultIndexingChain.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/DefaultIndexingChain.java?rev=1649007&r1=1649006&r2=1649007&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/DefaultIndexingChain.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/DefaultIndexingChain.java Fri Jan 2 12:02:31 2015
@@ -558,7 +558,7 @@ final class DefaultIndexingChain extends
void setInvertState() {
invertState = new FieldInvertState(fieldInfo.name);
- termsHashPerField = termsHash.addField(invertState, fieldInfo, docWriter.writer.rightJustifyTerms(fieldInfo.name));
+ termsHashPerField = termsHash.addField(invertState, fieldInfo);
if (fieldInfo.omitsNorms() == false) {
assert norms == null;
// Even if no documents actually succeed in setting a norm, we still write norms for this segment:
@@ -601,7 +601,6 @@ final class DefaultIndexingChain extends
}
// only bother checking offsets if something will consume them.
- // nocommit can't we do this todo now?
// TODO: after we fix analyzers, also check if termVectorOffsets will be indexed.
final boolean checkOffsets = indexOptions == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS;
@@ -653,8 +652,8 @@ final class DefaultIndexingChain extends
if (invertState.length < 0) {
throw new IllegalArgumentException("too many tokens in field '" + field.name() + "'");
}
- //System.out.println(" term=" + invertState.termAttribute);
-
+ //System.out.println(" term=" + fieldInfo.name + ":" + invertState.termAttribute);
+
// If we hit an exception in here, we abort
// all buffered documents since the last
// flush, on the likelihood that the
@@ -681,7 +680,6 @@ final class DefaultIndexingChain extends
if (uniqueValues != null) {
BytesRef token = BytesRef.deepCopyOf(invertState.termAttribute.getBytesRef());
- // nocommit must force reopen if too many values added, account for RAM, etc.
if (uniqueValues.add(token) == false &&
(delTerm == null ||
delTerm.field().equals(field.name()) == false ||
@@ -689,9 +687,11 @@ final class DefaultIndexingChain extends
// Unique constraint violated; document will be marked deleted above:
throw new NotUniqueException(field.name(), token);
}
- if (stream.incrementToken() != false) {
+ if (stream.incrementToken()) {
+ uniqueValues.delete(token);
throw new IllegalArgumentException("field \"" + field.name() + "\": unique fields must have a single token");
}
+ break;
}
}
Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/DirectoryReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/DirectoryReader.java?rev=1649007&r1=1649006&r2=1649007&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/DirectoryReader.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/DirectoryReader.java Fri Jan 2 12:02:31 2015
@@ -64,7 +64,7 @@ public abstract class DirectoryReader ex
}
/**
- * Open a near real time IndexReader from the {@link org.apache.lucene.index.IndexWriter}.
+ * Open a near real time IndexReader from the {@link IndexWriter}.
*
* @param writer The IndexWriter to open from
* @param applyAllDeletes If true, all buffered deletes will
@@ -86,6 +86,19 @@ public abstract class DirectoryReader ex
return writer.getReader(applyAllDeletes);
}
+ /** Open a near-real-time IndexReader from {@link IndexWriter}, applying all deletes.
+ * @param writer The IndexWriter to open from
+ * @return The new IndexReader
+ * @throws CorruptIndexException if the index is corrupt
+ * @throws IOException if there is a low-level IO error
+ *
+ * @see #open(IndexWriter,boolean)
+ *
+ */
+ public static DirectoryReader open(final IndexWriter writer) throws IOException {
+ return open(writer, true);
+ }
+
/** Expert: returns an IndexReader reading the index in the given
* {@link IndexCommit}.
* @param commit the commit point to open
Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/DocumentsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/DocumentsWriter.java?rev=1649007&r1=1649006&r2=1649007&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/DocumentsWriter.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/DocumentsWriter.java Fri Jan 2 12:02:31 2015
@@ -135,7 +135,7 @@ final class DocumentsWriter implements C
flushPolicy = config.getFlushPolicy();
this.writer = writer;
this.events = new ConcurrentLinkedQueue<>();
- flushControl = new DocumentsWriterFlushControl(this, config, writer.bufferedUpdatesStream);
+ flushControl = new DocumentsWriterFlushControl(this, config, writer.bufferedUpdatesStream, writer.uniqueValuesRAM);
}
synchronized boolean deleteQueries(final Query... queries) throws IOException {
Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterFlushControl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterFlushControl.java?rev=1649007&r1=1649006&r2=1649007&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterFlushControl.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterFlushControl.java Fri Jan 2 12:02:31 2015
@@ -25,6 +25,7 @@ import java.util.List;
import java.util.Queue;
import java.util.Set;
import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
import org.apache.lucene.index.DocumentsWriterPerThreadPool.ThreadState;
import org.apache.lucene.util.Accountable;
@@ -72,8 +73,9 @@ final class DocumentsWriterFlushControl
private final LiveIndexWriterConfig config;
private final BufferedUpdatesStream bufferedUpdatesStream;
private final InfoStream infoStream;
+ final AtomicLong uniqueValuesRAM;
- DocumentsWriterFlushControl(DocumentsWriter documentsWriter, LiveIndexWriterConfig config, BufferedUpdatesStream bufferedUpdatesStream) {
+ DocumentsWriterFlushControl(DocumentsWriter documentsWriter, LiveIndexWriterConfig config, BufferedUpdatesStream bufferedUpdatesStream, AtomicLong uniqueValuesRAM) {
this.infoStream = config.getInfoStream();
this.stallControl = new DocumentsWriterStallControl();
this.perThreadPool = documentsWriter.perThreadPool;
@@ -82,6 +84,7 @@ final class DocumentsWriterFlushControl
this.hardMaxBytesPerDWPT = config.getRAMPerThreadHardLimitMB() * 1024 * 1024;
this.documentsWriter = documentsWriter;
this.bufferedUpdatesStream = bufferedUpdatesStream;
+ this.uniqueValuesRAM = uniqueValuesRAM;
}
public synchronized long activeBytes() {
Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/FlushByRamOrCountsPolicy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/FlushByRamOrCountsPolicy.java?rev=1649007&r1=1649006&r2=1649007&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/FlushByRamOrCountsPolicy.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/FlushByRamOrCountsPolicy.java Fri Jan 2 12:02:31 2015
@@ -69,7 +69,7 @@ class FlushByRamOrCountsPolicy extends F
}
}
if ((flushOnRAM() &&
- control.getDeleteBytesUsed() > (1024*1024*indexWriterConfig.getRAMBufferSizeMB()))) {
+ control.getDeleteBytesUsed() + control.uniqueValuesRAM.get() > (1024*1024*indexWriterConfig.getRAMBufferSizeMB()))) {
control.setApplyAllDeletes();
if (infoStream.isEnabled("FP")) {
infoStream.message("FP", "force apply deletes bytesUsed=" + control.getDeleteBytesUsed() + " vs ramBuffer=" + (1024*1024*indexWriterConfig.getRAMBufferSizeMB()));
@@ -86,7 +86,7 @@ class FlushByRamOrCountsPolicy extends F
control.setFlushPending(state);
} else if (flushOnRAM()) {// flush by RAM
final long limit = (long) (indexWriterConfig.getRAMBufferSizeMB() * 1024.d * 1024.d);
- final long totalRam = control.activeBytes() + control.getDeleteBytesUsed();
+ final long totalRam = control.activeBytes() + control.getDeleteBytesUsed() + control.uniqueValuesRAM.get();
if (totalRam >= limit) {
if (infoStream.isEnabled("FP")) {
infoStream.message("FP", "trigger flush: activeBytes=" + control.activeBytes() + " deleteBytes=" + control.getDeleteBytesUsed() + " vs limit=" + limit);
Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/FreqProxFields.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/FreqProxFields.java?rev=1649007&r1=1649006&r2=1649007&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/FreqProxFields.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/FreqProxFields.java Fri Jan 2 12:02:31 2015
@@ -144,7 +144,6 @@ class FreqProxFields extends Fields {
final int[] sortedTermIDs;
final FreqProxPostingsArray postingsArray;
final BytesRef scratch = new BytesRef();
- final byte[] scratchBytes;
final int numTerms;
final Bits liveDocs;
int ord;
@@ -156,11 +155,6 @@ class FreqProxFields extends Fields {
sortedTermIDs = terms.sortedTermIDs;
assert sortedTermIDs != null;
postingsArray = (FreqProxPostingsArray) terms.postingsArray;
- if (terms.rightJustifyTerms) {
- scratchBytes = new byte[terms.maxTermLength];
- } else {
- scratchBytes = null;
- }
}
public void reset() {
@@ -178,7 +172,6 @@ class FreqProxFields extends Fields {
int mid = (lo + hi) >>> 1;
int textStart = postingsArray.textStarts[sortedTermIDs[mid]];
terms.bytePool.setBytesRef(scratch, textStart);
- maybeLeftZeroPad();
int cmp = scratch.compareTo(text);
if (cmp < 0) {
lo = mid + 1;
@@ -199,30 +192,15 @@ class FreqProxFields extends Fields {
} else {
int textStart = postingsArray.textStarts[sortedTermIDs[ord]];
terms.bytePool.setBytesRef(scratch, textStart);
- maybeLeftZeroPad();
assert term().compareTo(text) > 0;
return SeekStatus.NOT_FOUND;
}
}
- private void maybeLeftZeroPad() {
- if (terms.rightJustifyTerms) {
- int prefix = terms.maxTermLength - scratch.length;
- for(int i=0;i<prefix;i++) {
- scratchBytes[i] = 0;
- }
- System.arraycopy(scratch.bytes, scratch.offset, scratchBytes, prefix, scratch.length);
- scratch.bytes = scratchBytes;
- scratch.offset = 0;
- scratch.length = terms.maxTermLength;
- }
- }
-
public void seekExact(long ord) {
this.ord = (int) ord;
int textStart = postingsArray.textStarts[sortedTermIDs[this.ord]];
terms.bytePool.setBytesRef(scratch, textStart);
- maybeLeftZeroPad();
}
@Override
@@ -233,7 +211,6 @@ class FreqProxFields extends Fields {
} else {
int textStart = postingsArray.textStarts[sortedTermIDs[ord]];
terms.bytePool.setBytesRef(scratch, textStart);
- maybeLeftZeroPad();
return scratch;
}
}
Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriter.java?rev=1649007&r1=1649006&r2=1649007&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriter.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriter.java Fri Jan 2 12:02:31 2015
@@ -121,7 +121,7 @@ final class FreqProxTermsWriter extends
}
@Override
- public TermsHashPerField addField(FieldInvertState invertState, FieldInfo fieldInfo, boolean rightJustifyTerms) {
- return new FreqProxTermsWriterPerField(invertState, this, fieldInfo, nextTermsHash.addField(invertState, fieldInfo, rightJustifyTerms), rightJustifyTerms);
+ public TermsHashPerField addField(FieldInvertState invertState, FieldInfo fieldInfo) {
+ return new FreqProxTermsWriterPerField(invertState, this, fieldInfo, nextTermsHash.addField(invertState, fieldInfo));
}
}
Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriterPerField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriterPerField.java?rev=1649007&r1=1649006&r2=1649007&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriterPerField.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriterPerField.java Fri Jan 2 12:02:31 2015
@@ -45,8 +45,8 @@ final class FreqProxTermsWriterPerField
* segment. */
boolean sawPayloads;
- public FreqProxTermsWriterPerField(FieldInvertState invertState, TermsHash termsHash, FieldInfo fieldInfo, TermsHashPerField nextPerField, boolean rightJustifyTerms) {
- super(fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0 ? 2 : 1, invertState, termsHash, nextPerField, fieldInfo, rightJustifyTerms);
+ public FreqProxTermsWriterPerField(FieldInvertState invertState, TermsHash termsHash, FieldInfo fieldInfo, TermsHashPerField nextPerField) {
+ super(fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0 ? 2 : 1, invertState, termsHash, nextPerField, fieldInfo);
IndexOptions indexOptions = fieldInfo.getIndexOptions();
assert indexOptions != IndexOptions.NONE;
hasFreq = indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS) >= 0;
Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/IndexReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/IndexReader.java?rev=1649007&r1=1649006&r2=1649007&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/IndexReader.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/IndexReader.java Fri Jan 2 12:02:31 2015
@@ -103,12 +103,7 @@ public abstract class IndexReader implem
public void onClose(IndexReader reader) throws IOException;
}
- // nocommit need getFieldTypes; how should MultiReader impl?
-
- // nocommit make abstract
- public FieldTypes getFieldTypes() {
- return null;
- }
+ public abstract FieldTypes getFieldTypes();
private final Set<ReaderClosedListener> readerClosedListeners =
Collections.synchronizedSet(new LinkedHashSet<ReaderClosedListener>());
Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java?rev=1649007&r1=1649006&r2=1649007&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java Fri Jan 2 12:02:31 2015
@@ -268,8 +268,6 @@ public class IndexWriter implements Clos
final IndexFileDeleter deleter;
final Map<String,LiveUniqueValues> uniqueValues = new HashMap<>();
- // nocommit can we change DirectoryReader API so consumers just pull our reader manager instead? it's "better" than the low-level "open
- // your own DR" API?
private final ReaderManager readerManager;
// used by forceMerge to note those needing merging
@@ -390,8 +388,6 @@ public class IndexWriter implements Clos
DirectoryReader getReader(boolean applyAllDeletes) throws IOException {
ensureOpen();
- // nocommit fixme so it's only my readerManager that's calling this... or just make this private, so the only way for users is to use ReaderManager
-
final long tStart = System.currentTimeMillis();
if (infoStream.isEnabled("IW")) {
@@ -428,7 +424,6 @@ public class IndexWriter implements Clos
// just like we do when loading segments_N
synchronized(this) {
maybeApplyDeletes(applyAllDeletes);
- // nocommit must serialize field types into index commit data here?
r = StandardDirectoryReader.open(this, segmentInfos, applyAllDeletes);
if (infoStream.isEnabled("IW")) {
infoStream.message("IW", "return reader version=" + r.getVersion() + " reader=" + r);
@@ -861,14 +856,13 @@ public class IndexWriter implements Clos
messageState();
}
- // nocommit what to do here... cannot delegate codecs
+ // nocommit what to do here... can/should we use FilterCodec?
if ((config.getCodec() instanceof Lucene50Codec) == false) {
codec = config.getCodec();
} else {
codec = fieldTypes.getCodec();
}
- // nocommit can we make this lazy-open the reader?
readerManager = new ReaderManager(this, true);
success = true;
@@ -2326,6 +2320,9 @@ public class IndexWriter implements Clos
*
* <p>This requires this index not be among those to be added.
*
+ * <p>
+ * <b>NOTE</b>: this call does not check unique fields
+ *
* @throws CorruptIndexException if the index is corrupt
* @throws IOException if there is a low-level IO error
* @throws LockObtainFailedException if we were unable to
@@ -2334,8 +2331,6 @@ public class IndexWriter implements Clos
public void addIndexes(Directory... dirs) throws IOException {
ensureOpen();
- // nocommit must test that unique_atom fields don't conflict:
-
noDupDirs(dirs);
List<Lock> locks = acquireWriteLocks(dirs);
@@ -2375,7 +2370,6 @@ public class IndexWriter implements Clos
FieldInfos fis = readFieldInfos(info);
for(FieldInfo fi : fis) {
- // nocommit how to undo this on exception?
globalFieldNumberMap.addOrGet(fi.name, fi.number, fi.getDocValuesType());
}
infos.add(copySegmentAsIs(info, newSegName, context));
@@ -2460,18 +2454,18 @@ public class IndexWriter implements Clos
* aborts all running merges, then any thread still running this method might
* hit a {@link MergePolicy.MergeAbortedException}.
*
+ * <p>
+ * <b>NOTE</b>: this call does not check unique fields
+ *
* @throws CorruptIndexException
* if the index is corrupt
* @throws IOException
* if there is a low-level IO error
*/
- // nocommit what about FieldTypes?
public void addIndexes(IndexReader... readers) throws IOException {
ensureOpen();
int numDocs = 0;
- // nocommit must test that unique_atom fields don't conflict:
-
try {
if (infoStream.isEnabled("IW")) {
infoStream.message("IW", "flush at addIndexes(IndexReader...)");
@@ -2481,7 +2475,6 @@ public class IndexWriter implements Clos
String mergedName = newSegmentName();
final List<LeafReader> mergeReaders = new ArrayList<>();
for (IndexReader indexReader : readers) {
- // nocommit how to undo this on exc?
FieldTypes ft = indexReader.getFieldTypes();
if (ft != null) {
fieldTypes.addAll(ft);
@@ -3024,8 +3017,6 @@ public class IndexWriter implements Clos
}
}
- // nocommit we can fix IDVPF since it will only see given ID once now?
-
final synchronized void applyAllDeletesAndUpdates() throws IOException {
flushDeletesCount.incrementAndGet();
final BufferedUpdatesStream.ApplyDeletesResult result;
@@ -3890,7 +3881,6 @@ public class IndexWriter implements Clos
// fix the reader's live docs and del count
assert delCount > reader.numDeletedDocs(); // beware of zombies
- // nocommit we are passing our fieldTypes here:
SegmentReader newReader = new SegmentReader(fieldTypes, info, reader, liveDocs, info.info.getDocCount() - delCount);
boolean released = false;
try {
@@ -4621,25 +4611,23 @@ public class IndexWriter implements Clos
}
}
- // nocommit cutover tests to this, remove DirectoryReader(writer) API, remove ReaderManager(writer) ctor
-
- // nocommit must close this in close? why are tests not failing...
-
- /** Returns a {@link ReferenceManager} to get near-real-time readers. */
- public ReferenceManager<DirectoryReader> getReaderManager() {
+ // TODO: cutover to this, remove DirecotryReader/ReaderManager APIs taking writer directly?
+ /** Returns a {@link ReferenceManager} to get NRT readers. */
+ ReferenceManager<DirectoryReader> getReaderManager() {
return readerManager;
}
- // nocommit we could expose this to apps too? e.g. to check if a given id exists in the index
+ // TODO: we could expose this to apps too? e.g. to check if a given id exists in the index
- // nocommit explore other optos once we know field is unique
+ // TODO: what other optos can we do when we know field is unique?
+ final AtomicLong uniqueValuesRAM = new AtomicLong();
synchronized LiveUniqueValues getUniqueValues(String fieldName) {
LiveUniqueValues v;
if (fieldTypes.getIsUnique(fieldName)) {
v = uniqueValues.get(fieldName);
if (v == null) {
- v = new LiveUniqueValues(fieldName, readerManager);
+ v = new LiveUniqueValues(fieldName, readerManager, uniqueValuesRAM);
uniqueValues.put(fieldName, v);
}
} else {
Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/LiveUniqueValues.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/LiveUniqueValues.java?rev=1649007&r1=1649006&r2=1649007&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/LiveUniqueValues.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/LiveUniqueValues.java Fri Jan 2 12:02:31 2015
@@ -26,6 +26,7 @@ import java.util.List;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.atomic.AtomicLong;
import org.apache.lucene.index.DocsAndPositionsEnum;
import org.apache.lucene.index.DocsEnum;
@@ -41,14 +42,21 @@ import org.apache.lucene.util.Accountabl
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.CloseableThreadLocal;
+import org.apache.lucene.util.RamUsageEstimator;
-// nocommit javadocs
-
-// nocommit better name
-
-// TODO: should this class handle deletions better...?
-final class LiveUniqueValues implements ReferenceManager.RefreshListener, Closeable, Accountable {
+/** Tracks the already indexed values in a unique field ({@link FieldTypes.#getIsUnique}. */
+final class LiveUniqueValues implements ReferenceManager.RefreshListener, Closeable {
+ private final static long BYTES_PER_ENTRY =
+ // HashMap array @ 50% load
+ 2 * RamUsageEstimator.NUM_BYTES_OBJECT_REF +
+ // HashMap Entry
+ 3 * RamUsageEstimator.NUM_BYTES_OBJECT_REF + RamUsageEstimator.NUM_BYTES_INT+
+ // BytesRef
+ RamUsageEstimator.NUM_BYTES_OBJECT_HEADER + 2*RamUsageEstimator.NUM_BYTES_INT + RamUsageEstimator.NUM_BYTES_OBJECT_REF +
+ // byte[] (we count actual bytes based on length of id that's added)
+ RamUsageEstimator.NUM_BYTES_ARRAY_HEADER;
+
// Holds reused TermsEnum/DocsEnum state for faster lookups:
private final ConcurrentMap<IndexReader,CloseableThreadLocal<PerThreadLookup>> lookupStates = new ConcurrentHashMap<>();
@@ -68,15 +76,19 @@ final class LiveUniqueValues implements
private volatile Map<BytesRef,Boolean> current = newMap();
private final ReaderManager mgr;
private final String uidField;
+ private volatile long oldRAMBytesUsed;
+ private volatile long currentRAMBytesUsed;
+ private final AtomicLong ramBytesUsed;
private static Map<BytesRef,Boolean> newMap() {
return new HashMap<BytesRef,Boolean>();
}
/** Sole constructor. */
- public LiveUniqueValues(String uidField, ReaderManager mgr) {
+ public LiveUniqueValues(String uidField, ReaderManager mgr, AtomicLong ramBytesUsed) {
this.uidField = uidField;
this.mgr = mgr;
+ this.ramBytesUsed = ramBytesUsed;
mgr.addListener(this);
}
@@ -93,6 +105,8 @@ final class LiveUniqueValues implements
// try this new map, then fallback to old, then to the
// current searcher:
current = newMap();
+ oldRAMBytesUsed = currentRAMBytesUsed;
+ currentRAMBytesUsed = 0;
}
@Override
@@ -104,6 +118,8 @@ final class LiveUniqueValues implements
// actually already included in the previously opened
// reader. So we can safely clear old here:
old = newMap();
+ ramBytesUsed.addAndGet(-oldRAMBytesUsed);
+ oldRAMBytesUsed = 0;
}
/** Call this to try adding a value; this returns false if the add
@@ -123,6 +139,7 @@ final class LiveUniqueValues implements
if (v != null) {
if (v == Boolean.FALSE) {
current.put(id, Boolean.TRUE);
+ addRAM(BYTES_PER_ENTRY + id.length);
return true;
} else {
return false;
@@ -135,6 +152,7 @@ final class LiveUniqueValues implements
return false;
} else {
current.put(id, Boolean.TRUE);
+ addRAM(BYTES_PER_ENTRY + id.length);
return true;
}
} finally {
@@ -145,7 +163,15 @@ final class LiveUniqueValues implements
/** Call this after you've successfully deleted a document
* from the index. */
public synchronized void delete(BytesRef id) {
- current.put(id, Boolean.FALSE);
+ Boolean old = current.put(id, Boolean.FALSE);
+ if (old == null) {
+ addRAM(BYTES_PER_ENTRY + id.length);
+ }
+ }
+
+ private synchronized void addRAM(long bytes) {
+ currentRAMBytesUsed += bytes;
+ ramBytesUsed.addAndGet(bytes);
}
/** Returns the [approximate] number of id/value pairs
@@ -180,15 +206,6 @@ final class LiveUniqueValues implements
return lookupState;
}
- public long ramBytesUsed() {
- // nocommit todo
- return 0;
- }
-
- public Iterable<? extends Accountable> getChildResources() {
- return Collections.emptyList();
- }
-
// TODO: optimize this so that on toplevel reader reopen, we reuse TermsEnum for shared segments:
private final static class PerThreadLookup {
@@ -235,8 +252,8 @@ final class LiveUniqueValues implements
public boolean exists(BytesRef id) throws IOException {
for(int seg=0;seg<numSegs;seg++) {
if (termsEnums[seg].seekExact(id)) {
- // nocommit once we remove deleted postings on flush we don't need the live docs:
- DocsEnum docs = docsEnums[seg] = termsEnums[seg].docs(liveDocs[seg], docsEnums[seg], 0);
+ // NOTE: we don't need to pass live docs because IW now removes them on flush:
+ DocsEnum docs = docsEnums[seg] = termsEnums[seg].docs(null, docsEnums[seg], 0);
int docID = docs.nextDoc();
if (docID != DocsEnum.NO_MORE_DOCS) {
assert docs.nextDoc() == DocsEnum.NO_MORE_DOCS;
Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/MergeState.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/MergeState.java?rev=1649007&r1=1649006&r2=1649007&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/MergeState.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/MergeState.java Fri Jan 2 12:02:31 2015
@@ -102,12 +102,11 @@ public class MergeState {
docValuesProducers = new DocValuesProducer[numReaders];
fieldInfos = new FieldInfos[numReaders];
liveDocs = new Bits[numReaders];
-
for(int i=0;i<numReaders;i++) {
final LeafReader reader = readers.get(i);
// nocommit segment merger should do this?
FieldTypes readerFieldTypes = reader.getFieldTypes();
- if (readerFieldTypes != null) {
+ if (readerFieldTypes != null && fieldTypes != readerFieldTypes) {
fieldTypes.addAll(readerFieldTypes);
}
Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/MultiDocValues.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/MultiDocValues.java?rev=1649007&r1=1649006&r2=1649007&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/MultiDocValues.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/MultiDocValues.java Fri Jan 2 12:02:31 2015
@@ -521,7 +521,7 @@ public class MultiDocValues {
slices[i] = new ReaderSlice(0, 0, i);
indexes[i] = new TermsEnumIndex(subs[segmentMap.newToOld(i)], i);
}
- MultiTermsEnum mte = new MultiTermsEnum(slices, -1);
+ MultiTermsEnum mte = new MultiTermsEnum(slices);
mte.reset(indexes);
long globalOrd = 0;
while (mte.next() != null) {
Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/MultiFields.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/MultiFields.java?rev=1649007&r1=1649006&r2=1649007&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/MultiFields.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/MultiFields.java Fri Jan 2 12:02:31 2015
@@ -27,7 +27,6 @@ import java.util.List;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
-import org.apache.lucene.document.FieldTypes;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.MergedIterator;
@@ -186,12 +185,6 @@ public final class MultiFields extends F
this.subSlices = subSlices;
}
- private FieldTypes fieldTypes;
-
- public void setFieldTypes(FieldTypes fieldTypes) {
- this.fieldTypes = fieldTypes;
- }
-
@SuppressWarnings({"unchecked","rawtypes"})
@Override
public Iterator<String> iterator() {
@@ -227,7 +220,7 @@ public final class MultiFields extends F
// is unbounded.
} else {
result = new MultiTerms(subs2.toArray(Terms.EMPTY_ARRAY),
- slices2.toArray(ReaderSlice.EMPTY_ARRAY), fieldTypes != null && fieldTypes.rightJustifyTerms(field));
+ slices2.toArray(ReaderSlice.EMPTY_ARRAY));
terms.put(field, result);
}
Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/MultiTerms.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/MultiTerms.java?rev=1649007&r1=1649006&r2=1649007&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/MultiTerms.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/MultiTerms.java Fri Jan 2 12:02:31 2015
@@ -39,9 +39,7 @@ public final class MultiTerms extends Te
private final boolean hasOffsets;
private final boolean hasPositions;
private final boolean hasPayloads;
- private final boolean rightJustifyTerms;
private final int[] termLengths;
- private final int maxTermLength;
/** Sole constructor.
*
@@ -49,10 +47,9 @@ public final class MultiTerms extends Te
* @param subSlices A parallel array (matching {@code
* subs}) describing the sub-reader slices.
*/
- public MultiTerms(Terms[] subs, ReaderSlice[] subSlices, boolean rightJustifyTerms) throws IOException {
+ public MultiTerms(Terms[] subs, ReaderSlice[] subSlices) throws IOException {
this.subs = subs;
this.subSlices = subSlices;
- this.rightJustifyTerms = rightJustifyTerms;
assert subs.length > 0 : "inefficient: don't use MultiTerms over one sub";
boolean _hasFreqs = true;
@@ -60,7 +57,6 @@ public final class MultiTerms extends Te
boolean _hasPositions = true;
boolean _hasPayloads = false;
termLengths = new int[subs.length];
- int maxTermLength = Integer.MIN_VALUE;
for(int i=0;i<subs.length;i++) {
_hasFreqs &= subs[i].hasFreqs();
_hasOffsets &= subs[i].hasOffsets();
@@ -68,11 +64,8 @@ public final class MultiTerms extends Te
_hasPayloads |= subs[i].hasPayloads();
BytesRef minTerm = subs[i].getMin();
termLengths[i] = minTerm == null ? -1 : minTerm.length;
- assert rightJustifyTerms == false || (termLengths[i] != -1 && termLengths[i] == subs[i].getMax().length);
- maxTermLength = Math.max(maxTermLength, termLengths[i]);
}
- this.maxTermLength = rightJustifyTerms ? maxTermLength : -1;
hasFreqs = _hasFreqs;
hasOffsets = _hasOffsets;
hasPositions = _hasPositions;
@@ -91,7 +84,6 @@ public final class MultiTerms extends Te
@Override
public TermsEnum intersect(CompiledAutomaton compiled, BytesRef startTerm) throws IOException {
- // nocommit doesn't work w/ rightJustifyTerms?
final List<MultiTermsEnum.TermsEnumIndex> termsEnums = new ArrayList<>();
for(int i=0;i<subs.length;i++) {
final TermsEnum termsEnum = subs[i].intersect(compiled, startTerm);
@@ -101,7 +93,7 @@ public final class MultiTerms extends Te
}
if (termsEnums.size() > 0) {
- return new MultiTermsEnum(subSlices, maxTermLength).reset(termsEnums.toArray(MultiTermsEnum.TermsEnumIndex.EMPTY_ARRAY));
+ return new MultiTermsEnum(subSlices).reset(termsEnums.toArray(MultiTermsEnum.TermsEnumIndex.EMPTY_ARRAY));
} else {
return TermsEnum.EMPTY;
}
@@ -145,7 +137,7 @@ public final class MultiTerms extends Te
}
if (termsEnums.size() > 0) {
- return new MultiTermsEnum(subSlices, maxTermLength).reset(termsEnums.toArray(MultiTermsEnum.TermsEnumIndex.EMPTY_ARRAY));
+ return new MultiTermsEnum(subSlices).reset(termsEnums.toArray(MultiTermsEnum.TermsEnumIndex.EMPTY_ARRAY));
} else {
return TermsEnum.EMPTY;
}
Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/MultiTermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/MultiTermsEnum.java?rev=1649007&r1=1649006&r2=1649007&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/MultiTermsEnum.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/MultiTermsEnum.java Fri Jan 2 12:02:31 2015
@@ -40,8 +40,6 @@ public final class MultiTermsEnum extend
private final TermsEnumWithSlice[] top;
private final MultiDocsEnum.EnumWithSlice[] subDocs;
private final MultiDocsAndPositionsEnum.EnumWithSlice[] subDocsAndPositions;
- private final int zeroPadTermLength;
- private final Comparator<BytesRef> cmp;
private BytesRef lastSeek;
private boolean lastSeekExact;
@@ -50,7 +48,6 @@ public final class MultiTermsEnum extend
private int numTop;
private int numSubs;
private BytesRef current;
- private BytesRef scratch;
static class TermsEnumIndex {
public final static TermsEnumIndex[] EMPTY_ARRAY = new TermsEnumIndex[0];
@@ -77,19 +74,12 @@ public final class MultiTermsEnum extend
/** Sole constructor.
* @param slices Which sub-reader slices we should
* merge. */
- public MultiTermsEnum(ReaderSlice[] slices, int zeroPadTermLength) {
+ public MultiTermsEnum(ReaderSlice[] slices) {
queue = new TermMergeQueue(slices.length);
top = new TermsEnumWithSlice[slices.length];
subs = new TermsEnumWithSlice[slices.length];
subDocs = new MultiDocsEnum.EnumWithSlice[slices.length];
subDocsAndPositions = new MultiDocsAndPositionsEnum.EnumWithSlice[slices.length];
- this.zeroPadTermLength = zeroPadTermLength;
- if (zeroPadTermLength != -1) {
- scratch = new BytesRef(zeroPadTermLength);
- scratch.length = zeroPadTermLength;
- } else {
- scratch = null;
- }
for(int i=0;i<slices.length;i++) {
subs[i] = new TermsEnumWithSlice(i, slices[i]);
subDocs[i] = new MultiDocsEnum.EnumWithSlice();
@@ -98,11 +88,6 @@ public final class MultiTermsEnum extend
subDocsAndPositions[i].slice = slices[i];
}
currentSubs = new TermsEnumWithSlice[slices.length];
- if (zeroPadTermLength == -1) {
- cmp = BytesRef.getUTF8SortedAsUnicodeComparator();
- } else {
- cmp = BytesRef.getRightJustifiedComparator();
- }
}
@Override
@@ -146,7 +131,7 @@ public final class MultiTermsEnum extend
numTop = 0;
boolean seekOpt = false;
- if (lastSeek != null && cmp.compare(lastSeek, term) <= 0) {
+ if (lastSeek != null && lastSeek.compareTo(term) <= 0) {
seekOpt = true;
}
@@ -164,7 +149,7 @@ public final class MultiTermsEnum extend
if (seekOpt) {
final BytesRef curTerm = currentSubs[i].current;
if (curTerm != null) {
- final int x = cmp.compare(term, curTerm);
+ final int x = term.compareTo(curTerm);
if (x == 0) {
status = true;
} else if (x < 0) {
@@ -182,7 +167,6 @@ public final class MultiTermsEnum extend
if (status) {
top[numTop++] = currentSubs[i];
current = currentSubs[i].current = currentSubs[i].terms.term();
- maybeLeftZeroPad();
assert term.equals(currentSubs[i].current);
}
}
@@ -192,18 +176,6 @@ public final class MultiTermsEnum extend
return numTop > 0;
}
- private void maybeLeftZeroPad() {
- if (zeroPadTermLength != -1 && current != null) {
- int prefix = zeroPadTermLength - current.length;
- assert prefix >= 0: "prefix=" + prefix + " zeroPadTermLength=" + zeroPadTermLength + " vs " + current.length;
- for(int i=0;i<prefix;i++) {
- scratch.bytes[i] = 0;
- }
- System.arraycopy(current.bytes, current.offset, scratch.bytes, prefix, current.length);
- current = scratch;
- }
- }
-
@Override
public SeekStatus seekCeil(BytesRef term) throws IOException {
queue.clear();
@@ -211,7 +183,7 @@ public final class MultiTermsEnum extend
lastSeekExact = false;
boolean seekOpt = false;
- if (lastSeek != null && cmp.compare(lastSeek, term) <= 0) {
+ if (lastSeek != null && lastSeek.compareTo(term) <= 0) {
seekOpt = true;
}
@@ -229,7 +201,7 @@ public final class MultiTermsEnum extend
if (seekOpt) {
final BytesRef curTerm = currentSubs[i].current;
if (curTerm != null) {
- final int x = cmp.compare(term, curTerm);
+ final int x = term.compareTo(curTerm);
if (x == 0) {
status = SeekStatus.FOUND;
} else if (x < 0) {
@@ -247,7 +219,6 @@ public final class MultiTermsEnum extend
if (status == SeekStatus.FOUND) {
top[numTop++] = currentSubs[i];
current = currentSubs[i].current = currentSubs[i].terms.term();
- maybeLeftZeroPad();
} else {
if (status == SeekStatus.NOT_FOUND) {
currentSubs[i].current = currentSubs[i].terms.term();
@@ -295,7 +266,6 @@ public final class MultiTermsEnum extend
}
}
current = top[0].current;
- maybeLeftZeroPad();
}
private void pushTop() throws IOException {
@@ -536,7 +506,7 @@ public final class MultiTermsEnum extend
@Override
protected boolean lessThan(TermsEnumWithSlice termsA, TermsEnumWithSlice termsB) {
- final int x = cmp.compare(termsA.current, termsB.current);
+ final int x = termsA.current.compareTo(termsB.current);
if (x != 0) {
return x < 0;
} else {
Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/ParallelCompositeReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/ParallelCompositeReader.java?rev=1649007&r1=1649006&r2=1649007&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/ParallelCompositeReader.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/ParallelCompositeReader.java Fri Jan 2 12:02:31 2015
@@ -80,8 +80,16 @@ public class ParallelCompositeReader ext
reader.incRef();
}
}
- // nocommit must check that they are congruent:
- fieldTypes = readers.length > 0 ? readers[0].getFieldTypes() : null;
+ if (readers.length > 0) {
+ // Merge all field types; this will throw exc if any field types are not congurent:
+ fieldTypes = new FieldTypes(readers[0].getFieldTypes());
+ for (final IndexReader reader : completeReaderSet) {
+ // nocommit we should only add the fields that this reader "wins" on?
+ fieldTypes.addAll(reader.getFieldTypes());
+ }
+ } else {
+ fieldTypes = null;
+ }
// finally add our own synthetic readers, so we close or decRef them, too (it does not matter what we do)
completeReaderSet.addAll(getSequentialSubReaders());
Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/ParallelLeafReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/ParallelLeafReader.java?rev=1649007&r1=1649006&r2=1649007&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/ParallelLeafReader.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/ParallelLeafReader.java Fri Jan 2 12:02:31 2015
@@ -18,6 +18,7 @@ package org.apache.lucene.index;
*/
import java.io.IOException;
+import java.util.Arrays;
import java.util.Collections;
import java.util.IdentityHashMap;
import java.util.Iterator;
@@ -86,8 +87,7 @@ public class ParallelLeafReader extends
this.maxDoc = first.maxDoc();
this.numDocs = first.numDocs();
this.hasDeletions = first.hasDeletions();
- // nocommit must verify field types are congruent and take union?
- this.fieldTypes = first.getFieldTypes();
+ this.fieldTypes = new FieldTypes(first.getFieldTypes());
} else {
this.maxDoc = this.numDocs = 0;
this.hasDeletions = false;
@@ -108,6 +108,8 @@ public class ParallelLeafReader extends
// build FieldInfos and fieldToReader map:
for (final LeafReader reader : this.parallelReaders) {
final FieldInfos readerFieldInfos = reader.getFieldInfos();
+ // nocommit we should only add the fields that this reader "wins" on?
+ fieldTypes.addAll(reader.getFieldTypes());
for (FieldInfo fieldInfo : readerFieldInfos) {
// NOTE: first reader having a given field "wins":
if (!fieldToReader.containsKey(fieldInfo.name)) {
Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/ReadersAndUpdates.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/ReadersAndUpdates.java?rev=1649007&r1=1649006&r2=1649007&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/ReadersAndUpdates.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/ReadersAndUpdates.java Fri Jan 2 12:02:31 2015
@@ -20,8 +20,8 @@ package org.apache.lucene.index;
import java.io.IOException;
import java.util.HashMap;
import java.util.Iterator;
-import java.util.Map;
import java.util.Map.Entry;
+import java.util.Map;
import java.util.NoSuchElementException;
import java.util.Set;
import java.util.concurrent.atomic.AtomicInteger;
@@ -31,6 +31,7 @@ import org.apache.lucene.codecs.DocValue
import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.codecs.FieldInfosFormat;
import org.apache.lucene.codecs.LiveDocsFormat;
+import org.apache.lucene.document.FieldTypes;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.FlushInfo;
import org.apache.lucene.store.IOContext;
@@ -129,8 +130,7 @@ class ReadersAndUpdates {
public SegmentReader getReader(IOContext context) throws IOException {
if (reader == null) {
// We steal returned ref:
- // nocommit clone the field types?
- reader = new SegmentReader(writer.getFieldTypes(), info, context);
+ reader = new SegmentReader(writer.fieldTypes, info, context);
if (liveDocs == null) {
liveDocs = reader.getLiveDocs();
}
@@ -188,7 +188,7 @@ class ReadersAndUpdates {
// force new liveDocs in initWritableLiveDocs even if it's null
liveDocsShared = true;
if (liveDocs != null) {
- return new SegmentReader(writer.fieldTypes, reader.getSegmentInfo(), reader, liveDocs, info.info.getDocCount() - info.getDelCount() - pendingDeleteCount);
+ return new SegmentReader(new FieldTypes(writer.fieldTypes), reader.getSegmentInfo(), reader, liveDocs, info.info.getDocCount() - info.getDelCount() - pendingDeleteCount);
} else {
// liveDocs == null and reader != null. That can only be if there are no deletes
assert reader.getLiveDocs() == null;
@@ -476,7 +476,7 @@ class ReadersAndUpdates {
// reader could be null e.g. for a just merged segment (from
// IndexWriter.commitMergedDeletes).
- final SegmentReader reader = this.reader == null ? new SegmentReader(writer.getFieldTypes(), info, IOContext.READONCE) : this.reader;
+ final SegmentReader reader = this.reader == null ? new SegmentReader(writer.fieldTypes, info, IOContext.READONCE) : this.reader;
try {
// clone FieldInfos so that we can update their dvGen separately from
// the reader's infos and write them to a new fieldInfos_gen file