You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by mi...@apache.org on 2015/01/02 13:02:33 UTC

svn commit: r1649007 [4/10] - in /lucene/dev/branches/lucene6005/lucene: analysis/common/src/test/org/apache/lucene/analysis/core/ analysis/common/src/test/org/apache/lucene/collation/ benchmark/src/java/org/apache/lucene/benchmark/byTask/tasks/ benchm...

Added: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/ReindexingReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/ReindexingReader.java?rev=1649007&view=auto
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/ReindexingReader.java (added)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/ReindexingReader.java Fri Jan  2 12:02:31 2015
@@ -0,0 +1,656 @@
+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.nio.file.DirectoryStream;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+
+import org.apache.lucene.document.FieldTypes;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.FSDirectory;
+import org.apache.lucene.store.IOContext;
+import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.StringHelper;
+
+// TODO: how to use FieldTypes to give a higher level API here?
+
+/** Provides a {@link ReaderManager} allowing you to also make "parallel"
+ *  changes to the previously indexed documents, e.g. turning stored
+ *  fields into doc values, changing norm values, etc.
+ *
+ *  <p>
+ *  This uses ParallelLeafReader to index new
+ *  stuff (postings, DVs, etc.) from previously stored fields, on the
+ *  fly (during NRT reader reopen), after the  initial indexing.  The
+ *  test indexes just a single stored field with text "content X" (X is
+ *  a number embedded in the text).
+ *
+ *  <p>
+ *  Then, on reopen, for any newly created segments (flush or merge), it
+ *  builds a new parallel segment by loading all stored docs, parsing
+ *  out that X, and adding it as DV and numeric indexed (trie) field.
+ *
+ *  <p>
+ *  Finally, for searching, it builds a top-level MultiReader, with
+ *  ParallelLeafReader for each segment, and then tests that random
+ *  numeric range queries, and sorting by the new DV field, work
+ *  correctly.
+ *
+ *  <p>
+ *  Each per-segment index lives in a private directory next to the main
+ *  index, and they are deleted once their segments are removed from the
+ *  index.  They are "volatile", meaning if e.g. the index is replicated to
+ *  another machine, it's OK to not copy parallel segments indices,
+ *  since they will just be regnerated (at a cost though). */
+public abstract class ReindexingReader implements Closeable {
+
+  private static boolean DEBUG = false;
+
+  /** Key used to store the current schema gen in the SegmentInfo diagnostics */
+  public final static String SCHEMA_GEN_KEY = "schema_gen";
+
+  public final IndexWriter w;
+  public final ReaderManager mgr;
+
+  // Main index directory:
+  public final Directory indexDir;
+
+  // Parent directory holding sub directory index for each segment + gen:
+  private final Path segsRootPath;
+
+  /** Which segments have been closed, but their parallel index is not yet not removed. */
+  private final Set<SegmentIDAndGen> closedSegments = Collections.newSetFromMap(new ConcurrentHashMap<SegmentIDAndGen,Boolean>());
+
+  /** Holds currently open parallel readers for each segment. */
+  private final Map<SegmentIDAndGen,LeafReader> parallelReaders = new ConcurrentHashMap<>();
+
+  void printRefCounts() {
+    System.out.println("All refCounts:");
+    for(Map.Entry<SegmentIDAndGen,LeafReader> ent : parallelReaders.entrySet()) {
+      System.out.println("  " + ent.getKey() + " " + ent.getValue() + " refCount=" + ent.getValue().getRefCount());
+    }
+  }
+
+  public ReindexingReader(Directory indexDir, Path segsRootPath) throws IOException {
+
+    this.indexDir = indexDir;
+
+    // Per-segment parallel indices are stored under subdirs "segs":
+    this.segsRootPath = segsRootPath;
+    Files.createDirectories(segsRootPath);
+
+    IndexWriterConfig iwc = getIndexWriterConfig();
+    iwc.setMergePolicy(new ReindexingMergePolicy(iwc.getMergePolicy()));
+    if (DEBUG) {
+      System.out.println("TEST: use IWC:\n" + iwc);
+    }
+    w = new IndexWriter(indexDir, iwc);
+
+    w.getConfig().setMergedSegmentWarmer(new IndexWriter.IndexReaderWarmer() {
+        @Override
+        public void warm(LeafReader reader) throws IOException {
+          // This will build the parallel index for the merged segment before the merge becomes visible, so reopen delay is only due to
+          // newly flushed segments:
+          if (DEBUG) System.out.println(Thread.currentThread().getName() +": TEST: now warm " + reader);
+          // TODO: it's not great that we pass false here; it means we close the reader & reopen again for NRT reader; still we did "warm" by
+          // building the parallel index, if necessary
+          getParallelLeafReader(reader, false, getCurrentSchemaGen());
+        }
+      });
+
+    // start with empty commit:
+    w.commit();
+    mgr = new ReaderManager(new ParallelLeafDirectoryReader(DirectoryReader.open(w, true)));
+    if (DEBUG) {
+      DirectoryReader r = mgr.acquire();
+      try {
+        System.out.println("TEST: ReindexingReader.init current=" + r);
+      } finally {
+        mgr.release(r);
+      }
+    }
+  }
+
+  protected abstract IndexWriterConfig getIndexWriterConfig() throws IOException;
+
+  /** Optional method to validate that the provided parallel reader in fact reflects the changes in schemaGen. */
+  protected void checkParallelReader(LeafReader reader, LeafReader parallelReader, long schemaGen) throws IOException {
+  }
+
+  /** Override to customize Directory impl. */
+  protected Directory openDirectory(Path path) throws IOException {
+    return FSDirectory.open(path);
+  }
+
+  public void commit() throws IOException {
+    w.commit();
+  }
+    
+  LeafReader getCurrentReader(LeafReader reader, long schemaGen) throws IOException {
+    LeafReader parallelReader = getParallelLeafReader(reader, true, schemaGen);
+    if (parallelReader != null) {
+
+      // We should not be embedding one ParallelLeafReader inside another:
+      assert parallelReader instanceof ParallelLeafReader == false;
+      assert reader instanceof ParallelLeafReader == false;
+
+      // NOTE: important that parallelReader is first, so if there are field name overlaps, because changes to the schema
+      // overwrote existing field names, it wins:
+      LeafReader newReader = new ParallelLeafReader(false, parallelReader, reader) {
+          @Override
+          public Bits getLiveDocs() {
+            return getParallelReaders()[1].getLiveDocs();
+          }
+          @Override
+          public int numDocs() {
+            return getParallelReaders()[1].numDocs();
+          }
+        };
+
+      // Because ParallelLeafReader does its own (extra) incRef:
+      parallelReader.decRef();
+
+      return newReader;
+
+    } else {
+      // This segment was already current as of currentSchemaGen:
+      return reader;
+    }
+  }
+
+  private class ParallelLeafDirectoryReader extends FilterDirectoryReader {
+    private final FieldTypes fieldTypes;
+    public ParallelLeafDirectoryReader(DirectoryReader in) {
+      super(in, new FilterDirectoryReader.SubReaderWrapper() {
+          final long currentSchemaGen = getCurrentSchemaGen();
+          @Override
+          public LeafReader wrap(LeafReader reader) {
+            try {
+              return getCurrentReader(reader, currentSchemaGen);
+            } catch (IOException ioe) {
+              // TODO: must close on exc here:
+              throw new RuntimeException(ioe);
+            }
+          }
+        });
+
+      // nocommit move this logic "up":
+      fieldTypes = new FieldTypes(in.getFieldTypes());
+      for(LeafReaderContext ctx : leaves()) {
+        LeafReader leafReader = ctx.reader();
+        if (leafReader instanceof ParallelLeafReader) {
+          fieldTypes.addAll(((ParallelLeafReader) leafReader).getParallelReaders()[0].getFieldTypes());
+        }
+      }
+    }
+      
+    @Override
+    public FieldTypes getFieldTypes() {
+      return fieldTypes;
+    }
+
+    @Override
+    protected DirectoryReader doWrapDirectoryReader(DirectoryReader in) {
+      return new ParallelLeafDirectoryReader(in);
+    }
+
+    @Override
+    protected void doClose() throws IOException {
+      Throwable firstExc = null;
+      for (final LeafReader r : getSequentialSubReaders()) {
+        if (r instanceof ParallelLeafReader) {
+          // try to close each reader, even if an exception is thrown
+          try {
+            r.decRef();
+          } catch (Throwable t) {
+            if (firstExc == null) {
+              firstExc = t;
+            }
+          }
+        }
+      }
+      // Also close in, so it decRef's the SegmentInfos
+      try {
+        in.doClose();
+      } catch (Throwable t) {
+        if (firstExc == null) {
+          firstExc = t;
+        }
+      }
+      // throw the first exception
+      IOUtils.reThrow(firstExc);
+    }
+  }
+
+  @Override
+  public void close() throws IOException {
+    w.close();
+    if (DEBUG) System.out.println("TEST: after close writer index=" + SegmentInfos.readLatestCommit(indexDir).toString(indexDir));
+
+    /*
+      DirectoryReader r = mgr.acquire();
+      try {
+      TestUtil.checkReader(r);
+      } finally {
+      mgr.release(r);
+      }
+    */
+    mgr.close();
+    pruneOldSegments(true);
+    assertNoExtraSegments();
+  }
+
+  // Make sure we deleted all parallel indices for segments that are no longer in the main index: 
+  private void assertNoExtraSegments() throws IOException {
+    Set<String> liveIDs = new HashSet<String>();
+    for(SegmentCommitInfo info : SegmentInfos.readLatestCommit(indexDir)) {
+      String idString = StringHelper.idToString(info.info.getId());
+      liveIDs.add(idString);
+    }
+
+    // At this point (closing) the only segments in closedSegments should be the still-live ones:
+    for(SegmentIDAndGen segIDGen : closedSegments) {
+      assert liveIDs.contains(segIDGen.segID);
+    }
+
+    boolean fail = false;
+    try (DirectoryStream<Path> stream = Files.newDirectoryStream(segsRootPath)) {
+        for (Path path : stream) {
+          SegmentIDAndGen segIDGen = new SegmentIDAndGen(path.getFileName().toString());
+          if (liveIDs.contains(segIDGen.segID) == false) {
+            if (DEBUG) System.out.println("TEST: fail seg=" + path.getFileName() + " is not live but still has a parallel index");
+            fail = true;
+          }
+        }
+      }
+    assert fail == false;
+  }
+
+  private static class SegmentIDAndGen {
+    public final String segID;
+    public final long schemaGen;
+
+    public SegmentIDAndGen(String segID, long schemaGen) {
+      this.segID = segID;
+      this.schemaGen = schemaGen;
+    }
+
+    public SegmentIDAndGen(String s) {
+      String[] parts = s.split("_");
+      if (parts.length != 2) {
+        throw new IllegalArgumentException("invalid SegmentIDAndGen \"" + s + "\"");
+      }
+      // TODO: better checking of segID?
+      segID = parts[0];
+      schemaGen = Long.parseLong(parts[1]);
+    }
+
+    @Override
+    public int hashCode() {
+      return (int) (segID.hashCode() * schemaGen);
+    }
+
+    @Override
+    public boolean equals(Object _other) {
+      if (_other instanceof SegmentIDAndGen) {
+        SegmentIDAndGen other = (SegmentIDAndGen) _other;
+        return segID.equals(other.segID) && schemaGen == other.schemaGen;
+      } else {
+        return false;
+      }
+    }
+
+    @Override
+    public String toString() {
+      return segID + "_" + schemaGen;
+    }
+  }
+
+  private class ParallelReaderClosed implements LeafReader.ReaderClosedListener {
+    private final SegmentIDAndGen segIDGen;
+    private final Directory dir;
+
+    public ParallelReaderClosed(SegmentIDAndGen segIDGen, Directory dir) {
+      this.segIDGen = segIDGen;
+      this.dir = dir;
+    }
+
+    @Override
+    public void onClose(IndexReader ignored) {
+      try {
+        // TODO: make this sync finer, i.e. just the segment + schemaGen
+        synchronized(ReindexingReader.this) {
+          if (DEBUG) System.out.println(Thread.currentThread().getName() + ": TEST: now close parallel parLeafReader dir=" + dir + " segIDGen=" + segIDGen);
+          parallelReaders.remove(segIDGen);
+          dir.close();
+          closedSegments.add(segIDGen);
+        }
+      } catch (IOException ioe) {
+        System.out.println("TEST: hit IOExc closing dir=" + dir);
+        ioe.printStackTrace(System.out);
+        throw new RuntimeException(ioe);
+      }
+    }
+  }
+
+  // Returns a ref
+  LeafReader getParallelLeafReader(final LeafReader leaf, boolean doCache, long schemaGen) throws IOException {
+    assert leaf instanceof SegmentReader;
+    SegmentInfo info = ((SegmentReader) leaf).getSegmentInfo().info;
+
+    long infoSchemaGen = getSchemaGen(info);
+
+    if (DEBUG) System.out.println(Thread.currentThread().getName() + ": TEST: getParallelLeafReader: " + leaf + " infoSchemaGen=" + infoSchemaGen + " vs schemaGen=" + schemaGen + " doCache=" + doCache);
+
+    if (infoSchemaGen == schemaGen) {
+      if (DEBUG) System.out.println(Thread.currentThread().getName()+ ": TEST: segment is already current schemaGen=" + schemaGen + "; skipping");
+      return null;
+    }
+
+    if (infoSchemaGen > schemaGen) {
+      throw new IllegalStateException("segment infoSchemaGen (" + infoSchemaGen + ") cannot be greater than requested schemaGen (" + schemaGen + ")");
+    }
+
+    final SegmentIDAndGen segIDGen = new SegmentIDAndGen(StringHelper.idToString(info.getId()), schemaGen);
+
+    // While loop because the parallel reader may be closed out from under us, so we must retry:
+    while (true) {
+
+      // TODO: make this sync finer, i.e. just the segment + schemaGen
+      synchronized (this) {
+        LeafReader parReader = parallelReaders.get(segIDGen);
+      
+        assert doCache || parReader == null;
+
+        if (parReader == null) {
+
+          Path leafIndex = segsRootPath.resolve(segIDGen.toString());
+
+          final Directory dir = openDirectory(leafIndex);
+
+          if (Files.exists(leafIndex.resolve("done")) == false) {
+            if (DEBUG) System.out.println(Thread.currentThread().getName() + ": TEST: build segment index for " + leaf + " " + segIDGen + " (source: " + info.getDiagnostics().get("source") + ") dir=" + leafIndex);
+
+            if (dir.listAll().length != 0) {
+              // It crashed before finishing last time:
+              if (DEBUG) System.out.println(Thread.currentThread().getName() + ": TEST: remove old incomplete index files: " + leafIndex);
+              IOUtils.rm(leafIndex);
+            }
+
+            reindex(infoSchemaGen, schemaGen, leaf, dir);
+
+            // Marker file, telling us this index is in fact done.  This way if we crash while doing the reindexing for a given segment, we will
+            // later try again:
+            dir.createOutput("done", IOContext.DEFAULT).close();
+          } else {
+            if (DEBUG) System.out.println(Thread.currentThread().getName() + ": TEST: segment index already exists for " + leaf + " " + segIDGen + " (source: " + info.getDiagnostics().get("source") + ") dir=" + leafIndex);
+          }
+
+          if (DEBUG) System.out.println(Thread.currentThread().getName() + ": TEST: now check index " + dir);
+          //TestUtil.checkIndex(dir);
+
+          SegmentInfos infos = SegmentInfos.readLatestCommit(dir);
+          final LeafReader parLeafReader;
+          if (infos.size() == 1) {
+            parLeafReader = new SegmentReader(FieldTypes.getFieldTypes(infos.getUserData(), null, null), infos.info(0), IOContext.DEFAULT);
+          } else {
+            // This just means we didn't forceMerge above:
+            parLeafReader = SlowCompositeReaderWrapper.wrap(DirectoryReader.open(dir));
+          }
+
+          //checkParallelReader(leaf, parLeafReader, schemaGen);
+
+          if (DEBUG) System.out.println(Thread.currentThread().getName() + ": TEST: opened parallel reader: " + parLeafReader);
+          if (doCache) {
+            parallelReaders.put(segIDGen, parLeafReader);
+
+            // Our id+gen could have been previously closed, e.g. if it was a merged segment that was warmed, so we must clear this else
+            // the pruning may remove our directory:
+            closedSegments.remove(segIDGen);
+
+            parLeafReader.addReaderClosedListener(new ParallelReaderClosed(segIDGen, dir));
+
+          } else {
+            // Used only for merged segment warming:
+            // Messy: we close this reader now, instead of leaving open for reuse:
+            if (DEBUG) System.out.println("TEST: now decRef non cached refCount=" + parLeafReader.getRefCount());
+            parLeafReader.decRef();
+            dir.close();
+
+            // Must do this after dir is closed, else another thread could "rm -rf" while we are closing (which makes MDW.close's
+            // checkIndex angry):
+            closedSegments.add(segIDGen);
+            parReader = null;
+          }
+          parReader = parLeafReader;
+
+        } else {
+          if (parReader.tryIncRef() == false) {
+            // We failed: this reader just got closed by another thread, e.g. refresh thread opening a new reader, so this reader is now
+            // closed and we must try again.
+            if (DEBUG) System.out.println(Thread.currentThread().getName()+ ": TEST: tryIncRef failed for " + parReader + "; retry");
+            parReader = null;
+            continue;
+          }
+          if (DEBUG) System.out.println(Thread.currentThread().getName()+ ": TEST: use existing already opened parReader=" + parReader + " refCount=" + parReader.getRefCount());
+          //checkParallelReader(leaf, parReader, schemaGen);
+        }
+
+        // We return the new reference to caller
+        return parReader;
+      }
+    }
+  }
+
+  // TODO: we could pass a writer already opened...?
+  protected abstract void reindex(long oldSchemaGen, long newSchemaGen, LeafReader reader, Directory parallelDir) throws IOException;
+
+  /** Returns the gen for the current schema. */
+  protected abstract long getCurrentSchemaGen();
+
+  /** Returns the gen that should be merged, meaning those changes will be folded back into the main index. */
+  protected long getMergingSchemaGen() {
+    return getCurrentSchemaGen();
+  }
+
+  /** Removes the parallel index that are no longer in the last commit point.  We can't
+   *  remove this when the parallel reader is closed because it may still be referenced by
+   *  the last commit. */
+  private void pruneOldSegments(boolean removeOldGens) throws IOException {
+    SegmentInfos lastCommit = SegmentInfos.readLatestCommit(indexDir);
+    if (DEBUG) System.out.println("TEST: prune; removeOldGens=" + removeOldGens);
+
+    Set<String> liveIDs = new HashSet<String>();
+    for(SegmentCommitInfo info : lastCommit) {
+      String idString = StringHelper.idToString(info.info.getId());
+      liveIDs.add(idString);
+      if (DEBUG) System.out.println("TEST: live id " + idString + " seg=" + info);
+    }
+
+    long currentSchemaGen = getCurrentSchemaGen();
+
+    if (Files.exists(segsRootPath)) {
+      try (DirectoryStream<Path> stream = Files.newDirectoryStream(segsRootPath)) {
+          for (Path path : stream) {
+            if (Files.isDirectory(path)) {
+              SegmentIDAndGen segIDGen = new SegmentIDAndGen(path.getFileName().toString());
+              assert segIDGen.schemaGen <= currentSchemaGen;
+              if (DEBUG) System.out.println("TEST: check dir=" + path + " live?=" + (liveIDs.contains(segIDGen.segID)) + " closed=" + (closedSegments.contains(segIDGen)) + " currentSchemaGen=" + currentSchemaGen);
+              if (liveIDs.contains(segIDGen.segID) == false && (closedSegments.contains(segIDGen) || (removeOldGens && segIDGen.schemaGen < currentSchemaGen))) {
+                if (DEBUG) System.out.println("TEST: remove " + segIDGen);
+                try {
+                  IOUtils.rm(path);
+                  closedSegments.remove(segIDGen);
+                } catch (IOException ioe) {
+                  // OK, we'll retry later
+                  if (DEBUG) System.out.println("TEST: ignore ioe during delete " + path + ":" + ioe);
+                }
+              }
+            }
+          }
+        }
+    }
+  }
+
+  /** Just replaces the sub-readers with parallel readers, so reindexed fields are merged into new segments. */
+  private class ReindexingMergePolicy extends MergePolicy {
+
+    class ReindexingOneMerge extends OneMerge {
+
+      List<LeafReader> parallelReaders;
+      final long schemaGen;
+
+      ReindexingOneMerge(List<SegmentCommitInfo> segments) {
+        super(segments);
+        // Commit up front to which schemaGen we will merge; we don't want a schema change sneaking in for some of our leaf readers but not others:
+        schemaGen = getMergingSchemaGen();
+        long currentSchemaGen = getCurrentSchemaGen();
+
+        // Defensive sanity check:
+        if (schemaGen > currentSchemaGen) {
+          throw new IllegalStateException("currentSchemaGen (" + currentSchemaGen + ") must always be >= mergingSchemaGen (" + schemaGen + ")");
+        }
+      }
+
+      @Override
+      public List<LeafReader> getMergeReaders() throws IOException {
+        if (parallelReaders == null) {
+          parallelReaders = new ArrayList<>();
+          for (LeafReader reader : super.getMergeReaders()) {
+            parallelReaders.add(getCurrentReader(reader, schemaGen));
+          }
+        }
+
+        return parallelReaders;
+      }
+
+      @Override
+      public void mergeFinished() throws IOException {
+        Throwable th = null;
+        for(LeafReader r : parallelReaders) {
+          if (r instanceof ParallelLeafReader) {
+            try {
+              r.decRef();
+            } catch (Throwable t) {
+              if (th == null) {
+                th = t;
+              }
+            }
+          }
+        }
+
+        // If any error occured, throw it.
+        IOUtils.reThrow(th);
+      }
+    
+      @Override
+      public void setInfo(SegmentCommitInfo info) {
+        // Record that this merged segment is current as of this schemaGen:
+        info.info.getDiagnostics().put(SCHEMA_GEN_KEY, Long.toString(schemaGen));
+        super.setInfo(info);
+      }
+
+      @Override
+      public MergePolicy.DocMap getDocMap(final MergeState mergeState) {
+        return super.getDocMap(mergeState);
+      }
+    }
+
+    class ReindexingMergeSpecification extends MergeSpecification {
+      @Override
+      public void add(OneMerge merge) {
+        super.add(new ReindexingOneMerge(merge.segments));
+      }
+
+      @Override
+      public String segString(Directory dir) {
+        return "ReindexingMergeSpec(" + super.segString(dir) + ")";
+      }
+    }
+
+    MergeSpecification wrap(MergeSpecification spec) {
+      MergeSpecification wrapped = null;
+      if (spec != null) {
+        wrapped = new ReindexingMergeSpecification();
+        for (OneMerge merge : spec.merges) {
+          wrapped.add(merge);
+        }
+      }
+      return wrapped;
+    }
+
+    final MergePolicy in;
+
+    /** Create a new {@code MergePolicy} that sorts documents with the given {@code sort}. */
+    public ReindexingMergePolicy(MergePolicy in) {
+      this.in = in;
+    }
+
+    @Override
+    public MergeSpecification findMerges(MergeTrigger mergeTrigger,
+                                         SegmentInfos segmentInfos, IndexWriter writer) throws IOException {
+      return wrap(in.findMerges(mergeTrigger, segmentInfos, writer));
+    }
+
+    @Override
+    public MergeSpecification findForcedMerges(SegmentInfos segmentInfos,
+                                               int maxSegmentCount, Map<SegmentCommitInfo,Boolean> segmentsToMerge, IndexWriter writer)
+      throws IOException {
+      // TODO: do we need to force-force this?  Ie, wrapped MP may think index is already optimized, yet maybe its schemaGen is old?  need test!
+      return wrap(in.findForcedMerges(segmentInfos, maxSegmentCount, segmentsToMerge, writer));
+    }
+
+    @Override
+    public MergeSpecification findForcedDeletesMerges(SegmentInfos segmentInfos, IndexWriter writer)
+      throws IOException {
+      return wrap(in.findForcedDeletesMerges(segmentInfos, writer));
+    }
+
+    @Override
+    public boolean useCompoundFile(SegmentInfos segments,
+                                   SegmentCommitInfo newSegment, IndexWriter writer) throws IOException {
+      return in.useCompoundFile(segments, newSegment, writer);
+    }
+
+    @Override
+    public String toString() {
+      return "ReindexingMergePolicy(" + in + ")";
+    }
+  }
+
+  static long getSchemaGen(SegmentInfo info) {
+    String s = info.getDiagnostics().get(SCHEMA_GEN_KEY);
+    if (s == null) {
+      return -1;
+    } else {
+      return Long.parseLong(s);
+    }
+  }
+}
+

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/StandardDirectoryReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/StandardDirectoryReader.java?rev=1649007&r1=1649006&r2=1649007&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/StandardDirectoryReader.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/StandardDirectoryReader.java Fri Jan  2 12:02:31 2015
@@ -59,7 +59,6 @@ final class StandardDirectoryReader exte
       @Override
       protected DirectoryReader doBody(String segmentFileName) throws IOException {
         SegmentInfos sis = SegmentInfos.readCommit(directory, segmentFileName);
-        // nocommit fixme: can't be passing null analyzers
         FieldTypes fieldTypes = FieldTypes.getFieldTypes(sis.getUserData(), null, null);
         final SegmentReader[] readers = new SegmentReader[sis.size()];
         for (int i = sis.size()-1; i >= 0; i--) {
@@ -93,7 +92,6 @@ final class StandardDirectoryReader exte
     // Carry over current schema:
     segmentInfos.getUserData().put(FieldTypes.FIELD_TYPES_KEY, writer.fieldTypes.writeToString());
 
-    // nocommit fixme: can't be passing null analyzers
     FieldTypes fieldTypes = FieldTypes.getFieldTypes(segmentInfos.getUserData(), null, null);
 
     int infosUpto = 0;
@@ -108,7 +106,6 @@ final class StandardDirectoryReader exte
         assert info.info.dir == dir;
         final ReadersAndUpdates rld = writer.readerPool.get(info, true);
         try {
-          // nocommit this has writer's fieldTypes, not ours:
           final SegmentReader reader = rld.getReadOnlyClone(IOContext.READ);
           if (reader.numDocs() > 0 || writer.getKeepFullyDeletedSegments()) {
             // Steal the ref:
@@ -161,7 +158,6 @@ final class StandardDirectoryReader exte
     
     SegmentReader[] newReaders = new SegmentReader[infos.size()];
 
-    // nocommit fixme: can't be passing null analyzers
     FieldTypes fieldTypes = FieldTypes.getFieldTypes(infos.getUserData(), null, null);
 
     for (int i = infos.size() - 1; i>=0; i--) {

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/StoredFieldVisitor.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/StoredFieldVisitor.java?rev=1649007&r1=1649006&r2=1649007&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/StoredFieldVisitor.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/StoredFieldVisitor.java Fri Jan  2 12:02:31 2015
@@ -56,6 +56,9 @@ public abstract class StoredFieldVisitor
   public void stringField(FieldInfo fieldInfo, String value) throws IOException {
   }
 
+  // TODO: simplify this API; stored fields can just deal with long and FieldTypes instead of having to implement its own
+  // per-document-per-field baby schema here:
+
   /** Process a int numeric field. */
   public void intField(FieldInfo fieldInfo, int value) throws IOException {
   }

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/TermVectorsConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/TermVectorsConsumer.java?rev=1649007&r1=1649006&r2=1649007&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/TermVectorsConsumer.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/TermVectorsConsumer.java Fri Jan  2 12:02:31 2015
@@ -141,8 +141,8 @@ final class TermVectorsConsumer extends
   }
 
   @Override
-  public TermsHashPerField addField(FieldInvertState invertState, FieldInfo fieldInfo, boolean rightJustifyTerms) {
-    return new TermVectorsConsumerPerField(invertState, this, fieldInfo, rightJustifyTerms);
+  public TermsHashPerField addField(FieldInvertState invertState, FieldInfo fieldInfo) {
+    return new TermVectorsConsumerPerField(invertState, this, fieldInfo);
   }
 
   void addFieldToFlush(TermVectorsConsumerPerField fieldToFlush) {

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/TermVectorsConsumerPerField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/TermVectorsConsumerPerField.java?rev=1649007&r1=1649006&r2=1649007&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/TermVectorsConsumerPerField.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/TermVectorsConsumerPerField.java Fri Jan  2 12:02:31 2015
@@ -39,10 +39,9 @@ final class TermVectorsConsumerPerField
   OffsetAttribute offsetAttribute;
   PayloadAttribute payloadAttribute;
   boolean hasPayloads; // if enabled, and we actually saw any for this field
-  byte[] scratchBytes;
 
-  public TermVectorsConsumerPerField(FieldInvertState invertState, TermVectorsConsumer termsWriter, FieldInfo fieldInfo, boolean rightJustifyTerms) {
-    super(2, invertState, termsWriter, null, fieldInfo, rightJustifyTerms);
+  public TermVectorsConsumerPerField(FieldInvertState invertState, TermVectorsConsumer termsWriter, FieldInfo fieldInfo) {
+    super(2, invertState, termsWriter, null, fieldInfo);
     this.termsWriter = termsWriter;
   }
 
@@ -57,24 +56,6 @@ final class TermVectorsConsumerPerField
     termsWriter.addFieldToFlush(this);
   }
 
-  private void maybeLeftZeroPad(BytesRef flushTerm) {
-    if (rightJustifyTerms) {
-      // nocommit need to fix checkIndex to deal w/ this properly (it will be angry that sometimes term vectors terms are not left-padded
-      // "enough") since we may not have seen the max term yet when we write this doc:
-
-      // nocommit we could make this a "per document max term" instead?
-      int prefix = maxTermLength - flushTerm.length;
-      assert prefix >= 0;
-      for(int i=0;i<prefix;i++) {
-        scratchBytes[i] = 0;
-      }
-      System.arraycopy(flushTerm.bytes, flushTerm.offset, scratchBytes, prefix, flushTerm.length);
-      flushTerm.bytes = scratchBytes;
-      flushTerm.offset = 0;
-      flushTerm.length = maxTermLength;
-    }
-  }
-
   void finishDocument() throws IOException {
     if (doVectors == false) {
       return;
@@ -82,10 +63,6 @@ final class TermVectorsConsumerPerField
 
     doVectors = false;
 
-    if (rightJustifyTerms && (scratchBytes == null || scratchBytes.length < maxTermLength)) {
-      scratchBytes = new byte[maxTermLength];
-    }
-
     final int numPostings = bytesHash.size();
 
     final BytesRef flushTerm = termsWriter.flushTerm;
@@ -112,7 +89,6 @@ final class TermVectorsConsumerPerField
 
       // Get BytesRef
       termBytePool.setBytesRef(flushTerm, postings.textStarts[termID]);
-      maybeLeftZeroPad(flushTerm);
 
       tv.startTerm(flushTerm, freq);
       

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/TermsHash.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/TermsHash.java?rev=1649007&r1=1649006&r2=1649007&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/TermsHash.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/TermsHash.java Fri Jan  2 12:02:31 2015
@@ -86,7 +86,7 @@ abstract class TermsHash {
     }
   }
 
-  abstract TermsHashPerField addField(FieldInvertState fieldInvertState, FieldInfo fieldInfo, boolean rightJustifyTerms);
+  abstract TermsHashPerField addField(FieldInvertState fieldInvertState, FieldInfo fieldInfo);
 
   void finishDocument() throws IOException {
     if (nextTermsHash != null) {

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/TermsHashPerField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/TermsHashPerField.java?rev=1649007&r1=1649006&r2=1649007&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/TermsHashPerField.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/index/TermsHashPerField.java Fri Jan  2 12:02:31 2015
@@ -55,12 +55,11 @@ abstract class TermsHashPerField impleme
 
   ParallelPostingsArray postingsArray;
   private final Counter bytesUsed;
-  final boolean rightJustifyTerms;
 
   /** streamCount: how many streams this field stores per term.
    * E.g. doc(+freq) is 1 stream, prox+offset is a second. */
 
-  public TermsHashPerField(int streamCount, FieldInvertState fieldState, TermsHash termsHash, TermsHashPerField nextPerField, FieldInfo fieldInfo, boolean rightJustifyTerms) {
+  public TermsHashPerField(int streamCount, FieldInvertState fieldState, TermsHash termsHash, TermsHashPerField nextPerField, FieldInfo fieldInfo) {
     intPool = termsHash.intPool;
     bytePool = termsHash.bytePool;
     termBytePool = termsHash.termBytePool;
@@ -72,7 +71,6 @@ abstract class TermsHashPerField impleme
     numPostingInt = 2*streamCount;
     this.fieldInfo = fieldInfo;
     this.nextPerField = nextPerField;
-    this.rightJustifyTerms = rightJustifyTerms;
     PostingsBytesStartArray byteStarts = new PostingsBytesStartArray(this, bytesUsed);
     bytesHash = new BytesRefHash(termBytePool, HASH_INIT_SIZE, byteStarts);
   }
@@ -99,14 +97,7 @@ abstract class TermsHashPerField impleme
   /** Collapse the hash table and sort in-place; also sets
    * this.sortedTermIDs to the results */
   public int[] sortPostings() {
-    Comparator<BytesRef> cmp;
-    if (rightJustifyTerms) {
-      // Sort by a comparator as if terms were right justified:
-      cmp = BytesRef.getRightJustifiedComparator();
-    } else {
-      cmp = BytesRef.getUTF8SortedAsUnicodeComparator();
-    }
-    sortedTermIDs = bytesHash.sort(cmp);
+    sortedTermIDs = bytesHash.sort(BytesRef.getUTF8SortedAsUnicodeComparator());
     return sortedTermIDs;
   }
 
@@ -169,12 +160,6 @@ abstract class TermsHashPerField impleme
 
     if (termID >= 0) {// New posting
       maxTermLength = Math.max(maxTermLength, termBytesRef.length);
-      /*
-      if (rightJustifyTerms && termBytesRef.length > 0 && termBytesRef.bytes[termBytesRef.offset] == 0) {
-        // nocommit catch this earlier so it's not an aborting exception?
-        throw new IllegalArgumentException("left-zero-pad fields cannot lead with 0 bytes");
-      }
-      */
       bytesHash.byteStart(termID);
       // Init stream slices
       if (numPostingInt + intPool.intUpto > IntBlockPool.INT_BLOCK_SIZE) {

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/DocIdSet.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/DocIdSet.java?rev=1649007&r1=1649006&r2=1649007&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/DocIdSet.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/DocIdSet.java Fri Jan  2 12:02:31 2015
@@ -53,6 +53,32 @@ public abstract class DocIdSet implement
     }
   };
 
+  /** A full {@code DocIdSet} instance (matches all docs). */
+  // nocommit does this already exist somewhere?
+  public static DocIdSet full(final int maxDoc) {
+    return new DocIdSet() {
+      @Override
+      public DocIdSetIterator iterator() {
+        return DocIdSetIterator.full(maxDoc);
+      }
+    
+      @Override
+      public boolean isCacheable() {
+        return true;
+      }
+    
+      @Override
+      public Bits bits() {
+        return new Bits.MatchAllBits(maxDoc);
+      }
+
+      @Override
+      public long ramBytesUsed() {
+        return 0L;
+      }
+    };
+  };
+
   /** Provides a {@link DocIdSetIterator} to access the set.
    * This implementation can return <code>null</code> if there
    * are no docs that match. */

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/DocIdSetIterator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/DocIdSetIterator.java?rev=1649007&r1=1649006&r2=1649007&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/DocIdSetIterator.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/DocIdSetIterator.java Fri Jan  2 12:02:31 2015
@@ -58,6 +58,44 @@ public abstract class DocIdSetIterator {
       }
     };
   }
+
+  // nocommit does this already exist somewhere?  it's silly...
+  /** A full {@code DocIdSetIterator} instance */
+  public static final DocIdSetIterator full(final int maxDoc) {
+    return new DocIdSetIterator() {
+      int docID = -1;
+      
+      @Override
+      public int advance(int target) {
+        assert target >= 0;
+        if (target >= maxDoc) {
+          docID = NO_MORE_DOCS;
+        } else {
+          docID = target;
+        }
+        return docID;
+      }
+      
+      @Override
+      public int docID() {
+        return docID;
+      }
+
+      @Override
+      public int nextDoc() {
+        docID++;
+        if (docID >= maxDoc) {
+          docID = NO_MORE_DOCS;
+        }
+        return docID;
+      }
+      
+      @Override
+      public long cost() {
+        return 0;
+      }
+    };
+  }
   
   /**
    * When returned by {@link #nextDoc()}, {@link #advance(int)} and

Added: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/DoubleComparator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/DoubleComparator.java?rev=1649007&view=auto
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/DoubleComparator.java (added)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/DoubleComparator.java Fri Jan  2 12:02:31 2015
@@ -0,0 +1,44 @@
+package org.apache.lucene.search;
+
+import org.apache.lucene.util.NumericUtils;
+
+/*
+ * 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.
+ */
+
+/** Parses field's values as double (using {@link
+ *  org.apache.lucene.index.LeafReader#getNumericDocValues} and sorts by ascending value */
+public class DoubleComparator extends NumericComparator<Double> {
+
+  /** 
+   * Creates a new comparator based on {@link Double#compare} for {@code numHits}.
+   * When a document has no value for the field, {@code missingValue} is substituted. 
+   */
+  public DoubleComparator(int numHits, String field, Double missingValue) {
+    super(numHits, field, NumericUtils.doubleToLong(missingValue));
+  }
+
+  @Override
+  protected Double longToValue(long value) {
+    return NumericUtils.longToDouble(value);
+  }
+
+  @Override
+  protected long valueToLong(Double value) {
+    return NumericUtils.doubleToLong(value);
+  }
+}
+

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/FieldComparator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/FieldComparator.java?rev=1649007&r1=1649006&r2=1649007&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/FieldComparator.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/FieldComparator.java Fri Jan  2 12:02:31 2015
@@ -220,332 +220,6 @@ public abstract class FieldComparator<T>
     }
   }
 
-  /**
-   * Base FieldComparator class for numeric types
-   */
-  public static abstract class NumericComparator<T extends Number> extends FieldComparator<T> {
-    protected final T missingValue;
-    protected final String field;
-    protected Bits docsWithField;
-    protected NumericDocValues currentReaderValues;
-    
-    public NumericComparator(String field, T missingValue) {
-      this.field = field;
-      this.missingValue = missingValue;
-    }
-
-    @Override
-    public FieldComparator<T> setNextReader(LeafReaderContext context) throws IOException {
-      currentReaderValues = getNumericDocValues(context, field);
-      if (missingValue != null) {
-        docsWithField = DocValues.getDocsWithField(context.reader(), field);
-        // optimization to remove unneeded checks on the bit interface:
-        if (docsWithField instanceof Bits.MatchAllBits) {
-          docsWithField = null;
-        }
-      } else {
-        docsWithField = null;
-      }
-      return this;
-    }
-    
-    /** Retrieves the NumericDocValues for the field in this segment */
-    protected NumericDocValues getNumericDocValues(LeafReaderContext context, String field) throws IOException {
-      return DocValues.getNumeric(context.reader(), field);
-    }
-  }
-
-  /** Parses field's values as double (using {@link
-   *  org.apache.lucene.index.LeafReader#getNumericDocValues} and sorts by ascending value */
-  public static class DoubleComparator extends NumericComparator<Double> {
-    private final double[] values;
-    private double bottom;
-    private double topValue;
-
-    /** 
-     * Creates a new comparator based on {@link Double#compare} for {@code numHits}.
-     * When a document has no value for the field, {@code missingValue} is substituted. 
-     */
-    public DoubleComparator(int numHits, String field, Double missingValue) {
-      super(field, missingValue);
-      values = new double[numHits];
-    }
-
-    @Override
-    public int compare(int slot1, int slot2) {
-      return Double.compare(values[slot1], values[slot2]);
-    }
-
-    @Override
-    public int compareBottom(int doc) {
-      double v2 = NumericUtils.longToDouble(currentReaderValues.get(doc));
-      // Test for v2 == 0 to save Bits.get method call for
-      // the common case (doc has value and value is non-zero):
-      if (docsWithField != null && v2 == 0 && !docsWithField.get(doc)) {
-        v2 = missingValue;
-      }
-
-      return Double.compare(bottom, v2);
-    }
-
-    @Override
-    public void copy(int slot, int doc) {
-      double v2 = NumericUtils.longToDouble(currentReaderValues.get(doc));
-      // Test for v2 == 0 to save Bits.get method call for
-      // the common case (doc has value and value is non-zero):
-      if (docsWithField != null && v2 == 0 && !docsWithField.get(doc)) {
-        v2 = missingValue;
-      }
-
-      values[slot] = v2;
-    }
-    
-    @Override
-    public void setBottom(final int bottom) {
-      this.bottom = values[bottom];
-    }
-
-    @Override
-    public void setTopValue(Double value) {
-      topValue = value;
-    }
-
-    @Override
-    public Double value(int slot) {
-      return Double.valueOf(values[slot]);
-    }
-
-    @Override
-    public int compareTop(int doc) {
-      double docValue = NumericUtils.longToDouble(currentReaderValues.get(doc));
-      // Test for docValue == 0 to save Bits.get method call for
-      // the common case (doc has value and value is non-zero):
-      if (docsWithField != null && docValue == 0 && !docsWithField.get(doc)) {
-        docValue = missingValue;
-      }
-      return Double.compare(topValue, docValue);
-    }
-  }
-
-  /** Parses field's values as float (using {@link
-   *  org.apache.lucene.index.LeafReader#getNumericDocValues(String)} and sorts by ascending value */
-  public static class FloatComparator extends NumericComparator<Float> {
-    private final float[] values;
-    private float bottom;
-    private float topValue;
-
-    /** 
-     * Creates a new comparator based on {@link Float#compare} for {@code numHits}.
-     * When a document has no value for the field, {@code missingValue} is substituted. 
-     */
-    public FloatComparator(int numHits, String field, Float missingValue) {
-      super(field, missingValue);
-      values = new float[numHits];
-    }
-    
-    @Override
-    public int compare(int slot1, int slot2) {
-      return Float.compare(values[slot1], values[slot2]);
-    }
-
-    @Override
-    public int compareBottom(int doc) {
-      // TODO: are there sneaky non-branch ways to compute sign of float?
-      float v2 = NumericUtils.intToFloat((int)currentReaderValues.get(doc));
-      // Test for v2 == 0 to save Bits.get method call for
-      // the common case (doc has value and value is non-zero):
-      if (docsWithField != null && v2 == 0 && !docsWithField.get(doc)) {
-        v2 = missingValue;
-      }
-
-      return Float.compare(bottom, v2);
-    }
-
-    @Override
-    public void copy(int slot, int doc) {
-      float v2 =  NumericUtils.intToFloat((int)currentReaderValues.get(doc));
-      // Test for v2 == 0 to save Bits.get method call for
-      // the common case (doc has value and value is non-zero):
-      if (docsWithField != null && v2 == 0 && !docsWithField.get(doc)) {
-        v2 = missingValue;
-      }
-
-      values[slot] = v2;
-    }
-    
-    @Override
-    public void setBottom(final int bottom) {
-      this.bottom = values[bottom];
-    }
-
-    @Override
-    public void setTopValue(Float value) {
-      topValue = value;
-    }
-
-    @Override
-    public Float value(int slot) {
-      return Float.valueOf(values[slot]);
-    }
-
-    @Override
-    public int compareTop(int doc) {
-      float docValue = NumericUtils.intToFloat((int)currentReaderValues.get(doc));
-      // Test for docValue == 0 to save Bits.get method call for
-      // the common case (doc has value and value is non-zero):
-      if (docsWithField != null && docValue == 0 && !docsWithField.get(doc)) {
-        docValue = missingValue;
-      }
-      return Float.compare(topValue, docValue);
-    }
-  }
-
-  /** Parses field's values as int (using {@link
-   *  org.apache.lucene.index.LeafReader#getNumericDocValues(String)} and sorts by ascending value */
-  public static class IntComparator extends NumericComparator<Integer> {
-    private final int[] values;
-    private int bottom;                           // Value of bottom of queue
-    private int topValue;
-
-    /** 
-     * Creates a new comparator based on {@link Integer#compare} for {@code numHits}.
-     * When a document has no value for the field, {@code missingValue} is substituted. 
-     */
-    public IntComparator(int numHits, String field, Integer missingValue) {
-      super(field, missingValue);
-      values = new int[numHits];
-    }
-        
-    @Override
-    public int compare(int slot1, int slot2) {
-      return Integer.compare(values[slot1], values[slot2]);
-    }
-
-    @Override
-    public int compareBottom(int doc) {
-      int v2 = (int) currentReaderValues.get(doc);
-      // Test for v2 == 0 to save Bits.get method call for
-      // the common case (doc has value and value is non-zero):
-      if (docsWithField != null && v2 == 0 && !docsWithField.get(doc)) {
-        v2 = missingValue;
-      }
-
-      return Integer.compare(bottom, v2);
-    }
-
-    @Override
-    public void copy(int slot, int doc) {
-      int v2 = (int) currentReaderValues.get(doc);
-      // Test for v2 == 0 to save Bits.get method call for
-      // the common case (doc has value and value is non-zero):
-      if (docsWithField != null && v2 == 0 && !docsWithField.get(doc)) {
-        v2 = missingValue;
-      }
-
-      values[slot] = v2;
-    }
-    
-    @Override
-    public void setBottom(final int bottom) {
-      this.bottom = values[bottom];
-    }
-
-    @Override
-    public void setTopValue(Integer value) {
-      topValue = value;
-    }
-
-    @Override
-    public Integer value(int slot) {
-      return Integer.valueOf(values[slot]);
-    }
-
-    @Override
-    public int compareTop(int doc) {
-      int docValue = (int) currentReaderValues.get(doc);
-      // Test for docValue == 0 to save Bits.get method call for
-      // the common case (doc has value and value is non-zero):
-      if (docsWithField != null && docValue == 0 && !docsWithField.get(doc)) {
-        docValue = missingValue;
-      }
-      return Integer.compare(topValue, docValue);
-    }
-  }
-
-  /** Parses field's values as long (using {@link
-   *  org.apache.lucene.index.LeafReader#getNumericDocValues(String)} and sorts by ascending value */
-  public static class LongComparator extends NumericComparator<Long> {
-    private final long[] values;
-    private long bottom;
-    private long topValue;
-
-    /** 
-     * Creates a new comparator based on {@link Long#compare} for {@code numHits}.
-     * When a document has no value for the field, {@code missingValue} is substituted. 
-     */
-    public LongComparator(int numHits, String field, Long missingValue) {
-      super(field, missingValue);
-      values = new long[numHits];
-    }
-
-    @Override
-    public int compare(int slot1, int slot2) {
-      return Long.compare(values[slot1], values[slot2]);
-    }
-
-    @Override
-    public int compareBottom(int doc) {
-      // TODO: there are sneaky non-branch ways to compute
-      // -1/+1/0 sign
-      long v2 = currentReaderValues.get(doc);
-      // Test for v2 == 0 to save Bits.get method call for
-      // the common case (doc has value and value is non-zero):
-      if (docsWithField != null && v2 == 0 && !docsWithField.get(doc)) {
-        v2 = missingValue;
-      }
-
-      return Long.compare(bottom, v2);
-    }
-
-    @Override
-    public void copy(int slot, int doc) {
-      long v2 = currentReaderValues.get(doc);
-      // Test for v2 == 0 to save Bits.get method call for
-      // the common case (doc has value and value is non-zero):
-      if (docsWithField != null && v2 == 0 && !docsWithField.get(doc)) {
-        v2 = missingValue;
-      }
-
-      values[slot] = v2;
-    }
-    
-    @Override
-    public void setBottom(final int bottom) {
-      this.bottom = values[bottom];
-    }
-
-    @Override
-    public void setTopValue(Long value) {
-      topValue = value;
-    }
-
-    @Override
-    public Long value(int slot) {
-      return Long.valueOf(values[slot]);
-    }
-
-    @Override
-    public int compareTop(int doc) {
-      long docValue = currentReaderValues.get(doc);
-      // Test for docValue == 0 to save Bits.get method call for
-      // the common case (doc has value and value is non-zero):
-      if (docsWithField != null && docValue == 0 && !docsWithField.get(doc)) {
-        docValue = missingValue;
-      }
-      return Long.compare(topValue, docValue);
-    }
-  }
-
   /** Sorts by descending relevance.  NOTE: if you are
    *  sorting only by descending relevance and then
    *  secondarily by ascending docID, performance is faster

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/FieldValueHitQueue.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/FieldValueHitQueue.java?rev=1649007&r1=1649006&r2=1649007&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/FieldValueHitQueue.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/FieldValueHitQueue.java Fri Jan  2 12:02:31 2015
@@ -60,7 +60,7 @@ public abstract class FieldValueHitQueue
       super(fields, size);
 
       SortField field = fields[0];
-      setComparator(0,field.getComparator(size, 0));
+      setComparator(0, field.getComparator(size, 0));
       oneReverseMul = field.reverse ? -1 : 1;
 
       reverseMul[0] = oneReverseMul;

Added: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/FloatComparator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/FloatComparator.java?rev=1649007&view=auto
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/FloatComparator.java (added)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/FloatComparator.java Fri Jan  2 12:02:31 2015
@@ -0,0 +1,44 @@
+package org.apache.lucene.search;
+
+import org.apache.lucene.util.NumericUtils;
+
+/*
+ * 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.
+ */
+
+/** Parses field's values as float (using {@link
+ *  org.apache.lucene.index.LeafReader#getNumericDocValues(String)} and sorts by ascending value */
+public class FloatComparator extends NumericComparator<Float> {
+
+  /** 
+   * Creates a new comparator based on {@link Float#compare} for {@code numHits}.
+   * When a document has no value for the field, {@code missingValue} is substituted. 
+   */
+  public FloatComparator(int numHits, String field, Float missingValue) {
+    super(numHits, field, NumericUtils.floatToInt(missingValue));
+  }
+
+  @Override
+  protected Float longToValue(long value) {
+    return NumericUtils.intToFloat((int) value);
+  }
+
+  @Override
+  protected long valueToLong(Float value) {
+    return NumericUtils.floatToInt(value);
+  }
+}
+

Copied: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/HalfFloatComparator.java (from r1644463, lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/document/HalfFloatComparator.java)
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/HalfFloatComparator.java?p2=lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/HalfFloatComparator.java&p1=lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/document/HalfFloatComparator.java&r1=1644463&r2=1649007&rev=1649007&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/document/HalfFloatComparator.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/HalfFloatComparator.java Fri Jan  2 12:02:31 2015
@@ -1,4 +1,4 @@
-package org.apache.lucene.document;
+package org.apache.lucene.search;
 
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
@@ -21,76 +21,23 @@ import org.apache.lucene.search.FieldCom
 import org.apache.lucene.util.HalfFloat;
 import org.apache.lucene.util.NumericUtils;
 
-// nocommit can't we do all numeric comparators this way?  if we fix all numeric dv to write sortable versions?
-class HalfFloatComparator extends FieldComparator.NumericComparator<Float> {
-  private final short[] values;
-  private final short missingShortValue;
-  private short bottom;
-  private short topValue;
+public class HalfFloatComparator extends NumericComparator<Float> {
 
   /** 
    * Creates a new comparator based on {@link Float#compare} for {@code numHits}.
    * When a document has no value for the field, {@code missingValue} is substituted. 
    */
-  public HalfFloatComparator(int numHits, String field, Float missingValue) {
-    super(field, missingValue);
-    values = new short[numHits];
-    missingShortValue = (short) NumericUtils.sortableHalfFloatBits(HalfFloat.floatToShortBits(missingValue));
+  public HalfFloatComparator(int numHits, String field, float missingValue) {
+    super(numHits, field, NumericUtils.halfFloatToShort(missingValue));
   }
     
   @Override
-  public int compare(int slot1, int slot2) {
-    return (int) values[slot1] - (int) values[slot2];
+  protected Float longToValue(long value) {
+    return NumericUtils.shortToHalfFloat((short) value);
   }
 
   @Override
-  public int compareBottom(int doc) {
-    // TODO: are there sneaky non-branch ways to compute sign of float?
-    short v = (short) currentReaderValues.get(doc);
-    // Test for v == 0 to save Bits.get method call for
-    // the common case (doc has value and value is non-zero):
-    if (docsWithField != null && v == 0 && !docsWithField.get(doc)) {
-      v = missingShortValue;
-    }
-
-    return (int) bottom - (int) v;
-  }
-
-  @Override
-  public void copy(int slot, int doc) {
-    short v =  (short) currentReaderValues.get(doc);
-    // Test for v == 0 to save Bits.get method call for
-    // the common case (doc has value and value is non-zero):
-    if (docsWithField != null && v == 0 && !docsWithField.get(doc)) {
-      v = missingShortValue;
-    }
-
-    values[slot] = v;
-  }
-    
-  @Override
-  public void setBottom(final int bottom) {
-    this.bottom = values[bottom];
-  }
-
-  @Override
-  public void setTopValue(Float value) {
-    topValue = (short) NumericUtils.sortableHalfFloatBits(HalfFloat.floatToShortBits(value));
-  }
-
-  @Override
-  public Float value(int slot) {
-    return NumericUtils.shortToHalfFloat(values[slot]);
-  }
-
-  @Override
-  public int compareTop(int doc) {
-    short docValue = (short) currentReaderValues.get(doc);
-    // Test for docValue == 0 to save Bits.get method call for
-    // the common case (doc has value and value is non-zero):
-    if (docsWithField != null && docValue == 0 && !docsWithField.get(doc)) {
-      docValue = missingShortValue;
-    }
-    return (int) topValue - (int) docValue;
+  protected long valueToLong(Float value) {
+    return NumericUtils.halfFloatToShort(value);
   }
 }

Added: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/IntComparator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/IntComparator.java?rev=1649007&view=auto
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/IntComparator.java (added)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/IntComparator.java Fri Jan  2 12:02:31 2015
@@ -0,0 +1,42 @@
+package org.apache.lucene.search;
+
+/*
+ * 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.
+ */
+
+/** Parses field's values as int (using {@link
+ *  org.apache.lucene.index.LeafReader#getNumericDocValues(String)} and sorts by ascending value */
+public class IntComparator extends NumericComparator<Integer> {
+
+  /** 
+   * Creates a new comparator based on {@link Integer#compare} for {@code numHits}.
+   * When a document has no value for the field, {@code missingValue} is substituted. 
+   */
+  public IntComparator(int numHits, String field, Integer missingValue) {
+    super(numHits, field, missingValue);
+  }
+
+  @Override
+  protected Integer longToValue(long value) {
+    return (int) value;
+  }
+
+  @Override
+  protected long valueToLong(Integer value) {
+    return value.longValue();
+  }
+}
+

Added: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/LongComparator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/LongComparator.java?rev=1649007&view=auto
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/LongComparator.java (added)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/LongComparator.java Fri Jan  2 12:02:31 2015
@@ -0,0 +1,41 @@
+package org.apache.lucene.search;
+
+/*
+ * 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.
+ */
+
+/** Parses field's values as int (using {@link
+ *  org.apache.lucene.index.LeafReader#getNumericDocValues(String)} and sorts by ascending value */
+public class LongComparator extends NumericComparator<Long> {
+
+  /** 
+   * Creates a new comparator based on {@link Integer#compare} for {@code numHits}.
+   * When a document has no value for the field, {@code missingValue} is substituted. 
+   */
+  public LongComparator(int numHits, String field, Long missingValue) {
+    super(numHits, field, missingValue);
+  }
+
+  @Override
+  protected Long longToValue(long value) {
+    return value;
+  }
+
+  @Override
+  protected long valueToLong(Long value) {
+    return value;
+  }
+}

Added: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/NumericComparator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/NumericComparator.java?rev=1649007&view=auto
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/NumericComparator.java (added)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/NumericComparator.java Fri Jan  2 12:02:31 2015
@@ -0,0 +1,110 @@
+package org.apache.lucene.search;
+
+import java.io.IOException;
+
+import org.apache.lucene.index.DocValues;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.NumericDocValues;
+import org.apache.lucene.util.Bits;
+
+/*
+ * 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.
+ */
+
+/**
+ * Base FieldComparator class for numeric types
+ */
+public abstract class NumericComparator<T extends Number> extends FieldComparator<T> {
+  private final long[] values;
+  private final long missingValue;
+  private long bottom;
+  private long topValue;
+  protected final String field;
+  protected Bits docsWithField;
+  protected NumericDocValues currentReaderValues;
+    
+  public NumericComparator(int numHits, String field, long missingValue) {
+    this.field = field;
+    this.values = new long[numHits];
+    this.missingValue = missingValue;
+  }
+
+  @Override
+  public int compare(int slot1, int slot2) {
+    return Long.compare(values[slot1], values[slot2]);
+  }
+
+  private long getDocValue(int doc) {
+    long v = currentReaderValues.get(doc);
+    // Test for v == 0 to save Bits.get method call for
+    // the common case (doc has value and value is non-zero):
+    if (docsWithField != null && v == 0 && !docsWithField.get(doc)) {
+      v = missingValue;
+    }
+    return v;
+  }
+
+  @Override
+  public int compareBottom(int doc) {
+    return Long.compare(bottom, getDocValue(doc));
+  }
+
+  @Override
+  public void copy(int slot, int doc) {
+    values[slot] = getDocValue(doc);
+  }
+    
+  @Override
+  public void setBottom(final int bottom) {
+    this.bottom = values[bottom];
+  }
+
+  @Override
+  public void setTopValue(T value) {
+    topValue = valueToLong(value);
+  }
+
+  @Override
+  public FieldComparator<T> setNextReader(LeafReaderContext context) throws IOException {
+    currentReaderValues = getNumericDocValues(context, field);
+    docsWithField = DocValues.getDocsWithField(context.reader(), field);
+    // optimization to remove unneeded checks on the bit interface:
+    if (docsWithField instanceof Bits.MatchAllBits) {
+      docsWithField = null;
+    }
+    return this;
+  }
+    
+  @Override
+  public int compareTop(int doc) {
+    return Long.compare(topValue, getDocValue(doc));
+  }
+
+  protected abstract T longToValue(long value);
+
+  protected abstract long valueToLong(T value);
+
+  @Override
+  public T value(int slot) {
+    return longToValue(values[slot]);
+  }
+
+  /** Retrieves the NumericDocValues for the field in this segment */
+  protected NumericDocValues getNumericDocValues(LeafReaderContext context, String field) throws IOException {
+    return DocValues.getNumeric(context.reader(), field);
+  }
+}
+

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/SortField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/SortField.java?rev=1649007&r1=1649006&r2=1649007&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/SortField.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/SortField.java Fri Jan  2 12:02:31 2015
@@ -34,6 +34,8 @@ import org.apache.lucene.util.StringHelp
  */
 public class SortField {
 
+  // TODO: remove this eventually (FieldTypes knows the type):
+
   /**
    * Specifies the type of the terms to be sorted, or special types such as CUSTOM
    */
@@ -332,16 +334,16 @@ public class SortField {
       return new FieldComparator.DocComparator(numHits);
 
     case INT:
-      return new FieldComparator.IntComparator(numHits, field, (Integer) missingValue);
+      return new IntComparator(numHits, field, missingValue == null ? 0 : (Integer) missingValue);
 
     case FLOAT:
-      return new FieldComparator.FloatComparator(numHits, field, (Float) missingValue);
+      return new FloatComparator(numHits, field, missingValue == null ? 0 : (Float) missingValue);
 
     case LONG:
-      return new FieldComparator.LongComparator(numHits, field, (Long) missingValue);
+      return new LongComparator(numHits, field, missingValue == null ? 0 : (Long) missingValue);
 
     case DOUBLE:
-      return new FieldComparator.DoubleComparator(numHits, field, (Double) missingValue);
+      return new DoubleComparator(numHits, field, missingValue == null ? 0 : (Double) missingValue);
 
     case CUSTOM:
       assert comparatorSource != null;

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/SortedNumericSelector.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/SortedNumericSelector.java?rev=1649007&r1=1649006&r2=1649007&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/SortedNumericSelector.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/SortedNumericSelector.java Fri Jan  2 12:02:31 2015
@@ -50,13 +50,7 @@ public class SortedNumericSelector {
    * Wraps a multi-valued SortedNumericDocValues as a single-valued view, using the specified selector 
    * and numericType.
    */
-  public static NumericDocValues wrap(SortedNumericDocValues sortedNumeric, Type selector, SortField.Type numericType) {
-    if (numericType != SortField.Type.INT &&
-        numericType != SortField.Type.LONG && 
-        numericType != SortField.Type.FLOAT &&
-        numericType != SortField.Type.DOUBLE) {
-      throw new IllegalArgumentException("numericType must be a numeric type");
-    }
+  public static NumericDocValues wrap(SortedNumericDocValues sortedNumeric, Type selector) {
     final NumericDocValues view;
     NumericDocValues singleton = DocValues.unwrapSingleton(sortedNumeric);
     if (singleton != null) {
@@ -76,25 +70,8 @@ public class SortedNumericSelector {
           throw new AssertionError();
       }
     }
-    // undo the numericutils sortability
-    switch(numericType) {
-      case FLOAT:
-        return new NumericDocValues() {
-          @Override
-          public long get(int docID) {
-            return (int) view.get(docID);
-          }
-        };
-      case DOUBLE:
-        return new NumericDocValues() {
-          @Override
-          public long get(int docID) {
-            return view.get(docID);
-          }
-        };
-      default:
-        return view;
-    }
+
+    return view;
   }
   
   /** Wraps a SortedNumericDocValues and returns the first value (min) */

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/SortedNumericSortField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/SortedNumericSortField.java?rev=1649007&r1=1649006&r2=1649007&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/SortedNumericSortField.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/SortedNumericSortField.java Fri Jan  2 12:02:31 2015
@@ -135,31 +135,31 @@ public class SortedNumericSortField exte
   public FieldComparator<?> getComparator(int numHits, int sortPos) throws IOException {
     switch(type) {
       case INT:
-        return new FieldComparator.IntComparator(numHits, getField(), (Integer) missingValue) {
+        return new IntComparator(numHits, getField(), missingValue == null ? 0 : (Integer) missingValue) {
           @Override
           protected NumericDocValues getNumericDocValues(LeafReaderContext context, String field) throws IOException {
-            return SortedNumericSelector.wrap(DocValues.getSortedNumeric(context.reader(), field), selector, type);
+            return SortedNumericSelector.wrap(DocValues.getSortedNumeric(context.reader(), field), selector);
           } 
         };
       case FLOAT:
-        return new FieldComparator.FloatComparator(numHits, getField(), (Float) missingValue) {
+        return new FloatComparator(numHits, getField(), missingValue == null ? 0 : (Float) missingValue) {
           @Override
           protected NumericDocValues getNumericDocValues(LeafReaderContext context, String field) throws IOException {
-            return SortedNumericSelector.wrap(DocValues.getSortedNumeric(context.reader(), field), selector, type);
+            return SortedNumericSelector.wrap(DocValues.getSortedNumeric(context.reader(), field), selector);
           } 
         };
       case LONG:
-        return new FieldComparator.LongComparator(numHits, getField(), (Long) missingValue) {
+        return new LongComparator(numHits, getField(), missingValue == null ? 0 : (Long) missingValue) {
           @Override
           protected NumericDocValues getNumericDocValues(LeafReaderContext context, String field) throws IOException {
-            return SortedNumericSelector.wrap(DocValues.getSortedNumeric(context.reader(), field), selector, type);
+            return SortedNumericSelector.wrap(DocValues.getSortedNumeric(context.reader(), field), selector);
           }
         };
       case DOUBLE:
-        return new FieldComparator.DoubleComparator(numHits, getField(), (Double) missingValue) {
+        return new DoubleComparator(numHits, getField(), missingValue == null ? 0 : (Double) missingValue) {
           @Override
           protected NumericDocValues getNumericDocValues(LeafReaderContext context, String field) throws IOException {
-            return SortedNumericSelector.wrap(DocValues.getSortedNumeric(context.reader(), field), selector, type);
+            return SortedNumericSelector.wrap(DocValues.getSortedNumeric(context.reader(), field), selector);
           } 
         };
       default:

Copied: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/TermFilter.java (from r1643659, lucene/dev/branches/lucene6005/lucene/queries/src/java/org/apache/lucene/queries/TermFilter.java)
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/TermFilter.java?p2=lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/TermFilter.java&p1=lucene/dev/branches/lucene6005/lucene/queries/src/java/org/apache/lucene/queries/TermFilter.java&r1=1643659&r2=1649007&rev=1649007&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/queries/src/java/org/apache/lucene/queries/TermFilter.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/TermFilter.java Fri Jan  2 12:02:31 2015
@@ -1,4 +1,4 @@
-package org.apache.lucene.queries;
+package org.apache.lucene.search;
 
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
@@ -66,6 +66,14 @@ final public class TermFilter extends Fi
     if (!termsEnum.seekExact(term.bytes())) {
       return null;
     }
+
+    int maxDoc = context.reader().maxDoc();
+    if (termsEnum.docFreq() == maxDoc) {
+      // Term matches all docs
+      // TODO: is there a better way?  Do we have a BitsDocIdSet that i can just wrap acceptDocs with if it's non-null?
+      return BitsFilteredDocIdSet.wrap(DocIdSet.full(maxDoc), acceptDocs);
+    }
+
     return new DocIdSet() {
       @Override
       public DocIdSetIterator iterator() throws IOException {

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/TermRangeQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/TermRangeQuery.java?rev=1649007&r1=1649006&r2=1649007&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/TermRangeQuery.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/TermRangeQuery.java Fri Jan  2 12:02:31 2015
@@ -46,7 +46,6 @@ public class TermRangeQuery extends Mult
   private final boolean includeUpper;
   private final CompiledAutomaton compiled;
 
-
   /**
    * Constructs a query selecting all terms greater/equal than <code>lowerTerm</code>
    * but less/equal than <code>upperTerm</code>. 
@@ -111,12 +110,15 @@ public class TermRangeQuery extends Mult
       return TermsEnum.EMPTY;
     }
 
+    BytesRef minTerm = terms.getMin();
+    BytesRef maxTerm = terms.getMax();
+
     // Optimization: if our range is outside of the range indexed in this segment, skip it:
-    if (upperTerm != null && terms.getMin().compareTo(upperTerm) > 0) {
+    if (upperTerm != null && minTerm.compareTo(upperTerm) > 0) {
       return TermsEnum.EMPTY;
     }
 
-    if (lowerTerm != null && terms.getMax().compareTo(lowerTerm) < 0) {
+    if (lowerTerm != null && maxTerm.compareTo(lowerTerm) < 0) {
       return TermsEnum.EMPTY;
     }      
      
@@ -127,6 +129,8 @@ public class TermRangeQuery extends Mult
       return terms.iterator(null);
     }
 
+    // TODO: we can detect when range matches all terms here, but then how to optimize?  It's best to just let auto-prefix take it?
+
     return compiled.getTermsEnum(terms);
   }
 

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/util/Bits.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/util/Bits.java?rev=1649007&r1=1649006&r2=1649007&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/util/Bits.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/util/Bits.java Fri Jan  2 12:02:31 2015
@@ -43,7 +43,7 @@ public interface Bits {
   public static class MatchAllBits implements Bits {
     final int len;
     
-    public MatchAllBits( int len ) {
+    public MatchAllBits(int len) {
       this.len = len;
     }
 

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/util/BytesRef.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/util/BytesRef.java?rev=1649007&r1=1649006&r2=1649007&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/util/BytesRef.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/util/BytesRef.java Fri Jan  2 12:02:31 2015
@@ -178,25 +178,6 @@ public final class BytesRef implements C
     return utf8SortedAsUnicodeSortOrder;
   }
 
-  private final static Comparator<BytesRef> rightJustifiedComparator = new Comparator<BytesRef>() {
-    @Override
-    public int compare(BytesRef a, BytesRef b) {
-      //assert a.length == 0 || a.bytes[a.offset] != 0;
-      //assert b.length == 0 || b.bytes[b.offset] != 0;
-      if (a.length < b.length) {
-        return -1;
-      } else if (a.length > b.length) {
-        return 1;
-      } else {
-        return utf8SortedAsUnicodeSortOrder.compare(a, b);
-      }
-    }
-  };
-
-  public static Comparator<BytesRef> getRightJustifiedComparator() {
-    return rightJustifiedComparator;
-  }
-
   private static class UTF8SortedAsUnicodeComparator implements Comparator<BytesRef> {
     // Only singleton
     private UTF8SortedAsUnicodeComparator() {};

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/util/HalfFloat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/util/HalfFloat.java?rev=1649007&r1=1649006&r2=1649007&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/util/HalfFloat.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/util/HalfFloat.java Fri Jan  2 12:02:31 2015
@@ -17,20 +17,16 @@ package org.apache.lucene.util;
  * limitations under the License.
  */
 
-// nocommit move to NumericUtils
-
 /** Utility class to convert half-precision 16 bit floating-point number according
  *  to IEEE 754-2008. */
 
-// nocommit need separate test here
-// nocommit does this match the standard?
 public class HalfFloat {
 
   // From https://github.com/ata4/ioutils/blob/master/src/info/ata4/io/util/HalfFloat.java, in turn from http://stackoverflow.com/a/6162687
   private HalfFloat() {
   }
 
-  public static float shortBitsToFloat(short hbits) {
+  public static float shortToFloat(short hbits) {
     int mant = hbits & 0x03ff;          // 10 bits mantissa
     int exp = hbits & 0x7c00;           // 5 bits exponent
     if (exp == 0x7c00) {                // NaN/Inf
@@ -54,8 +50,7 @@ public class HalfFloat {
                                 | (exp | mant) << 13);      // value << ( 23 - 10 )
   }
 
-  // Only uses bottom 16 bits:
-  public static short floatToShortBits(float fval) {
+  public static short floatToShort(float fval) {
     int fbits = Float.floatToIntBits(fval);
     int sign = fbits >>> 16 & 0x8000;           // sign only
     int val = (fbits & 0x7fffffff) + 0x1000;    // rounded value

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/util/NumericUtils.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/util/NumericUtils.java?rev=1649007&r1=1649006&r2=1649007&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/util/NumericUtils.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/util/NumericUtils.java Fri Jan  2 12:02:31 2015
@@ -19,6 +19,7 @@ package org.apache.lucene.util;
 
 import java.io.IOException;
 import java.math.BigInteger;
+import java.util.Arrays;
 
 import org.apache.lucene.index.FilterLeafReader;
 import org.apache.lucene.index.FilteredTermsEnum;
@@ -44,7 +45,7 @@ public final class NumericUtils {
   private NumericUtils() {} // no instance!
 
   public static short halfFloatToShort(float value) {
-    return sortableHalfFloatBits((short) HalfFloat.floatToShortBits(value));
+    return sortableHalfFloatBits((short) HalfFloat.floatToShort(value));
   }
 
   /**
@@ -74,7 +75,7 @@ public final class NumericUtils {
   }
 
   public static float shortToHalfFloat(short v) {
-    return HalfFloat.shortBitsToFloat(sortableHalfFloatBits(v));
+    return HalfFloat.shortToFloat(sortableHalfFloatBits(v));
   }
 
   /**
@@ -196,7 +197,7 @@ public final class NumericUtils {
   }
 
   public static float bytesToHalfFloat(BytesRef bytes) {
-    return HalfFloat.shortBitsToFloat(sortableHalfFloatBits(bytesToShort(bytes)));
+    return HalfFloat.shortToFloat(sortableHalfFloatBits(bytesToShort(bytes)));
   }
 
   public static float bytesToFloat(BytesRef bytes) {
@@ -207,11 +208,20 @@ public final class NumericUtils {
     return longToDouble(bytesToLong(bytes));
   }
 
-  public static BytesRef bigIntToBytes(BigInteger value) {
+  public static BytesRef bigIntToBytes(BigInteger value, int maxBytes) {
     byte[] bytes = value.toByteArray();
-    sortableBigIntBytes(bytes);
-    //System.out.println(value + " -> " + new BytesRef(bytes));
-    return new BytesRef(bytes);
+    if (bytes.length > maxBytes) {
+      throw new IllegalArgumentException("BigInt " + value + " exceeds allowed byte width " + maxBytes);
+    }
+    byte[] bytes2 = new byte[maxBytes];
+    System.arraycopy(bytes, 0, bytes2, maxBytes-bytes.length, bytes.length);
+    if (bytes.length < maxBytes && (bytes[0] & 0x80) != 0) {
+      Arrays.fill(bytes2, 0, maxBytes-bytes.length, (byte) 0xff);
+    }
+    sortableBigIntBytes(bytes2);
+    BytesRef br = new BytesRef(bytes2);
+    //System.out.println("BI " + value + " -> " + br);
+    return br;
   }
 
   public static BigInteger bytesToBigInt(BytesRef bytes) {
@@ -222,7 +232,9 @@ public final class NumericUtils {
   }
 
   private static void sortableBigIntBytes(byte[] bytes) {
-    // nocommit does NOT work
-    //    bytes[0] ^= 0x80;
+    bytes[0] ^= 0x80;
+    for(int i=1;i<bytes.length;i++)  {
+      bytes[i] ^= 0;
+    }
   }
 }

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/util/StringHelper.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/util/StringHelper.java?rev=1649007&r1=1649006&r2=1649007&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/util/StringHelper.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/util/StringHelper.java Fri Jan  2 12:02:31 2015
@@ -253,7 +253,7 @@ public abstract class StringHelper {
       x0 = Long.parseLong(prop, 16);
       x1 = x0;
     } else {
-      // "Ghetto randomess" from 3 different sources:
+      // "Rough randomess" from 3 different sources:
       x0 = System.nanoTime();
       x1 = StringHelper.class.hashCode() << 32;
       StringBuilder sb = new StringBuilder();