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}.