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 2014/11/08 12:05:06 UTC
svn commit: r1637540 [1/2] - in /lucene/dev/branches/lucene6005/lucene:
core/src/java/org/apache/lucene/document/
core/src/java/org/apache/lucene/index/
core/src/java/org/apache/lucene/search/
core/src/test/org/apache/lucene/codecs/lucene50/ core/src/t...
Author: mikemccand
Date: Sat Nov 8 11:05:05 2014
New Revision: 1637540
URL: http://svn.apache.org/r1637540
Log:
LUCENE-6005: add UNIQUE_ATOM type (for primary key fields), which IW and CheckIndex enforce; add IW.getReaderManager(); add exists filter support (enabled by default); cutover some more tests / fix nocommits
Added:
lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/LiveUniqueValues.java (with props)
lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/NotUniqueException.java (with props)
lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestUniqueAtom.java (with props)
Modified:
lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/document/Document2.java
lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/document/FieldTypes.java
lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/DefaultIndexingChain.java
lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/DocConsumer.java
lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/DocumentsWriter.java
lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java
lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/FieldInfo.java
lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/FieldInfos.java
lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/FreqProxFields.java
lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriter.java
lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/ReadersAndUpdates.java
lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java
lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/TermRangeQuery.java
lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/codecs/lucene50/TestLucene50DocValuesFormat.java
lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/document/TestDocument2.java
lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/Test2BPostingsBytes.java
lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/Test2BTerms.java
lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestBagOfPositions.java
lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestBinaryDocValuesUpdates.java
lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestCodecs.java
lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestConcurrentMergeScheduler.java
lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestDirectoryReader.java
lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestDocValuesIndexing.java
lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterDelete.java
lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterExceptions.java
lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterForceMerge.java
lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterNRTIsCurrent.java
lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestLongPostings.java
lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestMultiFields.java
lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestNumericDocValuesUpdates.java
lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestRollingUpdates.java
lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestSegmentMerger.java
lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestSegmentReader.java
lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/search/TestTermRangeQuery.java
lucene/dev/branches/lucene6005/lucene/join/src/java/org/apache/lucene/search/join/ToParentBlockJoinQuery.java
lucene/dev/branches/lucene6005/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java
lucene/dev/branches/lucene6005/lucene/misc/src/test/org/apache/lucene/uninverting/TestFieldCacheVsDocValues.java
lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/index/RandomIndexWriter.java
Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/document/Document2.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/document/Document2.java?rev=1637540&r1=1637539&r2=1637540&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/document/Document2.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/document/Document2.java Sat Nov 8 11:05:05 2014
@@ -24,6 +24,7 @@ import java.util.ArrayList;
import java.util.Date;
import java.util.Iterator;
import java.util.List;
+import java.util.NoSuchElementException;
import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.analysis.TokenStream;
@@ -118,6 +119,11 @@ public class Document2 implements Iterab
sts.setValue(value);
return sts;
}
+
+ @Override
+ public String toString() {
+ return fieldName + ": " + value;
+ }
@Override
public TokenStream tokenStream(Analyzer analyzerIn, TokenStream reuse) throws IOException {
@@ -143,6 +149,7 @@ public class Document2 implements Iterab
case DATE:
return getReusedBinaryTokenStream(longToBytes(((Date) value).getTime()), reuse);
case ATOM:
+ case UNIQUE_ATOM:
if (fieldType.minTokenLength != null) {
if (value instanceof String) {
String s = (String) value;
@@ -359,11 +366,34 @@ public class Document2 implements Iterab
@Override
public Iterator<IndexableField> iterator() {
- // nocommit how to fix generics here so I can just return fields.iterator?
- //return fields.iterator();
- ArrayList<IndexableField> l = new ArrayList<>();
- l.addAll(fields);
- return l.iterator();
+ if (fieldTypes != null) {
+ assert fieldTypes.getStored(FieldTypes.FIELD_NAMES_FIELD) == false;
+ }
+
+ return new Iterator<IndexableField>() {
+ int index;
+ int fieldNamesIndex;
+
+ public boolean hasNext() {
+ return index < fields.size() || (changeSchema && fieldTypes != null && fieldTypes.enableExistsFilters && fieldNamesIndex < fields.size());
+ }
+
+ public void remove() {
+ throw new UnsupportedOperationException();
+ }
+
+ public IndexableField next() {
+ if (index < fields.size()) {
+ return fields.get(index++);
+ } else if (fieldTypes != null && changeSchema && fieldTypes.enableExistsFilters && fieldNamesIndex < fields.size()) {
+ // nocommit make a more efficient version? e.g. a single field that takes a list and iterates each via TokenStream. maybe we
+ // should addAtom(String...)?
+ return new FieldValue(FieldTypes.FIELD_NAMES_FIELD, fields.get(fieldNamesIndex++).fieldName);
+ } else {
+ throw new NoSuchElementException();
+ }
+ }
+ };
}
public List<FieldValue> getFieldValues() {
@@ -404,7 +434,11 @@ public class Document2 implements Iterab
fields.add(new FieldValue(fieldName, value));
}
- /** E.g. an "id" (primary key) field. Default: indexes this value as a single token, and disables norms and freqs. */
+ /** E.g. a binary single-token field. */
+ public void addAtom(String fieldName, byte[] value) {
+ addAtom(fieldName, new BytesRef(value));
+ }
+
public void addAtom(String fieldName, BytesRef value) {
if (changeSchema) {
fieldTypes.recordValueType(fieldName, FieldTypes.ValueType.ATOM);
@@ -412,6 +446,27 @@ public class Document2 implements Iterab
fields.add(new FieldValue(fieldName, value));
}
+ /** E.g. a primary key field. */
+ public void addUniqueAtom(String fieldName, String value) {
+ if (changeSchema) {
+ fieldTypes.recordValueType(fieldName, FieldTypes.ValueType.UNIQUE_ATOM);
+ }
+ fields.add(new FieldValue(fieldName, value));
+ }
+
+ /** E.g. a primary key field. */
+ public void addUniqueAtom(String fieldName, byte[] value) {
+ addUniqueAtom(fieldName, new BytesRef(value));
+ }
+
+ /** E.g. a primary key field. */
+ public void addUniqueAtom(String fieldName, BytesRef value) {
+ if (changeSchema) {
+ fieldTypes.recordValueType(fieldName, FieldTypes.ValueType.UNIQUE_ATOM);
+ }
+ fields.add(new FieldValue(fieldName, value));
+ }
+
/** E.g. a "title" field. Default: indexes this value as multiple tokens from analyzer, and disables norms and freqs, and also enables
* sorting (indexes sorted doc values). */
public void addShortText(String fieldName, String value) {
@@ -545,7 +600,7 @@ public class Document2 implements Iterab
static {
// nocommit is there a cleaner/general way to detect missing enum value in case switch statically? must we use ecj?
- assert FieldTypes.ValueType.values().length == 12: "missing case for switch statement below";
+ assert FieldTypes.ValueType.values().length == 13: "missing case for switch statement below";
}
/** Note: this FieldTypes must already know about all the fields in the incoming doc. */
@@ -570,6 +625,13 @@ public class Document2 implements Iterab
addAtom(fieldName, (String) field.value);
}
break;
+ case UNIQUE_ATOM:
+ if (field.value instanceof BytesRef) {
+ addUniqueAtom(fieldName, (BytesRef) field.value);
+ } else {
+ addUniqueAtom(fieldName, (String) field.value);
+ }
+ break;
case INT:
addInt(fieldName, field.numericValue().intValue());
break;
@@ -640,7 +702,7 @@ public class Document2 implements Iterab
}
public Boolean getBoolean(String fieldName) {
- // nocommit can we assert this is a known field and that its type is boolean...?
+ // nocommit can we assert this is a known field and that its type is correct?
for(FieldValue fieldValue : fields) {
if (fieldValue.fieldName.equals(fieldName)) {
return (Boolean) fieldValue.value;
@@ -651,7 +713,7 @@ public class Document2 implements Iterab
}
public Date getDate(String fieldName) {
- // nocommit can we assert this is a known field and that its type is date...?
+ // nocommit can we assert this is a known field and that its type is correct?
for(FieldValue fieldValue : fields) {
if (fieldValue.fieldName.equals(fieldName)) {
return (Date) fieldValue.value;
@@ -662,7 +724,7 @@ public class Document2 implements Iterab
}
public InetAddress getInetAddress(String fieldName) {
- // nocommit can we assert this is a known field and that its type is inet address...?
+ // nocommit can we assert this is a known field and that its type is correct?
for(FieldValue fieldValue : fields) {
if (fieldValue.fieldName.equals(fieldName)) {
return (InetAddress) fieldValue.value;
@@ -673,7 +735,7 @@ public class Document2 implements Iterab
}
public String getString(String fieldName) {
- // nocommit can we assert this is a known field and that its type is text/short_text...?
+ // nocommit can we assert this is a known field and that its type is correct?
for(FieldValue fieldValue : fields) {
if (fieldValue.fieldName.equals(fieldName)) {
return fieldValue.value.toString();
@@ -684,7 +746,7 @@ public class Document2 implements Iterab
}
public String[] getStrings(String fieldName) {
- // nocommit can we assert this is a known field and that its type is text/short_text...?
+ // nocommit can we assert this is a known field and that its type is correct?
List<String> values = new ArrayList<>();
for(FieldValue fieldValue : fields) {
if (fieldValue.fieldName.equals(fieldName)) {
@@ -696,7 +758,7 @@ public class Document2 implements Iterab
}
public BytesRef getBinary(String fieldName) {
- // nocommit can we assert this is a known field and that its type is text/short_text...?
+ // nocommit can we assert this is a known field and that its type is correct?
for(FieldValue fieldValue : fields) {
if (fieldValue.fieldName.equals(fieldName)) {
return (BytesRef) fieldValue.value;
@@ -707,7 +769,7 @@ public class Document2 implements Iterab
}
public Integer getInt(String fieldName) {
- // nocommit can we assert this is a known field and that its type is text/short_text...?
+ // nocommit can we assert this is a known field and that its type is correct?
for(FieldValue fieldValue : fields) {
if (fieldValue.fieldName.equals(fieldName)) {
return (Integer) fieldValue.value;
Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/document/FieldTypes.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/document/FieldTypes.java?rev=1637540&r1=1637539&r2=1637540&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/document/FieldTypes.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/document/FieldTypes.java Sat Nov 8 11:05:05 2014
@@ -46,13 +46,16 @@ import org.apache.lucene.index.IndexWrit
import org.apache.lucene.index.IndexableFieldType;
import org.apache.lucene.index.SegmentInfos;
import org.apache.lucene.index.Term;
+import org.apache.lucene.search.Filter;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.Query;
+import org.apache.lucene.search.QueryWrapperFilter;
import org.apache.lucene.search.Sort;
import org.apache.lucene.search.SortField;
import org.apache.lucene.search.SortedNumericSortField;
import org.apache.lucene.search.SortedSetSortField;
import org.apache.lucene.search.TermQuery;
+import org.apache.lucene.search.TermRangeFilter;
import org.apache.lucene.search.TermRangeQuery;
import org.apache.lucene.search.similarities.PerFieldSimilarityWrapper;
import org.apache.lucene.search.similarities.Similarity;
@@ -99,6 +102,18 @@ import org.apache.lucene.util.Version;
// PerFieldAnalyzerWrapper
// oal.document
+// nocommit byte, short?
+
+// nocommit allow adding array of atom values? fieldnamesfield would use it?
+
+// nocommit optimize field exists filter to MatchAllBits when all docs in the seg have the field; same opto as range query when min < terms.min & max > terms.max
+
+// nocommit use better pf when field is unique
+
+// nocommit filter caching? parent docs filter?
+
+// nocommit do we allow mixing of binary and non-binary atom?
+
// nocommit index field names the doc has?
// nocommit fix simple qp to optionally take this?
@@ -182,14 +197,12 @@ import org.apache.lucene.util.Version;
public class FieldTypes {
- /** Key used to store the field types inside {@link IndexWriter#setCommitData}. */
- public static final String FIELD_PROPERTIES_KEY = "field_properties";
-
enum ValueType {
NONE,
TEXT,
SHORT_TEXT,
ATOM, // nocommit binary sort of overlaps w/ this?
+ UNIQUE_ATOM, // nocommit binary sort of overlaps w/ this?
INT,
FLOAT,
LONG,
@@ -203,6 +216,12 @@ public class FieldTypes {
private final boolean readOnly;
+ public static final String FIELD_NAMES_FIELD = "$fieldnames";
+
+ /** So exists filters are fast */
+ boolean enableExistsFilters = true;
+ private boolean indexedDocs;
+
private final Version indexCreatedVersion;
final Map<String,FieldType> fields = new HashMap<>();
@@ -253,6 +272,8 @@ public class FieldTypes {
volatile Integer analyzerPositionGap;
volatile Integer analyzerOffsetGap;
+ // nocommit should we default max token length to ... 256?
+
// Min/max token length, or null if there are no limits:
volatile Integer minTokenLength;
volatile Integer maxTokenLength;
@@ -382,6 +403,7 @@ public class FieldTypes {
}
break;
case ATOM:
+ case UNIQUE_ATOM:
if (highlighted == Boolean.TRUE) {
illegalState(name, "type " + valueType + " cannot highlight");
}
@@ -397,6 +419,14 @@ public class FieldTypes {
if (indexOptions != IndexOptions.NONE && indexOptions.compareTo(IndexOptions.DOCS) > 0) {
illegalState(name, "type " + valueType + " can only be indexed as DOCS; got " + indexOptions);
}
+ if (valueType == ValueType.UNIQUE_ATOM) {
+ if (indexOptions != IndexOptions.DOCS) {
+ illegalState(name, "type " + valueType + " must be indexed as DOCS; got " + indexOptions);
+ }
+ if (multiValued == Boolean.TRUE) {
+ illegalState(name, "type " + valueType + " cannot be multivalued");
+ }
+ }
if (maxTokenCount != null) {
illegalState(name, "type " + valueType + " cannot set max token count");
}
@@ -767,6 +797,9 @@ public class FieldTypes {
case INET_ADDRESS:
out.writeByte((byte) 11);
break;
+ case UNIQUE_ATOM:
+ out.writeByte((byte) 12);
+ break;
default:
throw new AssertionError("missing ValueType in switch");
}
@@ -954,6 +987,9 @@ public class FieldTypes {
case 11:
valueType = ValueType.INET_ADDRESS;
break;
+ case 12:
+ valueType = ValueType.UNIQUE_ATOM;
+ break;
default:
throw new CorruptIndexException("invalid byte for ValueType: " + b, in);
}
@@ -1074,6 +1110,7 @@ public class FieldTypes {
}
// nocommit messy we steal this from commitdata namespace...
+ /** Key used to store the field types inside {@link IndexWriter#setCommitData}. */
public static final String FIELD_TYPES_KEY = "FieldTypes";
private Version loadFields(Map<String,String> commitUserData, boolean isNewIndex) throws IOException {
@@ -1082,14 +1119,30 @@ public class FieldTypes {
if (currentFieldTypes != null) {
return readFromString(currentFieldTypes);
} else if (isNewIndex == false) {
- // nocommit must handle back compat here :)
+ // nocommit must handle back compat here
// throw new CorruptIndexException("FieldTypes is missing from this index", "CommitUserData");
+ enableExistsFilters = false;
return Version.LATEST;
} else {
+ FieldType fieldType = new FieldType(FIELD_NAMES_FIELD);
+ fields.put(FIELD_NAMES_FIELD, fieldType);
+ fieldType.multiValued = Boolean.TRUE;
+ fieldType.valueType = ValueType.ATOM;
+ fieldType.sortable = Boolean.TRUE;
+ fieldType.stored = Boolean.FALSE;
+ setDefaults(fieldType);
return Version.LATEST;
}
}
+ private FieldType newFieldType(String fieldName) {
+ if (fieldName.equals(FIELD_NAMES_FIELD)) {
+ throw new IllegalArgumentException("field name \"" + fieldName + "\" is reserved");
+ }
+
+ return new FieldType(fieldName);
+ }
+
/** Decodes String previously created by bytesToString. */
private static byte[] stringToBytes(String s) {
byte[] bytesIn = s.getBytes(IOUtils.CHARSET_UTF_8);
@@ -1152,7 +1205,7 @@ public class FieldTypes {
FieldType current = fields.get(fieldName);
if (current == null) {
- current = new FieldType(fieldName);
+ current = newFieldType(fieldName);
current.postingsFormat = postingsFormat;
fields.put(fieldName, current);
changed();
@@ -1191,7 +1244,7 @@ public class FieldTypes {
FieldType current = fields.get(fieldName);
if (current == null) {
- current = new FieldType(fieldName);
+ current = newFieldType(fieldName);
current.docValuesFormat = docValuesFormat;
fields.put(fieldName, current);
changed();
@@ -1397,7 +1450,7 @@ public class FieldTypes {
public synchronized void setIndexAnalyzer(String fieldName, Analyzer analyzer) {
FieldType current = fields.get(fieldName);
if (current == null) {
- current = new FieldType(fieldName);
+ current = newFieldType(fieldName);
current.indexAnalyzer = analyzer;
fields.put(fieldName, current);
changed();
@@ -1428,7 +1481,7 @@ public class FieldTypes {
public synchronized void setQueryAnalyzer(String fieldName, Analyzer analyzer) {
FieldType current = fields.get(fieldName);
if (current == null) {
- current = new FieldType(fieldName);
+ current = newFieldType(fieldName);
current.queryAnalyzer = analyzer;
fields.put(fieldName, current);
changed();
@@ -1459,7 +1512,7 @@ public class FieldTypes {
public synchronized void setSimilarity(String fieldName, Similarity similarity) {
FieldType current = fields.get(fieldName);
if (current == null) {
- current = new FieldType(fieldName);
+ current = newFieldType(fieldName);
current.similarity = similarity;
fields.put(fieldName, current);
changed();
@@ -1478,7 +1531,7 @@ public class FieldTypes {
public synchronized void setMultiValued(String fieldName) {
FieldType current = fields.get(fieldName);
if (current == null) {
- current = new FieldType(fieldName);
+ current = newFieldType(fieldName);
current.multiValued = Boolean.TRUE;
fields.put(fieldName, current);
changed();
@@ -1509,7 +1562,7 @@ public class FieldTypes {
public synchronized void setMinMaxTokenLength(String fieldName, int minTokenLength, int maxTokenLength) {
FieldType current = fields.get(fieldName);
if (current == null) {
- current = new FieldType(fieldName);
+ current = newFieldType(fieldName);
current.minTokenLength = minTokenLength;
current.maxTokenLength = maxTokenLength;
fields.put(fieldName, current);
@@ -1557,7 +1610,7 @@ public class FieldTypes {
public synchronized void setMaxTokenCount(String fieldName, int maxTokenCount, boolean consumeAllTokens) {
FieldType current = fields.get(fieldName);
if (current == null) {
- current = new FieldType(fieldName);
+ current = newFieldType(fieldName);
current.maxTokenCount = maxTokenCount;
current.consumeAllTokens = consumeAllTokens;
fields.put(fieldName, current);
@@ -1597,7 +1650,7 @@ public class FieldTypes {
public synchronized void setAnalyzerPositionGap(String fieldName, int gap) {
FieldType current = fields.get(fieldName);
if (current == null) {
- current = new FieldType(fieldName);
+ current = newFieldType(fieldName);
current.analyzerPositionGap = gap;
fields.put(fieldName, current);
changed();
@@ -1623,7 +1676,7 @@ public class FieldTypes {
public synchronized void setAnalyzerOffsetGap(String fieldName, int gap) {
FieldType current = fields.get(fieldName);
if (current == null) {
- current = new FieldType(fieldName);
+ current = newFieldType(fieldName);
current.analyzerOffsetGap = gap;
fields.put(fieldName, current);
changed();
@@ -1668,7 +1721,7 @@ public class FieldTypes {
FieldType current = fields.get(fieldName);
if (current == null) {
- current = new FieldType(fieldName);
+ current = newFieldType(fieldName);
current.blockTreeMinItemsInBlock = minItemsInBlock;
current.blockTreeMaxItemsInBlock = maxItemsInBlock;
fields.put(fieldName, current);
@@ -1716,7 +1769,7 @@ public class FieldTypes {
FieldType current = fields.get(fieldName);
if (current == null) {
- current = new FieldType(fieldName);
+ current = newFieldType(fieldName);
current.blockTreeMinItemsInAutoPrefix = minItemsInAutoPrefix;
current.blockTreeMaxItemsInAutoPrefix = maxItemsInAutoPrefix;
fields.put(fieldName, current);
@@ -1750,7 +1803,7 @@ public class FieldTypes {
public synchronized void enableSorting(String fieldName, boolean reversed) {
FieldType current = fields.get(fieldName);
if (current == null) {
- current = new FieldType(fieldName);
+ current = newFieldType(fieldName);
current.sortable = Boolean.TRUE;
current.sortReversed = reversed;
fields.put(fieldName, current);
@@ -1782,7 +1835,7 @@ public class FieldTypes {
public synchronized void disableSorting(String fieldName) {
FieldType current = fields.get(fieldName);
if (current == null) {
- current = new FieldType(fieldName);
+ current = newFieldType(fieldName);
current.sortable = Boolean.FALSE;
fields.put(fieldName, current);
changed();
@@ -1852,7 +1905,7 @@ public class FieldTypes {
public synchronized void enableFastRanges(String fieldName) {
FieldType current = fields.get(fieldName);
if (current == null) {
- current = new FieldType(fieldName);
+ current = newFieldType(fieldName);
current.fastRanges = Boolean.TRUE;
fields.put(fieldName, current);
changed();
@@ -1877,7 +1930,7 @@ public class FieldTypes {
public synchronized void disableFastRanges(String fieldName) {
FieldType current = fields.get(fieldName);
if (current == null) {
- current = new FieldType(fieldName);
+ current = newFieldType(fieldName);
current.fastRanges = Boolean.FALSE;
fields.put(fieldName, current);
changed();
@@ -1897,7 +1950,7 @@ public class FieldTypes {
public synchronized void enableHighlighting(String fieldName) {
FieldType current = fields.get(fieldName);
if (current == null) {
- current = new FieldType(fieldName);
+ current = newFieldType(fieldName);
current.highlighted = Boolean.TRUE;
fields.put(fieldName, current);
changed();
@@ -1922,7 +1975,7 @@ public class FieldTypes {
public synchronized void disableHighlighting(String fieldName) {
FieldType current = fields.get(fieldName);
if (current == null) {
- current = new FieldType(fieldName);
+ current = newFieldType(fieldName);
current.highlighted = Boolean.FALSE;
fields.put(fieldName, current);
changed();
@@ -1951,7 +2004,7 @@ public class FieldTypes {
// throws exc if norms were already disabled
FieldType current = fields.get(fieldName);
if (current == null) {
- current = new FieldType(fieldName);
+ current = newFieldType(fieldName);
current.indexNorms = Boolean.TRUE;
fields.put(fieldName, current);
changed();
@@ -1975,7 +2028,7 @@ public class FieldTypes {
public synchronized void disableNorms(String fieldName) {
FieldType current = fields.get(fieldName);
if (current == null) {
- current = new FieldType(fieldName);
+ current = newFieldType(fieldName);
current.indexNorms = Boolean.FALSE;
fields.put(fieldName, current);
changed();
@@ -1993,7 +2046,7 @@ public class FieldTypes {
public synchronized void enableStored(String fieldName) {
FieldType current = fields.get(fieldName);
if (current == null) {
- current = new FieldType(fieldName);
+ current = newFieldType(fieldName);
current.stored = Boolean.TRUE;
fields.put(fieldName, current);
changed();
@@ -2008,7 +2061,7 @@ public class FieldTypes {
public synchronized void disableStored(String fieldName) {
FieldType current = fields.get(fieldName);
if (current == null) {
- current = new FieldType(fieldName);
+ current = newFieldType(fieldName);
current.stored = Boolean.FALSE;
fields.put(fieldName, current);
changed();
@@ -2030,7 +2083,7 @@ public class FieldTypes {
public synchronized void enableTermVectors(String fieldName) {
FieldType current = fields.get(fieldName);
if (current == null) {
- current = new FieldType(fieldName);
+ current = newFieldType(fieldName);
current.storeTermVectors = Boolean.TRUE;
fields.put(fieldName, current);
changed();
@@ -2044,7 +2097,7 @@ public class FieldTypes {
public synchronized void disableTermVectors(String fieldName) {
FieldType current = fields.get(fieldName);
if (current == null) {
- current = new FieldType(fieldName);
+ current = newFieldType(fieldName);
current.storeTermVectors = Boolean.FALSE;
fields.put(fieldName, current);
changed();
@@ -2148,7 +2201,7 @@ public class FieldTypes {
}
FieldType current = fields.get(fieldName);
if (current == null) {
- current = new FieldType(fieldName);
+ current = newFieldType(fieldName);
current.indexOptions = indexOptions;
current.indexOptionsSet = true;
fields.put(fieldName, current);
@@ -2197,7 +2250,7 @@ public class FieldTypes {
}
FieldType current = fields.get(fieldName);
if (current == null) {
- current = new FieldType(fieldName);
+ current = newFieldType(fieldName);
current.docValuesType = dvType;
current.docValuesTypeSet = true;
fields.put(fieldName, current);
@@ -2235,9 +2288,10 @@ public class FieldTypes {
synchronized void recordValueType(String fieldName, ValueType valueType) {
ensureWritable();
+ indexedDocs = true;
FieldType current = fields.get(fieldName);
if (current == null) {
- current = new FieldType(fieldName);
+ current = newFieldType(fieldName);
current.valueType = valueType;
fields.put(fieldName, current);
setDefaults(current);
@@ -2263,9 +2317,10 @@ public class FieldTypes {
synchronized void recordLargeTextType(String fieldName, boolean allowStored, boolean indexed) {
ensureWritable();
+ indexedDocs = true;
FieldType current = fields.get(fieldName);
if (current == null) {
- current = new FieldType(fieldName);
+ current = newFieldType(fieldName);
current.valueType = ValueType.TEXT;
fields.put(fieldName, current);
setDefaults(current);
@@ -2278,17 +2333,23 @@ public class FieldTypes {
changed();
} else if (current.valueType == ValueType.NONE) {
// This can happen if e.g. the app first calls FieldTypes.setStored(...)
+ Boolean oldStored = current.stored;
boolean success = false;
try {
current.valueType = ValueType.TEXT;
- current.validate();
- if (allowStored == false && current.stored == Boolean.TRUE) {
- illegalState(fieldName, "can only store String large text fields");
+ if (allowStored == false) {
+ if (current.stored == Boolean.TRUE) {
+ illegalState(fieldName, "can only store String large text fields");
+ } else if (current.stored == null) {
+ current.stored = Boolean.FALSE;
+ }
}
+ current.validate();
success = true;
} finally {
if (success == false) {
current.valueType = ValueType.NONE;
+ current.stored = oldStored;
}
}
setDefaults(current);
@@ -2401,6 +2462,7 @@ public class FieldTypes {
break;
case ATOM:
+ case UNIQUE_ATOM:
case INET_ADDRESS:
if (field.highlighted == null) {
field.highlighted = Boolean.FALSE;
@@ -2722,8 +2784,7 @@ public class FieldTypes {
return new TermQuery(new Term(fieldName, new BytesRef(token.getAddress())));
}
- // nocommit shouldn't this be a filter?
- public Query newRangeQuery(String fieldName, Number min, boolean minInclusive, Number max, boolean maxInclusive) {
+ public Filter newRangeFilter(String fieldName, Number min, boolean minInclusive, Number max, boolean maxInclusive) {
// Field must exist:
FieldType fieldType = getFieldType(fieldName);
@@ -2772,11 +2833,14 @@ public class FieldTypes {
return null;
}
- return new TermRangeQuery(fieldName, minTerm, maxTerm, minInclusive, maxInclusive);
+ return new TermRangeFilter(fieldName, minTerm, maxTerm, minInclusive, maxInclusive);
}
- // nocommit shouldn't this be a filter?
- public Query newRangeQuery(String fieldName, BytesRef minTerm, boolean minInclusive, BytesRef maxTerm, boolean maxInclusive) {
+ public Filter newRangeFilter(String fieldName, byte[] minTerm, boolean minInclusive, byte[] maxTerm, boolean maxInclusive) {
+ return newRangeFilter(fieldName, new BytesRef(minTerm), minInclusive, new BytesRef(maxTerm), maxInclusive);
+ }
+
+ public Filter newRangeFilter(String fieldName, BytesRef minTerm, boolean minInclusive, BytesRef maxTerm, boolean maxInclusive) {
// Field must exist:
FieldType fieldType = getFieldType(fieldName);
@@ -2792,13 +2856,12 @@ public class FieldTypes {
// nocommit verify type is BINARY or ATOM?
- return new TermRangeQuery(fieldName, minTerm, maxTerm, minInclusive, maxInclusive);
+ return new TermRangeFilter(fieldName, minTerm, maxTerm, minInclusive, maxInclusive);
}
// nocommit Date sugar for a range query matching a specific hour/day/month/year/etc.? need locale/timezone... should we use DateTools?
- // nocommit shouldn't this be a filter?
- public Query newRangeQuery(String fieldName, Date min, boolean minInclusive, Date max, boolean maxInclusive) {
+ public Filter newRangeFilter(String fieldName, Date min, boolean minInclusive, Date max, boolean maxInclusive) {
// Field must exist:
FieldType fieldType = getFieldType(fieldName);
@@ -2819,12 +2882,11 @@ public class FieldTypes {
BytesRef minTerm = min == null ? null : Document2.longToBytes(min.getTime());
BytesRef maxTerm = max == null ? null : Document2.longToBytes(max.getTime());
- return new TermRangeQuery(fieldName, minTerm, maxTerm, minInclusive, maxInclusive);
+ return new TermRangeFilter(fieldName, minTerm, maxTerm, minInclusive, maxInclusive);
}
- // nocommit shouldn't this be a filter?
// nocommit also add "range filter using net mask" sugar version
- public Query newRangeQuery(String fieldName, InetAddress min, boolean minInclusive, InetAddress max, boolean maxInclusive) {
+ public Filter newRangeFilter(String fieldName, InetAddress min, boolean minInclusive, InetAddress max, boolean maxInclusive) {
// Field must exist:
FieldType fieldType = getFieldType(fieldName);
@@ -2845,7 +2907,7 @@ public class FieldTypes {
BytesRef minTerm = min == null ? null : new BytesRef(min.getAddress());
BytesRef maxTerm = max == null ? null : new BytesRef(max.getAddress());
- return new TermRangeQuery(fieldName, minTerm, maxTerm, minInclusive, maxInclusive);
+ return new TermRangeFilter(fieldName, minTerm, maxTerm, minInclusive, maxInclusive);
}
// nocommit newPhraseQuery?
@@ -2999,6 +3061,7 @@ public class FieldTypes {
case SHORT_TEXT:
case ATOM:
+ case UNIQUE_ATOM:
case BINARY:
case BOOLEAN:
case INET_ADDRESS:
@@ -3034,11 +3097,21 @@ public class FieldTypes {
// Dead code but javac disagrees:
return null;
}
- }
+ }
+
+ /** Returns a {@link Filter} accepting documents that have this field. */
+ public Filter newFieldExistsFilter(String fieldName) {
+ if (enableExistsFilters == false) {
+ throw new IllegalStateException("field exists filter was disabled");
+ }
+
+ // nocommit TermFilter?
+ // nocommit optimize this filter to MatchAllDocs when Terms.getDocCount() == maxDoc
+ return new QueryWrapperFilter(new TermQuery(new Term(FIELD_NAMES_FIELD, fieldName)));
+ }
private synchronized void changed() {
ensureWritable();
- // Push to IW's commit data
changeCount++;
}
@@ -3070,6 +3143,9 @@ public class FieldTypes {
out.writeVInt(indexCreatedVersion.minor);
out.writeVInt(indexCreatedVersion.bugfix);
+ writeBoolean(out, enableExistsFilters);
+ writeBoolean(out, indexedDocs);
+
out.writeVInt(fields.size());
for(FieldType fieldType : fields.values()) {
fieldType.write(out);
@@ -3086,6 +3162,7 @@ public class FieldTypes {
/** Reads FieldTypes from previously saved. */
private synchronized Version readFromString(String stringIn) throws IOException {
+
byte[] bytesIn = stringToBytes(stringIn);
RAMFile file = new RAMFile();
RAMOutputStream out = new RAMOutputStream(file, false);
@@ -3098,6 +3175,9 @@ public class FieldTypes {
Version indexCreatedVersion = Version.fromBits(in.readVInt(), in.readVInt(), in.readVInt());
+ enableExistsFilters = readBoolean(in);
+ indexedDocs = readBoolean(in);
+
int count = in.readVInt();
for(int i=0;i<count;i++) {
FieldType fieldType = new FieldType(in);
@@ -3134,6 +3214,10 @@ public class FieldTypes {
return new FieldTypes(commitUserData, defaultQueryAnalyzer, defaultSimilarity);
}
+ public boolean isUniqueAtom(String fieldName) {
+ return getFieldType(fieldName).valueType == ValueType.UNIQUE_ATOM;
+ }
+
public Iterable<String> getFieldNames() {
return Collections.unmodifiableSet(fields.keySet());
}
@@ -3151,10 +3235,50 @@ public class FieldTypes {
}
}
+ /** Returns true if values in this field must be unique across all documents in the index. */
+ public synchronized boolean isUnique(String fieldName) {
+ FieldType current = fields.get(fieldName);
+ return current != null && current.valueType == ValueType.UNIQUE_ATOM;
+ }
+
/** Defines a dynamic field, computed by a Javascript expression referring
* to other field values, to be used for sorting. */
public void addIntExpressionField(String fieldName, String expression) {
// nocommit how to do this? must we make a FieldTypes subclass in expressions module = pita?
}
+
// nocommit also long, float, double
+
+ public synchronized void enableExistsFilters() {
+ if (enableExistsFilters == false && indexedDocs) {
+ throw new IllegalStateException("cannot enable exists filters after documents were already indexed");
+ }
+ enableExistsFilters = true;
+ }
+
+ public synchronized void disableExistsFilters() {
+ if (enableExistsFilters && indexedDocs) {
+ throw new IllegalStateException("cannot disable exists filters after documents were already indexed");
+ }
+ enableExistsFilters = false;
+ }
+
+ private static void writeBoolean(DataOutput out, boolean value) throws IOException {
+ if (value) {
+ out.writeByte((byte) 1);
+ } else {
+ out.writeByte((byte) 0);
+ }
+ }
+
+ private static boolean readBoolean(DataInput in) throws IOException {
+ byte b = in.readByte();
+ if (b == 1) {
+ return true;
+ } else if (b == 0) {
+ return false;
+ } else {
+ throw new CorruptIndexException("invalid byte for boolean: " + b, 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=1637540&r1=1637539&r2=1637540&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 Sat Nov 8 11:05:05 2014
@@ -478,8 +478,11 @@ 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;
@@ -582,6 +585,8 @@ public class CheckIndex implements Close
result.newSegments.clear();
result.maxSegmentName = -1;
+ IndexReader[] segmentReaders = new IndexReader[numSegments];
+
for(int i=0;i<numSegments;i++) {
final SegmentCommitInfo info = sis.info(i);
int segmentName = Integer.parseInt(info.info.name.substring(1), Character.MAX_RADIX);
@@ -637,6 +642,7 @@ public class CheckIndex implements Close
infoStream.print(" test: open reader.........");
reader = new SegmentReader(fieldTypes, info, IOContext.DEFAULT);
msg(infoStream, "OK");
+ segmentReaders[i] = reader;
segInfoStat.openReaderPassed = true;
@@ -727,15 +733,47 @@ public class CheckIndex implements Close
result.totLoseDocCount += toLoseDocCount;
result.numBadSegments++;
continue;
- } finally {
- if (reader != null)
- reader.close();
}
// Keeper
result.newSegments.add(info.clone());
}
+ if (onlySegments == null && result.numBadSegments == 0) {
+ MultiReader topReader = new MultiReader(segmentReaders);
+ try {
+ for(String fieldName : fieldTypes.getFieldNames()) {
+ if (fieldTypes.isUniqueAtom(fieldName)) {
+ Terms terms = MultiFields.getTerms(topReader, fieldName);
+ if (terms != null) {
+ Bits liveDocs = MultiFields.getLiveDocs(topReader);
+ 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) {
+ int docID2 = docsEnum.nextDoc();
+ if (docID2 != DocsEnum.NO_MORE_DOCS) {
+ msg(infoStream, "FAILED");
+ // nocommit should "isUnique" be in low schema?
+ // nocommit have -fix delete the offenders:
+ String comment = "UNIQUE_ATOM field=\"" + fieldName + "\" is not unique: term=" + termsEnum.term() + " matches both docID=" + docID + " and docID=" + docID2 + "; unable to fix this index";
+ msg(infoStream, comment);
+ if (failFast) {
+ throw new RuntimeException(comment);
+ }
+ }
+ }
+ }
+ }
+ }
+ }
+ } finally {
+ topReader.close();
+ }
+ }
+
if (0 == result.numBadSegments) {
result.clean = true;
} else
@@ -2044,17 +2082,16 @@ public class CheckIndex implements Close
// Again, with the one doc deleted:
checkFields(tfv, onlyDocIsDeleted, 1, fieldInfos, false, true, infoStream, verbose);
- // Only agg stats if the doc is live:
- final boolean doStats = liveDocs == null || liveDocs.get(j);
-
- if (doStats) {
- status.docCount++;
+ if (liveDocs != null && liveDocs.get(j) == false) {
+ // Only check live docs
+ continue;
}
+ status.docCount++;
+
for(String field : tfv) {
- if (doStats) {
- status.totVectors++;
- }
+
+ status.totVectors++;
// Make sure FieldInfo thinks this field is vector'd:
final FieldInfo fieldInfo = fieldInfos.fieldInfo(field);
@@ -2268,7 +2305,7 @@ public class CheckIndex implements Close
segment(s). This can be specified multiple times,
to check more than one segment, eg <code>-segment _2
-segment _a</code>. You can't use this with the -exorcise
- option.
+ option. Note that this skips certain top-level checks.
</ul>
<p><b>WARNING</b>: <code>-exorcise</code> should only be used on an emergency basis as it will cause
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=1637540&r1=1637539&r2=1637540&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 Sat Nov 8 11:05:05 2014
@@ -23,12 +23,12 @@ import java.util.HashMap;
import java.util.Map;
import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.analysis.tokenattributes.TermToBytesRefAttribute;
import org.apache.lucene.codecs.DocValuesConsumer;
import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.codecs.NormsConsumer;
import org.apache.lucene.codecs.NormsFormat;
import org.apache.lucene.codecs.StoredFieldsWriter;
-import org.apache.lucene.document.FieldType;
import org.apache.lucene.search.similarities.Similarity;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.util.ArrayUtil;
@@ -89,6 +89,7 @@ final class DefaultIndexingChain extends
// aborting on any exception from this method
int numDocs = state.segmentInfo.getDocCount();
+
writeNorms(state);
writeDocValues(state);
@@ -286,7 +287,7 @@ final class DefaultIndexingChain extends
}
@Override
- public void processDocument() throws IOException {
+ public void processDocument(Term delTerm) throws IOException {
// How many indexed field names we've seen (collapses
// multiple field instances by the same name):
@@ -308,7 +309,7 @@ final class DefaultIndexingChain extends
try {
for (IndexableField field : docState.doc) {
- fieldCount = processField(field, fieldGen, fieldCount);
+ fieldCount = processField(field, fieldGen, fieldCount, delTerm);
}
} finally {
if (docWriter.aborting == false) {
@@ -333,7 +334,7 @@ final class DefaultIndexingChain extends
}
}
- private int processField(IndexableField field, long fieldGen, int fieldCount) throws IOException {
+ private int processField(IndexableField field, long fieldGen, int fieldCount, Term delTerm) throws IOException {
String fieldName = field.name();
IndexableFieldType fieldType = field.fieldType();
@@ -353,7 +354,7 @@ final class DefaultIndexingChain extends
fp = getOrAddField(fieldName, fieldType, true);
boolean first = fp.fieldGen != fieldGen;
- fp.invert(field, first);
+ fp.invert(field, first, delTerm);
if (first) {
fields[fieldCount++] = fp;
@@ -483,7 +484,7 @@ final class DefaultIndexingChain extends
}
/** Returns a previously created {@link PerField},
- * absorbing the type information from {@link FieldType},
+ * absorbing the type information from {@link IndexableFieldType},
* and creates a new {@link PerField} if this field name
* wasn't seen yet. */
private PerField getOrAddField(String name, IndexableFieldType fieldType, boolean invert) {
@@ -519,6 +520,8 @@ final class DefaultIndexingChain extends
} else {
fp.fieldInfo.update(fieldType);
+ // NOTE: messy, but we must do this in case field was first seen w/o being
+ // indexed, and now is seen again, this time being indexed:
if (invert && fp.invertState == null) {
fp.setInvertState();
}
@@ -535,6 +538,7 @@ final class DefaultIndexingChain extends
FieldInvertState invertState;
TermsHashPerField termsHashPerField;
+ final LiveUniqueValues uniqueValues;
// Non-null if this field ever had doc values in this
// segment:
@@ -559,6 +563,7 @@ final class DefaultIndexingChain extends
if (invert) {
setInvertState();
}
+ uniqueValues = docWriter.writer.getUniqueValues(fieldInfo.name);
}
void setInvertState() {
@@ -587,7 +592,7 @@ final class DefaultIndexingChain extends
/** Inverts one field for one document; first is true
* if this is the first time we are seeing this field
* name in this document. */
- public void invert(IndexableField field, boolean first) throws IOException {
+ public void invert(IndexableField field, boolean first, Term delTerm) throws IOException {
if (first) {
// First time we're seeing this field (indexed) in
// this document:
@@ -666,6 +671,23 @@ final class DefaultIndexingChain extends
aborting = true;
termsHashPerField.add();
aborting = false;
+
+ // maybe low-schema should know "isUnique"?
+
+ 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 ||
+ delTerm.bytes().equals(token) == false)) {
+ // Unique constraint violated; document will be marked deleted above:
+ throw new NotUniqueException(field.name(), token);
+ }
+ if (stream.incrementToken() != false) {
+ throw new IllegalArgumentException("field \"" + field.name() + "\": unique fields must have a single token");
+ }
+ }
}
// trigger streams to perform end-of-stream operations
Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/DocConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/DocConsumer.java?rev=1637540&r1=1637539&r2=1637540&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/DocConsumer.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/DocConsumer.java Sat Nov 8 11:05:05 2014
@@ -20,7 +20,7 @@ package org.apache.lucene.index;
import java.io.IOException;
abstract class DocConsumer {
- abstract void processDocument() throws IOException;
+ abstract void processDocument(Term delTerm) throws IOException;
abstract void flush(final SegmentWriteState state) throws IOException;
abstract void abort();
}
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=1637540&r1=1637539&r2=1637540&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 Sat Nov 8 11:05:05 2014
@@ -19,9 +19,11 @@ package org.apache.lucene.index;
import java.io.Closeable;
import java.io.IOException;
+import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.HashSet;
+import java.util.List;
import java.util.Queue;
import java.util.Set;
import java.util.concurrent.ConcurrentLinkedQueue;
@@ -138,6 +140,7 @@ final class DocumentsWriter implements C
synchronized boolean deleteQueries(final Query... queries) throws IOException {
// TODO why is this synchronized?
+ // nocommit do we disallow this when there are unique fields?
final DocumentsWriterDeleteQueue deleteQueue = this.deleteQueue;
deleteQueue.addDelete(queries);
flushControl.doOnDelete();
@@ -149,10 +152,18 @@ final class DocumentsWriter implements C
// per-DWPT map (but still must go into the global map)
synchronized boolean deleteTerms(final Term... terms) throws IOException {
// TODO why is this synchronized?
+ for(Term term : terms) {
+ LiveUniqueValues uniqueValues = writer.getUniqueValues(term.field());
+ if (uniqueValues != null) {
+ // We must live-delete this field:
+ uniqueValues.delete(term.bytes());
+ }
+ }
+
final DocumentsWriterDeleteQueue deleteQueue = this.deleteQueue;
deleteQueue.addDelete(terms);
flushControl.doOnDelete();
- return applyAllDeletes( deleteQueue);
+ return applyAllDeletes(deleteQueue);
}
synchronized boolean updateDocValues(DocValuesUpdate... updates) throws IOException {
Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java?rev=1637540&r1=1637539&r2=1637540&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java Sat Nov 8 11:05:05 2014
@@ -26,6 +26,7 @@ import java.util.concurrent.atomic.Atomi
import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.codecs.Codec;
+import org.apache.lucene.document.FieldTypes;
import org.apache.lucene.index.DocumentsWriterDeleteQueue.DeleteSlice;
import org.apache.lucene.search.similarities.Similarity;
import org.apache.lucene.store.Directory;
@@ -157,16 +158,20 @@ class DocumentsWriterPerThread {
final IntBlockPool.Allocator intBlockAllocator;
private final AtomicLong pendingNumDocs;
private final LiveIndexWriterConfig indexWriterConfig;
-
+ final FieldTypes fieldTypes;
+ final IndexWriter writer;
+
public DocumentsWriterPerThread(String segmentName, IndexWriter writer, Directory directory,
InfoStream infoStream, DocumentsWriterDeleteQueue deleteQueue,
FieldInfos.Builder fieldInfos, AtomicLong pendingNumDocs) throws IOException {
this.directoryOrig = directory;
+ this.writer = writer;
this.directory = new TrackingDirectoryWrapper(directory);
this.fieldInfos = fieldInfos;
this.indexWriterConfig = writer.config;
this.infoStream = infoStream;
this.codec = writer.codec;
+ this.fieldTypes = writer.fieldTypes;
this.docState = new DocState(this, infoStream);
this.docState.similarity = writer.fieldTypes.getSimilarity();
this.pendingNumDocs = pendingNumDocs;
@@ -238,7 +243,7 @@ class DocumentsWriterPerThread {
boolean success = false;
try {
try {
- consumer.processDocument();
+ consumer.processDocument(delTerm);
} finally {
docState.clear();
}
@@ -282,7 +287,7 @@ class DocumentsWriterPerThread {
boolean success = false;
try {
- consumer.processDocument();
+ consumer.processDocument(delTerm);
success = true;
} finally {
if (!success) {
Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/FieldInfo.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/FieldInfo.java?rev=1637540&r1=1637539&r2=1637540&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/FieldInfo.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/FieldInfo.java Sat Nov 8 11:05:05 2014
@@ -147,7 +147,7 @@ public final class FieldInfo {
if (type == null) {
throw new NullPointerException("DocValuesType cannot be null (field: \"" + name + "\")");
}
- if (docValuesType != DocValuesType.NONE && docValuesType != type) {
+ if (docValuesType != DocValuesType.NONE && type != DocValuesType.NONE && docValuesType != type) {
throw new IllegalArgumentException("cannot change DocValues type from " + docValuesType + " to " + type + " for field \"" + name + "\"");
}
docValuesType = type;
Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/FieldInfos.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/FieldInfos.java?rev=1637540&r1=1637539&r2=1637540&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/FieldInfos.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/FieldInfos.java Sat Nov 8 11:05:05 2014
@@ -279,7 +279,7 @@ public class FieldInfos implements Itera
/** NOTE: this method does not carry over termVector
* the indexer chain must set these fields when they
- * succeed in consuming the document */
+ * succeed in consuming the document, nor the DocValuesType */
public FieldInfo addOrUpdate(String name, IndexableFieldType fieldType) {
// TODO: really, indexer shouldn't even call this
// method (it's only called from DocFieldProcessor);
@@ -288,7 +288,7 @@ public class FieldInfos implements Itera
// be updated by maybe FreqProxTermsWriterPerField:
return addOrUpdateInternal(name, -1, false,
fieldType.omitNorms(), false,
- fieldType.indexOptions(), fieldType.docValueType());
+ fieldType.indexOptions(), DocValuesType.NONE);
}
private FieldInfo addOrUpdateInternal(String name, int preferredFieldNumber,
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=1637540&r1=1637539&r2=1637540&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 Sat Nov 8 11:05:05 2014
@@ -37,6 +37,8 @@ import org.apache.lucene.util.BytesRefBu
class FreqProxFields extends Fields {
final Map<String,FreqProxTermsWriterPerField> fields = new LinkedHashMap<>();
+ private Bits liveDocs;
+
public FreqProxFields(List<FreqProxTermsWriterPerField> fieldList) {
// NOTE: fields are already sorted by field name
for(FreqProxTermsWriterPerField field : fieldList) {
@@ -44,6 +46,10 @@ class FreqProxFields extends Fields {
}
}
+ public void setLiveDocs(Bits liveDocs) {
+ this.liveDocs = liveDocs;
+ }
+
public Iterator<String> iterator() {
return fields.keySet().iterator();
}
@@ -51,7 +57,7 @@ class FreqProxFields extends Fields {
@Override
public Terms terms(String field) throws IOException {
FreqProxTermsWriterPerField perField = fields.get(field);
- return perField == null ? null : new FreqProxTerms(perField);
+ return perField == null ? null : new FreqProxTerms(perField, liveDocs);
}
@Override
@@ -62,9 +68,11 @@ class FreqProxFields extends Fields {
private static class FreqProxTerms extends Terms {
final FreqProxTermsWriterPerField terms;
+ final Bits liveDocs;
- public FreqProxTerms(FreqProxTermsWriterPerField terms) {
+ public FreqProxTerms(FreqProxTermsWriterPerField terms, Bits liveDocs) {
this.terms = terms;
+ this.liveDocs = liveDocs;
}
@Override
@@ -72,8 +80,9 @@ class FreqProxFields extends Fields {
FreqProxTermsEnum termsEnum;
if (reuse instanceof FreqProxTermsEnum && ((FreqProxTermsEnum) reuse).terms == this.terms) {
termsEnum = (FreqProxTermsEnum) reuse;
+ assert termsEnum.liveDocs == this.liveDocs;
} else {
- termsEnum = new FreqProxTermsEnum(terms);
+ termsEnum = new FreqProxTermsEnum(terms, liveDocs);
}
termsEnum.reset();
return termsEnum;
@@ -136,11 +145,13 @@ class FreqProxFields extends Fields {
final FreqProxPostingsArray postingsArray;
final BytesRef scratch = new BytesRef();
final int numTerms;
+ final Bits liveDocs;
int ord;
- public FreqProxTermsEnum(FreqProxTermsWriterPerField terms) {
+ public FreqProxTermsEnum(FreqProxTermsWriterPerField terms, Bits liveDocs) {
this.terms = terms;
this.numTerms = terms.bytesHash.size();
+ this.liveDocs = liveDocs;
sortedTermIDs = terms.sortedTermIDs;
assert sortedTermIDs != null;
postingsArray = (FreqProxPostingsArray) terms.postingsArray;
@@ -231,8 +242,8 @@ class FreqProxFields extends Fields {
}
@Override
- public DocsEnum docs(Bits liveDocs, DocsEnum reuse, int flags) {
- if (liveDocs != null) {
+ public DocsEnum docs(Bits liveDocsIn, DocsEnum reuse, int flags) {
+ if (liveDocsIn != null) {
throw new IllegalArgumentException("liveDocs must be null");
}
@@ -247,18 +258,20 @@ class FreqProxFields extends Fields {
if (reuse instanceof FreqProxDocsEnum) {
docsEnum = (FreqProxDocsEnum) reuse;
if (docsEnum.postingsArray != postingsArray) {
- docsEnum = new FreqProxDocsEnum(terms, postingsArray);
+ docsEnum = new FreqProxDocsEnum(terms, postingsArray, liveDocs);
+ } else {
+ assert docsEnum.liveDocs == liveDocs;
}
} else {
- docsEnum = new FreqProxDocsEnum(terms, postingsArray);
+ docsEnum = new FreqProxDocsEnum(terms, postingsArray, liveDocs);
}
docsEnum.reset(sortedTermIDs[ord]);
return docsEnum;
}
@Override
- public DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse, int flags) {
- if (liveDocs != null) {
+ public DocsAndPositionsEnum docsAndPositions(Bits liveDocsIn, DocsAndPositionsEnum reuse, int flags) {
+ if (liveDocsIn != null) {
throw new IllegalArgumentException("liveDocs must be null");
}
FreqProxDocsAndPositionsEnum posEnum;
@@ -278,10 +291,12 @@ class FreqProxFields extends Fields {
if (reuse instanceof FreqProxDocsAndPositionsEnum) {
posEnum = (FreqProxDocsAndPositionsEnum) reuse;
if (posEnum.postingsArray != postingsArray) {
- posEnum = new FreqProxDocsAndPositionsEnum(terms, postingsArray);
+ posEnum = new FreqProxDocsAndPositionsEnum(terms, postingsArray, liveDocs);
+ } else {
+ assert posEnum.liveDocs == liveDocs;
}
} else {
- posEnum = new FreqProxDocsAndPositionsEnum(terms, postingsArray);
+ posEnum = new FreqProxDocsAndPositionsEnum(terms, postingsArray, liveDocs);
}
posEnum.reset(sortedTermIDs[ord]);
return posEnum;
@@ -314,15 +329,17 @@ class FreqProxFields extends Fields {
final FreqProxPostingsArray postingsArray;
final ByteSliceReader reader = new ByteSliceReader();
final boolean readTermFreq;
+ final Bits liveDocs;
int docID;
int freq;
boolean ended;
int termID;
- public FreqProxDocsEnum(FreqProxTermsWriterPerField terms, FreqProxPostingsArray postingsArray) {
+ public FreqProxDocsEnum(FreqProxTermsWriterPerField terms, FreqProxPostingsArray postingsArray, Bits liveDocs) {
this.terms = terms;
this.postingsArray = postingsArray;
this.readTermFreq = terms.hasFreq;
+ this.liveDocs = liveDocs;
}
public void reset(int termID) {
@@ -353,33 +370,37 @@ class FreqProxFields extends Fields {
if (docID == -1) {
docID = 0;
}
- if (reader.eof()) {
- if (ended) {
- return NO_MORE_DOCS;
- } else {
- ended = true;
- docID = postingsArray.lastDocIDs[termID];
- if (readTermFreq) {
- freq = postingsArray.termFreqs[termID];
+ while (true) {
+ if (reader.eof()) {
+ if (ended) {
+ return NO_MORE_DOCS;
+ } else {
+ ended = true;
+ docID = postingsArray.lastDocIDs[termID];
+ if (readTermFreq) {
+ freq = postingsArray.termFreqs[termID];
+ }
}
- }
- } else {
- int code = reader.readVInt();
- if (!readTermFreq) {
- docID += code;
} else {
- docID += code >>> 1;
- if ((code & 1) != 0) {
- freq = 1;
+ int code = reader.readVInt();
+ if (!readTermFreq) {
+ docID += code;
} else {
- freq = reader.readVInt();
+ docID += code >>> 1;
+ if ((code & 1) != 0) {
+ freq = 1;
+ } else {
+ freq = reader.readVInt();
+ }
}
+
+ assert docID != postingsArray.lastDocIDs[termID];
}
- assert docID != postingsArray.lastDocIDs[termID];
+ if (liveDocs == null || liveDocs.get(docID)) {
+ return docID;
+ }
}
-
- return docID;
}
@Override
@@ -400,6 +421,7 @@ class FreqProxFields extends Fields {
final ByteSliceReader reader = new ByteSliceReader();
final ByteSliceReader posReader = new ByteSliceReader();
final boolean readOffsets;
+ final Bits liveDocs;
int docID;
int freq;
int pos;
@@ -411,10 +433,11 @@ class FreqProxFields extends Fields {
boolean hasPayload;
BytesRefBuilder payload = new BytesRefBuilder();
- public FreqProxDocsAndPositionsEnum(FreqProxTermsWriterPerField terms, FreqProxPostingsArray postingsArray) {
+ public FreqProxDocsAndPositionsEnum(FreqProxTermsWriterPerField terms, FreqProxPostingsArray postingsArray, Bits liveDocs) {
this.terms = terms;
this.postingsArray = postingsArray;
this.readOffsets = terms.hasOffsets;
+ this.liveDocs = liveDocs;
assert terms.hasProx;
assert terms.hasFreq;
}
@@ -440,34 +463,40 @@ class FreqProxFields extends Fields {
@Override
public int nextDoc() throws IOException {
- while (posLeft != 0) {
- nextPosition();
- }
+ while (true) {
+ while (posLeft != 0) {
+ nextPosition();
+ }
- if (reader.eof()) {
- if (ended) {
- return NO_MORE_DOCS;
+ if (reader.eof()) {
+ if (ended) {
+ return NO_MORE_DOCS;
+ } else {
+ ended = true;
+ docID = postingsArray.lastDocIDs[termID];
+ freq = postingsArray.termFreqs[termID];
+ }
} else {
- ended = true;
- docID = postingsArray.lastDocIDs[termID];
- freq = postingsArray.termFreqs[termID];
+ int code = reader.readVInt();
+ docID += code >>> 1;
+ if ((code & 1) != 0) {
+ freq = 1;
+ } else {
+ freq = reader.readVInt();
+ }
+
+ assert docID != postingsArray.lastDocIDs[termID];
}
- } else {
- int code = reader.readVInt();
- docID += code >>> 1;
- if ((code & 1) != 0) {
- freq = 1;
- } else {
- freq = reader.readVInt();
+
+ posLeft = freq;
+ pos = 0;
+ startOffset = 0;
+ if (liveDocs != null && liveDocs.get(docID) == false) {
+ continue;
}
- assert docID != postingsArray.lastDocIDs[termID];
+ return docID;
}
-
- posLeft = freq;
- pos = 0;
- startOffset = 0;
- return docID;
}
@Override
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=1637540&r1=1637539&r2=1637540&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 Sat Nov 8 11:05:05 2014
@@ -97,10 +97,14 @@ final class FreqProxTermsWriter extends
// Sort by field name
CollectionUtil.introSort(allFields);
- Fields fields = new FreqProxFields(allFields);
+ FreqProxFields fields = new FreqProxFields(allFields);
applyDeletes(state, fields);
+ if (state.liveDocs != null) {
+ fields.setLiveDocs(state.liveDocs);
+ }
+
FieldsConsumer consumer = state.segmentInfo.getCodec().postingsFormat().fieldsConsumer(state);
boolean success = false;
try {
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=1637540&r1=1637539&r2=1637540&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 Sat Nov 8 11:05:05 2014
@@ -52,6 +52,7 @@ import org.apache.lucene.index.FieldInfo
import org.apache.lucene.index.IndexWriterConfig.OpenMode;
import org.apache.lucene.index.MergeState.CheckAbort;
import org.apache.lucene.search.Query;
+import org.apache.lucene.search.ReferenceManager;
import org.apache.lucene.store.AlreadyClosedException;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
@@ -268,6 +269,11 @@ public class IndexWriter implements Clos
private final DocumentsWriter docWriter;
private final Queue<Event> eventQueue;
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
private Map<SegmentCommitInfo,Boolean> segmentsToMerge = new HashMap<>();
@@ -387,6 +393,8 @@ 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")) {
@@ -871,14 +879,17 @@ public class IndexWriter implements Clos
codec = fieldTypes.getCodec();
}
+ // nocommit can we make this lazy-open the reader?
+ readerManager = new ReaderManager(this, true);
+
success = true;
} finally {
if (!success) {
if (infoStream.isEnabled("IW")) {
- infoStream.message("IW", "init: hit exception on init; releasing write lock");
+ infoStream.message("IW", "init: hit exception on init; releasing write lock and closing");
}
- IOUtils.closeWhileHandlingException(writeLock);
+ IOUtils.closeWhileHandlingException(writeLock, this);
writeLock = null;
}
}
@@ -2011,9 +2022,11 @@ public class IndexWriter implements Clos
}
// Must pre-close in case it increments changeCount so that we can then
- // set it to false before calling closeInternal
+ // set it to false before closing
mergeScheduler.close();
+ readerManager.close();
+
bufferedUpdatesStream.clear();
docWriter.close(); // mark it as closed first to prevent subsequent indexing actions/flushes
docWriter.abort(this); // don't sync on IW here
@@ -2088,7 +2101,7 @@ public class IndexWriter implements Clos
}
// close all the closeables we can (but important is readerPool and writeLock to prevent leaks)
- IOUtils.closeWhileHandlingException(readerPool, deleter, writeLock);
+ IOUtils.closeWhileHandlingException(readerManager, readerPool, deleter, writeLock);
writeLock = null;
}
closed = true;
@@ -2409,6 +2422,8 @@ 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);
@@ -2543,6 +2558,8 @@ public class IndexWriter implements Clos
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...)");
@@ -3089,7 +3106,9 @@ public class IndexWriter implements Clos
infoStream.message("IW", "don't apply deletes now delTermCount=" + bufferedUpdatesStream.numTerms() + " bytesUsed=" + bufferedUpdatesStream.ramBytesUsed());
}
}
-
+
+ // 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;
@@ -4597,7 +4616,6 @@ public class IndexWriter implements Clos
maybeMerge(config.getMergePolicy(), MergeTrigger.SEGMENT_FLUSH, UNBOUNDED_MAX_MERGE_SEGMENTS);
}
}
-
}
synchronized void incRefDeleter(SegmentInfos segmentInfos) throws IOException {
@@ -4606,7 +4624,7 @@ public class IndexWriter implements Clos
}
synchronized void decRefDeleter(SegmentInfos segmentInfos) throws IOException {
- ensureOpen();
+ ensureOpen(false);
deleter.decRef(segmentInfos);
}
@@ -4671,4 +4689,32 @@ public class IndexWriter implements Clos
throw new IllegalStateException("number of documents in the index cannot exceed " + actualMaxDocs);
}
}
+
+ // 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() {
+ return readerManager;
+ }
+
+ // nocommit 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
+
+ synchronized LiveUniqueValues getUniqueValues(String uidFieldName) {
+ LiveUniqueValues v;
+ if (fieldTypes.isUnique(uidFieldName)) {
+ v = uniqueValues.get(uidFieldName);
+ if (v == null) {
+ v = new LiveUniqueValues(uidFieldName, readerManager);
+ uniqueValues.put(uidFieldName, v);
+ }
+ } else {
+ v = null;
+ }
+
+ return v;
+ }
}
Added: 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=1637540&view=auto
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/LiveUniqueValues.java (added)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/LiveUniqueValues.java Sat Nov 8 11:05:05 2014
@@ -0,0 +1,254 @@
+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.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+import org.apache.lucene.index.DocsAndPositionsEnum;
+import org.apache.lucene.index.DocsEnum;
+import org.apache.lucene.index.Fields;
+import org.apache.lucene.index.IndexReader.ReaderClosedListener;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.NumericDocValues;
+import org.apache.lucene.index.Terms;
+import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.search.ReferenceManager;
+import org.apache.lucene.util.Accountable;
+import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.CloseableThreadLocal;
+
+// nocommit javadocs
+
+// nocommit better name
+
+// TODO: should this class handle deletions better...?
+final class LiveUniqueValues implements ReferenceManager.RefreshListener, Closeable, Accountable {
+
+ // Holds reused TermsEnum/DocsEnum state for faster lookups:
+ private final ConcurrentMap<IndexReader,CloseableThreadLocal<PerThreadLookup>> lookupStates = new ConcurrentHashMap<>();
+
+ // Evicts this reader from lookupStates once it's closed:
+ private final ReaderClosedListener removeLookupState = new ReaderClosedListener() {
+ @Override
+ public void onClose(IndexReader reader) {
+ CloseableThreadLocal<PerThreadLookup> ctl = lookupStates.remove(reader);
+ if (ctl != null) {
+ ctl.close();
+ }
+ }
+ };
+
+ // Maps the id to TRUE if it's live, else FALSE:
+ private volatile Map<BytesRef,Boolean> old = newMap();
+ private volatile Map<BytesRef,Boolean> current = newMap();
+ private final ReaderManager mgr;
+ private final String uidField;
+
+ private static Map<BytesRef,Boolean> newMap() {
+ return new HashMap<BytesRef,Boolean>();
+ }
+
+ /** Sole constructor. */
+ public LiveUniqueValues(String uidField, ReaderManager mgr) {
+ this.uidField = uidField;
+ this.mgr = mgr;
+ mgr.addListener(this);
+ }
+
+ @Override
+ public void close() {
+ mgr.removeListener(this);
+ }
+
+ @Override
+ public synchronized void beforeRefresh() throws IOException {
+ old = current;
+ // Start sending all updates after this point to the new
+ // map. While reopen is running, any lookup will first
+ // try this new map, then fallback to old, then to the
+ // current searcher:
+ current = newMap();
+ }
+
+ @Override
+ public synchronized void afterRefresh(boolean didRefresh) throws IOException {
+ // Now drop all the old values because they are now
+ // visible via the searcher that was just opened; if
+ // didRefresh is false, it's possible old has some
+ // entries in it, which is fine: it means they were
+ // actually already included in the previously opened
+ // reader. So we can safely clear old here:
+ old = newMap();
+ }
+
+ /** Call this to try adding a value; this returns false if the add
+ * fails because the value is already present in this field. */
+ // TODO: improve concurrency
+ public synchronized boolean add(BytesRef id) throws IOException {
+ Boolean v = current.get(id);
+ if (v != null) {
+ if (v == Boolean.FALSE) {
+ current.put(id, Boolean.TRUE);
+ return true;
+ } else {
+ return false;
+ }
+ }
+ v = old.get(id);
+ if (v != null) {
+ if (v == Boolean.FALSE) {
+ current.put(id, Boolean.TRUE);
+ return true;
+ } else {
+ return false;
+ }
+ }
+ DirectoryReader reader = mgr.acquire();
+ try {
+ PerThreadLookup lookup = getLookupState(reader);
+ if (lookup.exists(id)) {
+ return false;
+ } else {
+ current.put(id, Boolean.TRUE);
+ return true;
+ }
+ } finally {
+ mgr.release(reader);
+ }
+ }
+
+ /** Call this after you've successfully deleted a document
+ * from the index. */
+ public synchronized void delete(BytesRef id) {
+ current.put(id, Boolean.FALSE);
+ }
+
+ /** Returns the [approximate] number of id/value pairs
+ * buffered in RAM. */
+ public synchronized int size() {
+ return current.size() + old.size();
+ }
+
+ private PerThreadLookup getLookupState(DirectoryReader reader) throws IOException {
+ CloseableThreadLocal<PerThreadLookup> ctl = lookupStates.get(reader);
+ if (ctl == null) {
+ // First time we are seeing this reader; make a new CTL:
+ ctl = new CloseableThreadLocal<PerThreadLookup>();
+ CloseableThreadLocal<PerThreadLookup> other = lookupStates.putIfAbsent(reader, ctl);
+ if (other == null) {
+ // Our CTL won, we must remove it when the reader is closed:
+ reader.addReaderClosedListener(removeLookupState);
+ } else {
+ // Another thread beat us to it: just use their CTL:
+ ctl.close();
+ ctl = other;
+ }
+ }
+
+ PerThreadLookup lookupState = ctl.get();
+ if (lookupState == null) {
+ // First time this thread searches this reader:
+ lookupState = new PerThreadLookup(reader, uidField);
+ ctl.set(lookupState);
+ }
+
+ 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 {
+
+ private final LeafReaderContext[] readerContexts;
+ private final TermsEnum[] termsEnums;
+ private final DocsEnum[] docsEnums;
+ private final Bits[] liveDocs;
+ private final int numSegs;
+ private final boolean hasDeletions;
+
+ public PerThreadLookup(IndexReader r, String uidFieldName) throws IOException {
+
+ List<LeafReaderContext> leaves = new ArrayList<>(r.leaves());
+
+ readerContexts = leaves.toArray(new LeafReaderContext[leaves.size()]);
+ termsEnums = new TermsEnum[leaves.size()];
+ docsEnums = new DocsEnum[leaves.size()];
+ liveDocs = new Bits[leaves.size()];
+ int numSegs = 0;
+ boolean hasDeletions = false;
+
+ // iterate backwards to optimize for the frequently updated documents
+ // which are likely to be in the last segments
+ for(int i=leaves.size()-1;i>=0;i--) {
+ LeafReaderContext readerContext = leaves.get(i);
+ Fields fields = readerContext.reader().fields();
+ if (fields != null) {
+ Terms terms = fields.terms(uidFieldName);
+ if (terms != null) {
+ readerContexts[numSegs] = readerContext;
+ termsEnums[numSegs] = terms.iterator(null);
+ assert termsEnums[numSegs] != null;
+ liveDocs[numSegs] = readerContext.reader().getLiveDocs();
+ hasDeletions |= readerContext.reader().hasDeletions();
+ numSegs++;
+ }
+ }
+ }
+ this.numSegs = numSegs;
+ this.hasDeletions = hasDeletions;
+ }
+
+ /** Return true if id is found. */
+ 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);
+ int docID = docs.nextDoc();
+ if (docID != DocsEnum.NO_MORE_DOCS) {
+ assert docs.nextDoc() == DocsEnum.NO_MORE_DOCS;
+ return true;
+ } else {
+ assert hasDeletions;
+ }
+ }
+ }
+ return false;
+ }
+ }
+
+}
+
Added: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/NotUniqueException.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/NotUniqueException.java?rev=1637540&view=auto
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/NotUniqueException.java (added)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/NotUniqueException.java Sat Nov 8 11:05:05 2014
@@ -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.util.BytesRef;
+
+public class NotUniqueException extends IllegalArgumentException {
+ public NotUniqueException(String fieldName, BytesRef value) {
+ super("field \"" + fieldName + "\" must be unique, but value=" + value + " appears more than once");
+ }
+}
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=1637540&r1=1637539&r2=1637540&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 Sat Nov 8 11:05:05 2014
@@ -497,11 +497,13 @@ class ReadersAndUpdates {
}
// create new fields or update existing ones to have NumericDV type
for (String f : dvUpdates.numericDVUpdates.keySet()) {
- builder.addOrUpdate(f, NumericDocValuesField.TYPE);
+ FieldInfo fieldInfo = builder.addOrUpdate(f, NumericDocValuesField.TYPE);
+ fieldInfo.setDocValuesType(DocValuesType.NUMERIC);
}
// create new fields or update existing ones to have BinaryDV type
for (String f : dvUpdates.binaryDVUpdates.keySet()) {
- builder.addOrUpdate(f, BinaryDocValuesField.TYPE);
+ FieldInfo fieldInfo = builder.addOrUpdate(f, BinaryDocValuesField.TYPE);
+ fieldInfo.setDocValuesType(DocValuesType.BINARY);
}
fieldInfos = builder.finish();
Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java?rev=1637540&r1=1637539&r2=1637540&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java Sat Nov 8 11:05:05 2014
@@ -823,6 +823,7 @@ public final class SegmentInfos implemen
/** Clear all {@link SegmentCommitInfo}s. */
public void clear() {
segments.clear();
+ userData.clear();
}
/** Remove the provided {@link SegmentCommitInfo}.