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 2016/06/11 15:49:25 UTC

[09/21] lucene-solr:branch_6x: LUCENE-6766: make new Lucene62Codec, with new segment info format that reads/writes the index sort, to support index sorting; remove all usage of SlowCompositeReaderWrapper; also sort ids when merging norms; CheckIndex veri

LUCENE-6766: make new Lucene62Codec, with new segment info format that reads/writes the index sort, to support index sorting; remove all usage of SlowCompositeReaderWrapper; also sort ids when merging norms; CheckIndex verifies sort


Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/1530c5a9
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/1530c5a9
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/1530c5a9

Branch: refs/heads/branch_6x
Commit: 1530c5a9a5dacd0dbf1ae4bba2394ecee251c75b
Parents: fdc220e
Author: Mike McCandless <mi...@apache.org>
Authored: Fri May 6 16:41:26 2016 -0400
Committer: Mike McCandless <mi...@apache.org>
Committed: Sat Jun 11 11:48:40 2016 -0400

----------------------------------------------------------------------
 .../lucene/codecs/lucene50/Lucene50Codec.java   |   2 +-
 .../lucene50/Lucene50SegmentInfoFormat.java     | 105 +++++
 .../lucene/codecs/lucene60/Lucene60Codec.java   | 178 +++++++++
 .../services/org.apache.lucene.codecs.Codec     |   1 +
 .../lucene/codecs/lucene50/Lucene50RWCodec.java |   7 +
 .../lucene50/Lucene50RWSegmentInfoFormat.java   | 142 +++++++
 .../benchmark/byTask/tasks/CreateIndexTask.java |   4 +-
 lucene/classification/build.xml                 |   6 +-
 .../classification/utils/DatasetSplitter.java   |  27 +-
 .../classification/utils/DataSplitterTest.java  |   6 +-
 .../simpletext/SimpleTextSegmentInfoFormat.java |   2 +
 .../java/org/apache/lucene/codecs/Codec.java    |   2 +-
 .../apache/lucene/codecs/DocValuesConsumer.java |  16 +-
 .../org/apache/lucene/codecs/NormsConsumer.java |  73 ++--
 .../CompressingStoredFieldsWriter.java          |   6 +
 .../CompressingTermVectorsWriter.java           |   5 +
 .../lucene50/Lucene50SegmentInfoFormat.java     | 171 --------
 .../lucene/codecs/lucene54/package-info.java    |   4 +-
 .../lucene/codecs/lucene60/Lucene60Codec.java   | 178 ---------
 .../codecs/lucene60/Lucene60PointsWriter.java   |   7 +
 .../lucene/codecs/lucene60/package-info.java    | 397 +------------------
 .../lucene/codecs/lucene62/Lucene62Codec.java   | 178 +++++++++
 .../lucene62/Lucene62SegmentInfoFormat.java     | 289 ++++++++++++++
 .../lucene/codecs/lucene62/package-info.java    |  24 ++
 .../org/apache/lucene/index/CheckIndex.java     |  87 ++++
 .../lucene/index/MappingMultiPostingsEnum.java  |   7 +-
 .../java/org/apache/lucene/index/Sorter.java    |   1 -
 .../apache/lucene/index/SortingLeafReader.java  |   5 +-
 .../services/org.apache.lucene.codecs.Codec     |   2 +-
 .../lucene50/TestLucene50SegmentInfoFormat.java |  39 --
 ...cene50StoredFieldsFormatHighCompression.java |   8 +-
 .../lucene53/TestLucene53NormsFormat.java       |   4 +-
 .../lucene62/TestLucene62SegmentInfoFormat.java |  39 ++
 .../org/apache/lucene/index/Test2BPoints.java   |   4 +-
 .../org/apache/lucene/index/Test2BTerms.java    |   2 +-
 .../apache/lucene/index/TestIndexSorting.java   |  24 +-
 .../apache/lucene/index/TestPointValues.java    |   8 +-
 .../apache/lucene/search/TestPointQueries.java  |   4 +-
 .../search/highlight/TermVectorLeafReader.java  |   7 +-
 .../apache/lucene/index/memory/MemoryIndex.java |   5 +
 .../search/TestDiversifiedTopDocsCollector.java |   1 -
 .../TestEarlyTerminatingSortingCollector.java   |  19 +-
 .../org/apache/lucene/document/TestNearest.java |   2 +-
 .../apache/lucene/spatial3d/TestGeo3DPoint.java |   4 +-
 .../analyzing/AnalyzingInfixSuggester.java      |   6 +-
 .../suggest/document/TestSuggestField.java      |   4 +-
 .../apache/lucene/geo/BaseGeoPointTestCase.java |   2 +-
 .../index/BaseSegmentInfoFormatTestCase.java    |   6 +
 .../util/TestRuleSetupAndRestoreClassEnv.java   |   6 +-
 .../java/org/apache/lucene/util/TestUtil.java   |   4 +-
 50 files changed, 1204 insertions(+), 926 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1530c5a9/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene50/Lucene50Codec.java
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene50/Lucene50Codec.java b/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene50/Lucene50Codec.java
index 001439c..19d6e3b 100644
--- a/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene50/Lucene50Codec.java
+++ b/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene50/Lucene50Codec.java
@@ -108,7 +108,7 @@ public class Lucene50Codec extends Codec {
   }
   
   @Override
-  public final SegmentInfoFormat segmentInfoFormat() {
+  public SegmentInfoFormat segmentInfoFormat() {
     return segmentInfosFormat;
   }
   

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1530c5a9/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene50/Lucene50SegmentInfoFormat.java
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene50/Lucene50SegmentInfoFormat.java b/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene50/Lucene50SegmentInfoFormat.java
new file mode 100644
index 0000000..9c5453f
--- /dev/null
+++ b/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene50/Lucene50SegmentInfoFormat.java
@@ -0,0 +1,105 @@
+/*
+ * 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.
+ */
+package org.apache.lucene.codecs.lucene50;
+
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.codecs.SegmentInfoFormat;
+import org.apache.lucene.index.CorruptIndexException;
+import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.index.IndexWriter; // javadocs
+import org.apache.lucene.index.SegmentInfo; // javadocs
+import org.apache.lucene.index.SegmentInfos; // javadocs
+import org.apache.lucene.store.ChecksumIndexInput;
+import org.apache.lucene.store.DataOutput; // javadocs
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
+import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.util.Version;
+
+/**
+ * Lucene 5.0 Segment info format.
+ * @deprecated Only for reading old 5.0-6.0 segments
+ */
+@Deprecated
+public class Lucene50SegmentInfoFormat extends SegmentInfoFormat {
+
+  /** Sole constructor. */
+  public Lucene50SegmentInfoFormat() {
+  }
+  
+  @Override
+  public SegmentInfo read(Directory dir, String segment, byte[] segmentID, IOContext context) throws IOException {
+    final String fileName = IndexFileNames.segmentFileName(segment, "", Lucene50SegmentInfoFormat.SI_EXTENSION);
+    try (ChecksumIndexInput input = dir.openChecksumInput(fileName, context)) {
+      Throwable priorE = null;
+      SegmentInfo si = null;
+      try {
+        int format = CodecUtil.checkIndexHeader(input, Lucene50SegmentInfoFormat.CODEC_NAME,
+                                          Lucene50SegmentInfoFormat.VERSION_START,
+                                          Lucene50SegmentInfoFormat.VERSION_CURRENT,
+                                          segmentID, "");
+        final Version version = Version.fromBits(input.readInt(), input.readInt(), input.readInt());
+        
+        final int docCount = input.readInt();
+        if (docCount < 0) {
+          throw new CorruptIndexException("invalid docCount: " + docCount, input);
+        }
+        final boolean isCompoundFile = input.readByte() == SegmentInfo.YES;
+        
+        final Map<String,String> diagnostics;
+        final Set<String> files;
+        final Map<String,String> attributes;
+        
+        if (format >= VERSION_SAFE_MAPS) {
+          diagnostics = input.readMapOfStrings();
+          files = input.readSetOfStrings();
+          attributes = input.readMapOfStrings();
+        } else {
+          diagnostics = Collections.unmodifiableMap(input.readStringStringMap());
+          files = Collections.unmodifiableSet(input.readStringSet());
+          attributes = Collections.unmodifiableMap(input.readStringStringMap());
+        }
+        
+        si = new SegmentInfo(dir, version, segment, docCount, isCompoundFile, null, diagnostics, segmentID, attributes, null);
+        si.setFiles(files);
+      } catch (Throwable exception) {
+        priorE = exception;
+      } finally {
+        CodecUtil.checkFooter(input, priorE);
+      }
+      return si;
+    }
+  }
+
+  @Override
+  public void write(Directory dir, SegmentInfo si, IOContext ioContext) throws IOException {
+    throw new UnsupportedOperationException("this codec can only be used for reading");
+  }
+
+  /** File extension used to store {@link SegmentInfo}. */
+  public final static String SI_EXTENSION = "si";
+  static final String CODEC_NAME = "Lucene50SegmentInfo";
+  static final int VERSION_START = 0;
+  static final int VERSION_SAFE_MAPS = 1;
+  static final int VERSION_CURRENT = VERSION_SAFE_MAPS;
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1530c5a9/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene60/Lucene60Codec.java
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene60/Lucene60Codec.java b/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene60/Lucene60Codec.java
new file mode 100644
index 0000000..7210b3f
--- /dev/null
+++ b/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene60/Lucene60Codec.java
@@ -0,0 +1,178 @@
+/*
+ * 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.
+ */
+package org.apache.lucene.codecs.lucene60;
+
+// nocommit if index time sorting is in use, don't try to bulk merge ... later we can make crazy bulk merger that looks for long runs from
+// one sub?
+
+import java.util.Objects;
+
+import org.apache.lucene.codecs.Codec;
+import org.apache.lucene.codecs.CompoundFormat;
+import org.apache.lucene.codecs.DocValuesFormat;
+import org.apache.lucene.codecs.FieldInfosFormat;
+import org.apache.lucene.codecs.FilterCodec;
+import org.apache.lucene.codecs.LiveDocsFormat;
+import org.apache.lucene.codecs.NormsFormat;
+import org.apache.lucene.codecs.PointsFormat;
+import org.apache.lucene.codecs.PostingsFormat;
+import org.apache.lucene.codecs.SegmentInfoFormat;
+import org.apache.lucene.codecs.StoredFieldsFormat;
+import org.apache.lucene.codecs.TermVectorsFormat;
+import org.apache.lucene.codecs.lucene50.Lucene50CompoundFormat;
+import org.apache.lucene.codecs.lucene50.Lucene50LiveDocsFormat;
+import org.apache.lucene.codecs.lucene50.Lucene50SegmentInfoFormat;
+import org.apache.lucene.codecs.lucene50.Lucene50StoredFieldsFormat.Mode;
+import org.apache.lucene.codecs.lucene50.Lucene50StoredFieldsFormat;
+import org.apache.lucene.codecs.lucene50.Lucene50TermVectorsFormat;
+import org.apache.lucene.codecs.lucene53.Lucene53NormsFormat;
+import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat;
+import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
+
+/**
+ * Implements the Lucene 6.0 index format, with configurable per-field postings
+ * and docvalues formats.
+ * <p>
+ * If you want to reuse functionality of this codec in another codec, extend
+ * {@link FilterCodec}.
+ *
+ * @see org.apache.lucene.codecs.lucene60 package documentation for file format details.
+ *
+ * @lucene.experimental
+ */
+public class Lucene60Codec extends Codec {
+  private final TermVectorsFormat vectorsFormat = new Lucene50TermVectorsFormat();
+  private final FieldInfosFormat fieldInfosFormat = new Lucene60FieldInfosFormat();
+  private final SegmentInfoFormat segmentInfosFormat = new Lucene50SegmentInfoFormat();
+  private final LiveDocsFormat liveDocsFormat = new Lucene50LiveDocsFormat();
+  private final CompoundFormat compoundFormat = new Lucene50CompoundFormat();
+  
+  private final PostingsFormat postingsFormat = new PerFieldPostingsFormat() {
+    @Override
+    public PostingsFormat getPostingsFormatForField(String field) {
+      return Lucene60Codec.this.getPostingsFormatForField(field);
+    }
+  };
+  
+  private final DocValuesFormat docValuesFormat = new PerFieldDocValuesFormat() {
+    @Override
+    public DocValuesFormat getDocValuesFormatForField(String field) {
+      return Lucene60Codec.this.getDocValuesFormatForField(field);
+    }
+  };
+  
+  private final StoredFieldsFormat storedFieldsFormat;
+
+  /** 
+   * Instantiates a new codec.
+   */
+  public Lucene60Codec() {
+    this(Mode.BEST_SPEED);
+  }
+  
+  /** 
+   * Instantiates a new codec, specifying the stored fields compression
+   * mode to use.
+   * @param mode stored fields compression mode to use for newly 
+   *             flushed/merged segments.
+   */
+  public Lucene60Codec(Mode mode) {
+    super("Lucene60");
+    this.storedFieldsFormat = new Lucene50StoredFieldsFormat(Objects.requireNonNull(mode));
+  }
+  
+  @Override
+  public final StoredFieldsFormat storedFieldsFormat() {
+    return storedFieldsFormat;
+  }
+  
+  @Override
+  public final TermVectorsFormat termVectorsFormat() {
+    return vectorsFormat;
+  }
+
+  @Override
+  public final PostingsFormat postingsFormat() {
+    return postingsFormat;
+  }
+  
+  @Override
+  public final FieldInfosFormat fieldInfosFormat() {
+    return fieldInfosFormat;
+  }
+  
+  @Override
+  public final SegmentInfoFormat segmentInfoFormat() {
+    return segmentInfosFormat;
+  }
+  
+  @Override
+  public final LiveDocsFormat liveDocsFormat() {
+    return liveDocsFormat;
+  }
+
+  @Override
+  public final CompoundFormat compoundFormat() {
+    return compoundFormat;
+  }
+
+  @Override
+  public final PointsFormat pointsFormat() {
+    return new Lucene60PointsFormat();
+  }
+
+  /** Returns the postings format that should be used for writing 
+   *  new segments of <code>field</code>.
+   *  
+   *  The default implementation always returns "Lucene50".
+   *  <p>
+   *  <b>WARNING:</b> if you subclass, you are responsible for index 
+   *  backwards compatibility: future version of Lucene are only 
+   *  guaranteed to be able to read the default implementation. 
+   */
+  public PostingsFormat getPostingsFormatForField(String field) {
+    return defaultFormat;
+  }
+  
+  /** Returns the docvalues format that should be used for writing 
+   *  new segments of <code>field</code>.
+   *  
+   *  The default implementation always returns "Lucene54".
+   *  <p>
+   *  <b>WARNING:</b> if you subclass, you are responsible for index 
+   *  backwards compatibility: future version of Lucene are only 
+   *  guaranteed to be able to read the default implementation. 
+   */
+  public DocValuesFormat getDocValuesFormatForField(String field) {
+    return defaultDVFormat;
+  }
+  
+  @Override
+  public final DocValuesFormat docValuesFormat() {
+    return docValuesFormat;
+  }
+
+  private final PostingsFormat defaultFormat = PostingsFormat.forName("Lucene50");
+  private final DocValuesFormat defaultDVFormat = DocValuesFormat.forName("Lucene54");
+
+  private final NormsFormat normsFormat = new Lucene53NormsFormat();
+
+  @Override
+  public final NormsFormat normsFormat() {
+    return normsFormat;
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1530c5a9/lucene/backward-codecs/src/resources/META-INF/services/org.apache.lucene.codecs.Codec
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/resources/META-INF/services/org.apache.lucene.codecs.Codec b/lucene/backward-codecs/src/resources/META-INF/services/org.apache.lucene.codecs.Codec
index 7f66de8..71aa938 100644
--- a/lucene/backward-codecs/src/resources/META-INF/services/org.apache.lucene.codecs.Codec
+++ b/lucene/backward-codecs/src/resources/META-INF/services/org.apache.lucene.codecs.Codec
@@ -16,3 +16,4 @@
 org.apache.lucene.codecs.lucene50.Lucene50Codec
 org.apache.lucene.codecs.lucene53.Lucene53Codec
 org.apache.lucene.codecs.lucene54.Lucene54Codec
+org.apache.lucene.codecs.lucene60.Lucene60Codec

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1530c5a9/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene50/Lucene50RWCodec.java
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene50/Lucene50RWCodec.java b/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene50/Lucene50RWCodec.java
index 359e2ec..8fdeb20 100644
--- a/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene50/Lucene50RWCodec.java
+++ b/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene50/Lucene50RWCodec.java
@@ -18,6 +18,7 @@ package org.apache.lucene.codecs.lucene50;
 
 
 import org.apache.lucene.codecs.NormsFormat;
+import org.apache.lucene.codecs.SegmentInfoFormat;
 
 /**
  * Codec for testing 5.0 index format
@@ -26,9 +27,15 @@ import org.apache.lucene.codecs.NormsFormat;
 @Deprecated
 final class Lucene50RWCodec extends Lucene50Codec {
   private final NormsFormat normsFormat = new Lucene50RWNormsFormat();
+  private final SegmentInfoFormat segmentInfoFormat = new Lucene50RWSegmentInfoFormat();
 
   @Override
   public NormsFormat normsFormat() {
     return normsFormat;
   }
+
+  @Override
+  public SegmentInfoFormat segmentInfoFormat() {
+    return segmentInfoFormat;
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1530c5a9/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene50/Lucene50RWSegmentInfoFormat.java
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene50/Lucene50RWSegmentInfoFormat.java b/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene50/Lucene50RWSegmentInfoFormat.java
new file mode 100644
index 0000000..d457243
--- /dev/null
+++ b/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene50/Lucene50RWSegmentInfoFormat.java
@@ -0,0 +1,142 @@
+/*
+ * 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.
+ */
+package org.apache.lucene.codecs.lucene50;
+
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.codecs.SegmentInfoFormat;
+import org.apache.lucene.index.CorruptIndexException;
+import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.index.IndexWriter; // javadocs
+import org.apache.lucene.index.SegmentInfo; // javadocs
+import org.apache.lucene.index.SegmentInfos; // javadocs
+import org.apache.lucene.store.ChecksumIndexInput;
+import org.apache.lucene.store.DataOutput; // javadocs
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
+import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.util.Version;
+
+/**
+ * Read-write version of 5.0 SegmentInfoFormat for testing
+ * @deprecated for test purposes only
+ */
+@Deprecated
+public class Lucene50RWSegmentInfoFormat extends Lucene50SegmentInfoFormat {
+
+  /** Sole constructor. */
+  public Lucene50RWSegmentInfoFormat() {
+  }
+  
+  @Override
+  public SegmentInfo read(Directory dir, String segment, byte[] segmentID, IOContext context) throws IOException {
+    final String fileName = IndexFileNames.segmentFileName(segment, "", Lucene50SegmentInfoFormat.SI_EXTENSION);
+    try (ChecksumIndexInput input = dir.openChecksumInput(fileName, context)) {
+      Throwable priorE = null;
+      SegmentInfo si = null;
+      try {
+        int format = CodecUtil.checkIndexHeader(input, Lucene50SegmentInfoFormat.CODEC_NAME,
+                                          Lucene50SegmentInfoFormat.VERSION_START,
+                                          Lucene50SegmentInfoFormat.VERSION_CURRENT,
+                                          segmentID, "");
+        final Version version = Version.fromBits(input.readInt(), input.readInt(), input.readInt());
+        
+        final int docCount = input.readInt();
+        if (docCount < 0) {
+          throw new CorruptIndexException("invalid docCount: " + docCount, input);
+        }
+        final boolean isCompoundFile = input.readByte() == SegmentInfo.YES;
+        
+        final Map<String,String> diagnostics;
+        final Set<String> files;
+        final Map<String,String> attributes;
+        
+        if (format >= VERSION_SAFE_MAPS) {
+          diagnostics = input.readMapOfStrings();
+          files = input.readSetOfStrings();
+          attributes = input.readMapOfStrings();
+        } else {
+          diagnostics = Collections.unmodifiableMap(input.readStringStringMap());
+          files = Collections.unmodifiableSet(input.readStringSet());
+          attributes = Collections.unmodifiableMap(input.readStringStringMap());
+        }
+        
+        si = new SegmentInfo(dir, version, segment, docCount, isCompoundFile, null, diagnostics, segmentID, attributes, null);
+        si.setFiles(files);
+      } catch (Throwable exception) {
+        priorE = exception;
+      } finally {
+        CodecUtil.checkFooter(input, priorE);
+      }
+      return si;
+    }
+  }
+
+  @Override
+  public void write(Directory dir, SegmentInfo si, IOContext ioContext) throws IOException {
+    final String fileName = IndexFileNames.segmentFileName(si.name, "", Lucene50SegmentInfoFormat.SI_EXTENSION);
+    // nocommit indexSort
+    if (si.getIndexSort() != null) {
+      throw new IllegalArgumentException("teach me to write indexSort");
+    }
+
+    try (IndexOutput output = dir.createOutput(fileName, ioContext)) {
+      // Only add the file once we've successfully created it, else IFD assert can trip:
+      si.addFile(fileName);
+      CodecUtil.writeIndexHeader(output, 
+                                   Lucene50SegmentInfoFormat.CODEC_NAME, 
+                                   Lucene50SegmentInfoFormat.VERSION_CURRENT,
+                                   si.getId(),
+                                   "");
+      Version version = si.getVersion();
+      if (version.major < 5) {
+        throw new IllegalArgumentException("invalid major version: should be >= 5 but got: " + version.major + " segment=" + si);
+      }
+      // Write the Lucene version that created this segment, since 3.1
+      output.writeInt(version.major);
+      output.writeInt(version.minor);
+      output.writeInt(version.bugfix);
+      assert version.prerelease == 0;
+      output.writeInt(si.maxDoc());
+
+      output.writeByte((byte) (si.getUseCompoundFile() ? SegmentInfo.YES : SegmentInfo.NO));
+      output.writeMapOfStrings(si.getDiagnostics());
+      Set<String> files = si.files();
+      for (String file : files) {
+        if (!IndexFileNames.parseSegmentName(file).equals(si.name)) {
+          throw new IllegalArgumentException("invalid files: expected segment=" + si.name + ", got=" + files);
+        }
+      }
+      output.writeSetOfStrings(files);
+      output.writeMapOfStrings(si.getAttributes());
+      
+      CodecUtil.writeFooter(output);
+    }
+  }
+
+  /** File extension used to store {@link SegmentInfo}. */
+  public final static String SI_EXTENSION = "si";
+  static final String CODEC_NAME = "Lucene50SegmentInfo";
+  static final int VERSION_START = 0;
+  static final int VERSION_SAFE_MAPS = 1;
+  static final int VERSION_CURRENT = VERSION_SAFE_MAPS;
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1530c5a9/lucene/benchmark/src/java/org/apache/lucene/benchmark/byTask/tasks/CreateIndexTask.java
----------------------------------------------------------------------
diff --git a/lucene/benchmark/src/java/org/apache/lucene/benchmark/byTask/tasks/CreateIndexTask.java b/lucene/benchmark/src/java/org/apache/lucene/benchmark/byTask/tasks/CreateIndexTask.java
index 74486d6..df8a1b4 100644
--- a/lucene/benchmark/src/java/org/apache/lucene/benchmark/byTask/tasks/CreateIndexTask.java
+++ b/lucene/benchmark/src/java/org/apache/lucene/benchmark/byTask/tasks/CreateIndexTask.java
@@ -29,7 +29,7 @@ import org.apache.lucene.benchmark.byTask.PerfRunData;
 import org.apache.lucene.benchmark.byTask.utils.Config;
 import org.apache.lucene.codecs.Codec;
 import org.apache.lucene.codecs.PostingsFormat;
-import org.apache.lucene.codecs.lucene60.Lucene60Codec;
+import org.apache.lucene.codecs.lucene62.Lucene62Codec;
 import org.apache.lucene.index.ConcurrentMergeScheduler;
 import org.apache.lucene.index.IndexCommit;
 import org.apache.lucene.index.IndexDeletionPolicy;
@@ -139,7 +139,7 @@ public class CreateIndexTask extends PerfTask {
     if (defaultCodec == null && postingsFormat != null) {
       try {
         final PostingsFormat postingsFormatChosen = PostingsFormat.forName(postingsFormat);
-        iwConf.setCodec(new Lucene60Codec() {
+        iwConf.setCodec(new Lucene62Codec() {
           @Override
           public PostingsFormat getPostingsFormatForField(String field) {
             return postingsFormatChosen;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1530c5a9/lucene/classification/build.xml
----------------------------------------------------------------------
diff --git a/lucene/classification/build.xml b/lucene/classification/build.xml
index 3ddb9bd..704cae8 100644
--- a/lucene/classification/build.xml
+++ b/lucene/classification/build.xml
@@ -28,7 +28,6 @@
     <path refid="base.classpath"/>
     <pathelement path="${queries.jar}"/>
     <pathelement path="${grouping.jar}"/>
-    <pathelement path="${misc.jar}"/>
   </path>
 
   <path id="test.classpath">
@@ -37,17 +36,16 @@
     <path refid="test.base.classpath"/>
   </path>
 
-  <target name="compile-core" depends="jar-misc,jar-grouping,jar-queries,jar-analyzers-common,common.compile-core" />
+  <target name="compile-core" depends="jar-grouping,jar-queries,jar-analyzers-common,common.compile-core" />
 
   <target name="jar-core" depends="common.jar-core" />
 
-  <target name="javadocs" depends="javadocs-misc,javadocs-grouping,javadocs-misc,compile-core,check-javadocs-uptodate"
+  <target name="javadocs" depends="javadocs-grouping,compile-core,check-javadocs-uptodate"
           unless="javadocs-uptodate-${name}">
     <invoke-module-javadoc>
       <links>
         <link href="../queries"/>
         <link href="../grouping"/>
-        <link href="../misc"/>
       </links>
     </invoke-module-javadoc>
   </target>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1530c5a9/lucene/classification/src/java/org/apache/lucene/classification/utils/DatasetSplitter.java
----------------------------------------------------------------------
diff --git a/lucene/classification/src/java/org/apache/lucene/classification/utils/DatasetSplitter.java b/lucene/classification/src/java/org/apache/lucene/classification/utils/DatasetSplitter.java
index fce786b..c1c8ad1 100644
--- a/lucene/classification/src/java/org/apache/lucene/classification/utils/DatasetSplitter.java
+++ b/lucene/classification/src/java/org/apache/lucene/classification/utils/DatasetSplitter.java
@@ -29,6 +29,7 @@ import org.apache.lucene.index.IndexWriter;
 import org.apache.lucene.index.IndexWriterConfig;
 import org.apache.lucene.index.IndexableField;
 import org.apache.lucene.index.LeafReader;
+import org.apache.lucene.index.SortedDocValues;
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.MatchAllDocsQuery;
@@ -38,7 +39,6 @@ import org.apache.lucene.search.grouping.GroupDocs;
 import org.apache.lucene.search.grouping.GroupingSearch;
 import org.apache.lucene.search.grouping.TopGroups;
 import org.apache.lucene.store.Directory;
-import org.apache.lucene.uninverting.UninvertingReader;
 
 /**
  * Utility class for creating training / test / cross validation indexes from the original index.
@@ -68,7 +68,7 @@ public class DatasetSplitter {
    * @param crossValidationIndex a {@link Directory} used to write the cross validation index
    * @param analyzer             {@link Analyzer} used to create the new docs
    * @param termVectors          {@code true} if term vectors should be kept
-   * @param classFieldName       names of the field used as the label for classification
+   * @param classFieldName       name of the field used as the label for classification; this must be indexed with sorted doc values
    * @param fieldNames           names of fields that need to be put in the new indexes or <code>null</code> if all should be used
    * @throws IOException if any writing operation fails on any of the indexes
    */
@@ -80,30 +80,23 @@ public class DatasetSplitter {
     IndexWriter cvWriter = new IndexWriter(crossValidationIndex, new IndexWriterConfig(analyzer));
     IndexWriter trainingWriter = new IndexWriter(trainingIndex, new IndexWriterConfig(analyzer));
 
-    // try to get the exact no. of existing classes
-    Terms terms = originalIndex.terms(classFieldName);
-    long noOfClasses = -1;
-    if (terms != null) {
-      noOfClasses = terms.size();
-
-    }
-    if (noOfClasses == -1) {
-      noOfClasses = 10000; // fallback
+    // get the exact no. of existing classes
+    SortedDocValues classValues = originalIndex.getSortedDocValues(classFieldName);
+    if (classValues == null) {
+      throw new IllegalStateException("the classFieldName \"" + classFieldName + "\" must index sorted doc values");
     }
 
-    HashMap<String, UninvertingReader.Type> mapping = new HashMap<>();
-    mapping.put(classFieldName, UninvertingReader.Type.SORTED);
-    UninvertingReader uninvertingReader = new UninvertingReader(originalIndex, mapping);
+    int noOfClasses = classValues.getValueCount();
 
     try {
 
-      IndexSearcher indexSearcher = new IndexSearcher(uninvertingReader);
+      IndexSearcher indexSearcher = new IndexSearcher(originalIndex);
       GroupingSearch gs = new GroupingSearch(classFieldName);
       gs.setGroupSort(Sort.INDEXORDER);
       gs.setSortWithinGroup(Sort.INDEXORDER);
       gs.setAllGroups(true);
       gs.setGroupDocsLimit(originalIndex.maxDoc());
-      TopGroups<Object> topGroups = gs.search(indexSearcher, new MatchAllDocsQuery(), 0, (int) noOfClasses);
+      TopGroups<Object> topGroups = gs.search(indexSearcher, new MatchAllDocsQuery(), 0, noOfClasses);
 
       // set the type to be indexed, stored, with term vectors
       FieldType ft = new FieldType(TextField.TYPE_STORED);
@@ -156,7 +149,7 @@ public class DatasetSplitter {
       testWriter.close();
       cvWriter.close();
       trainingWriter.close();
-      uninvertingReader.close();
+      originalIndex.close();
     }
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1530c5a9/lucene/classification/src/test/org/apache/lucene/classification/utils/DataSplitterTest.java
----------------------------------------------------------------------
diff --git a/lucene/classification/src/test/org/apache/lucene/classification/utils/DataSplitterTest.java b/lucene/classification/src/test/org/apache/lucene/classification/utils/DataSplitterTest.java
index 0b6f077..fdd4b0b 100644
--- a/lucene/classification/src/test/org/apache/lucene/classification/utils/DataSplitterTest.java
+++ b/lucene/classification/src/test/org/apache/lucene/classification/utils/DataSplitterTest.java
@@ -70,7 +70,9 @@ public class DataSplitterTest extends LuceneTestCase {
       doc = new Document();
       doc.add(new Field(idFieldName, "id" + Integer.toString(i), ft));
       doc.add(new Field(textFieldName, TestUtil.randomUnicodeString(rnd, 1024), ft));
-      doc.add(new Field(classFieldName, Integer.toString(rnd.nextInt(10)), ft));
+      String className = Integer.toString(rnd.nextInt(10));
+      doc.add(new Field(classFieldName, className, ft));
+      doc.add(new SortedDocValuesField(classFieldName, new BytesRef(className)));
       indexWriter.addDocument(doc);
     }
 
@@ -89,13 +91,11 @@ public class DataSplitterTest extends LuceneTestCase {
     super.tearDown();
   }
 
-
   @Test
   public void testSplitOnAllFields() throws Exception {
     assertSplit(originalIndex, 0.1, 0.1);
   }
 
-
   @Test
   public void testSplitOnSomeFields() throws Exception {
     assertSplit(originalIndex, 0.2, 0.35, idFieldName, textFieldName);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1530c5a9/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextSegmentInfoFormat.java
----------------------------------------------------------------------
diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextSegmentInfoFormat.java b/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextSegmentInfoFormat.java
index 594fccf..bf9d3de 100644
--- a/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextSegmentInfoFormat.java
+++ b/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextSegmentInfoFormat.java
@@ -216,6 +216,7 @@ public class SimpleTextSegmentInfoFormat extends SegmentInfoFormat {
                 break;
             }
             break;
+          // nocommit need the rest
           default:
             throw new AssertionError();
         }
@@ -337,6 +338,7 @@ public class SimpleTextSegmentInfoFormat extends SegmentInfoFormat {
           case INT:
             sortType = "int";
             break;
+          // nocommit the rest:
           default:
             throw new IllegalStateException("Unexpected sort type: " + sortField.getType());
         }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1530c5a9/lucene/core/src/java/org/apache/lucene/codecs/Codec.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/Codec.java b/lucene/core/src/java/org/apache/lucene/codecs/Codec.java
index 5d704ca..442445c 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/Codec.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/Codec.java
@@ -57,7 +57,7 @@ public abstract class Codec implements NamedSPILoader.NamedSPI {
     }
     
     // TODO: should we use this, or maybe a system property is better?
-    static Codec defaultCodec = LOADER.lookup("Lucene60");
+    static Codec defaultCodec = LOADER.lookup("Lucene62");
   }
 
   private final String name;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1530c5a9/lucene/core/src/java/org/apache/lucene/codecs/DocValuesConsumer.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/DocValuesConsumer.java b/lucene/core/src/java/org/apache/lucene/codecs/DocValuesConsumer.java
index 52bf9b2..79cc422 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/DocValuesConsumer.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/DocValuesConsumer.java
@@ -316,16 +316,14 @@ public abstract class DocValuesConsumer implements Closeable {
                           }
 
                           private boolean setNext() {
-                            while (true) {
-                              NumericDocValuesSub sub = docIDMerger.next();
-                              if (sub == null) {
-                                return false;
-                              }
-                              nextIsSet = true;
-                              nextValue = sub.values.get(sub.docID);
-                              nextHasValue = nextValue != 0 || sub.docsWithField.get(sub.docID);
-                              return true;
+                            NumericDocValuesSub sub = docIDMerger.next();
+                            if (sub == null) {
+                              return false;
                             }
+                            nextIsSet = true;
+                            nextValue = sub.values.get(sub.docID);
+                            nextHasValue = nextValue != 0 || sub.docsWithField.get(sub.docID);
+                            return true;
                           }
                         };
                       }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1530c5a9/lucene/core/src/java/org/apache/lucene/codecs/NormsConsumer.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/NormsConsumer.java b/lucene/core/src/java/org/apache/lucene/codecs/NormsConsumer.java
index b771aab..76f8be7 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/NormsConsumer.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/NormsConsumer.java
@@ -16,7 +16,6 @@
  */
 package org.apache.lucene.codecs;
 
-
 import java.io.Closeable;
 import java.io.IOException;
 import java.util.ArrayList;
@@ -24,6 +23,7 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.NoSuchElementException;
 
+import org.apache.lucene.index.DocIDMerger;
 import org.apache.lucene.index.DocValues;
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.MergeState;
@@ -31,6 +31,8 @@ import org.apache.lucene.index.NumericDocValues;
 import org.apache.lucene.index.SegmentWriteState;
 import org.apache.lucene.util.Bits;
 
+import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
+
 /** 
  * Abstract API that consumes normalization values.  
  * Concrete implementations of this
@@ -98,6 +100,30 @@ public abstract class NormsConsumer implements Closeable {
     }
   }
   
+  /** Tracks state of one numeric sub-reader that we are merging */
+  private static class NumericDocValuesSub extends DocIDMerger.Sub {
+
+    private final NumericDocValues values;
+    private int docID = -1;
+    private final int maxDoc;
+
+    public NumericDocValuesSub(MergeState.DocMap docMap, Bits liveDocs, NumericDocValues values, int maxDoc) {
+      super(docMap, liveDocs);
+      this.values = values;
+      this.maxDoc = maxDoc;
+    }
+
+    @Override
+    public int nextDoc() {
+      docID++;
+      if (docID == maxDoc) {
+        return NO_MORE_DOCS;
+      } else {
+        return docID;
+      }
+    }
+  }
+
   /**
    * Merges the norms from <code>toMerge</code>.
    * <p>
@@ -111,13 +137,18 @@ public abstract class NormsConsumer implements Closeable {
                     new Iterable<Number>() {
                       @Override
                       public Iterator<Number> iterator() {
+
+                        // We must make a new DocIDMerger for each iterator:
+                        List<NumericDocValuesSub> subs = new ArrayList<>();
+                        assert mergeState.docMaps.length == toMerge.size();
+                        for(int i=0;i<toMerge.size();i++) {
+                          subs.add(new NumericDocValuesSub(mergeState.docMaps[i], mergeState.liveDocs[i], toMerge.get(i), mergeState.maxDocs[i]));
+                        }
+
+                        final DocIDMerger<NumericDocValuesSub> docIDMerger = new DocIDMerger<>(subs, mergeState.segmentInfo.getIndexSort() != null);
+
                         return new Iterator<Number>() {
-                          int readerUpto = -1;
-                          int docIDUpto;
                           long nextValue;
-                          int maxDoc;
-                          NumericDocValues currentValues;
-                          Bits currentLiveDocs;
                           boolean nextIsSet;
 
                           @Override
@@ -141,31 +172,13 @@ public abstract class NormsConsumer implements Closeable {
                           }
 
                           private boolean setNext() {
-                            while (true) {
-                              if (readerUpto == toMerge.size()) {
-                                return false;
-                              }
-
-                              if (currentValues == null || docIDUpto == maxDoc) {
-                                readerUpto++;
-                                if (readerUpto < toMerge.size()) {
-                                  currentValues = toMerge.get(readerUpto);
-                                  currentLiveDocs = mergeState.liveDocs[readerUpto];
-                                  maxDoc = mergeState.maxDocs[readerUpto];
-                                }
-                                docIDUpto = 0;
-                                continue;
-                              }
-
-                              if (currentLiveDocs == null || currentLiveDocs.get(docIDUpto)) {
-                                nextIsSet = true;
-                                nextValue = currentValues.get(docIDUpto);
-                                docIDUpto++;
-                                return true;
-                              }
-
-                              docIDUpto++;
+                            NumericDocValuesSub sub = docIDMerger.next();
+                            if (sub == null) {
+                              return false;
                             }
+                            nextIsSet = true;
+                            nextValue = sub.values.get(sub.docID);
+                            return true;
                           }
                         };
                       }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1530c5a9/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsWriter.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsWriter.java b/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsWriter.java
index 79dfb27..d5bf4ad 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsWriter.java
@@ -486,6 +486,12 @@ public final class CompressingStoredFieldsWriter extends StoredFieldsWriter {
 
   @Override
   public int merge(MergeState mergeState) throws IOException {
+    if (mergeState.segmentInfo.getIndexSort() != null) {
+      // TODO: can we gain back some optos even if index is sorted?  E.g. if sort results in large chunks of contiguous docs from one sub
+      // being copied over...?
+      return super.merge(mergeState);
+    }
+
     int docCount = 0;
     int numReaders = mergeState.maxDocs.length;
     

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1530c5a9/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingTermVectorsWriter.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingTermVectorsWriter.java b/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingTermVectorsWriter.java
index 07f2711..9f8f44e 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingTermVectorsWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingTermVectorsWriter.java
@@ -730,6 +730,11 @@ public final class CompressingTermVectorsWriter extends TermVectorsWriter {
 
   @Override
   public int merge(MergeState mergeState) throws IOException {
+    if (mergeState.segmentInfo.getIndexSort() != null) {
+      // TODO: can we gain back some optos even if index is sorted?  E.g. if sort results in large chunks of contiguous docs from one sub
+      // being copied over...?
+      return super.merge(mergeState);
+    }
     int docCount = 0;
     int numReaders = mergeState.maxDocs.length;
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1530c5a9/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50SegmentInfoFormat.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50SegmentInfoFormat.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50SegmentInfoFormat.java
deleted file mode 100644
index 9786ec1..0000000
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50SegmentInfoFormat.java
+++ /dev/null
@@ -1,171 +0,0 @@
-/*
- * 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.
- */
-package org.apache.lucene.codecs.lucene50;
-
-
-import java.io.IOException;
-import java.util.Collections;
-import java.util.Map;
-import java.util.Set;
-
-import org.apache.lucene.codecs.CodecUtil;
-import org.apache.lucene.codecs.SegmentInfoFormat;
-import org.apache.lucene.index.CorruptIndexException;
-import org.apache.lucene.index.IndexFileNames;
-import org.apache.lucene.index.IndexWriter; // javadocs
-import org.apache.lucene.index.SegmentInfo; // javadocs
-import org.apache.lucene.index.SegmentInfos; // javadocs
-import org.apache.lucene.store.ChecksumIndexInput;
-import org.apache.lucene.store.DataOutput; // javadocs
-import org.apache.lucene.store.Directory;
-import org.apache.lucene.store.IOContext;
-import org.apache.lucene.store.IndexOutput;
-import org.apache.lucene.util.Version;
-
-/**
- * Lucene 5.0 Segment info format.
- * <p>
- * Files:
- * <ul>
- *   <li><tt>.si</tt>: Header, SegVersion, SegSize, IsCompoundFile, Diagnostics, Files, Attributes, Footer
- * </ul>
- * Data types:
- * <ul>
- *   <li>Header --&gt; {@link CodecUtil#writeIndexHeader IndexHeader}</li>
- *   <li>SegSize --&gt; {@link DataOutput#writeInt Int32}</li>
- *   <li>SegVersion --&gt; {@link DataOutput#writeString String}</li>
- *   <li>Files --&gt; {@link DataOutput#writeSetOfStrings Set&lt;String&gt;}</li>
- *   <li>Diagnostics,Attributes --&gt; {@link DataOutput#writeMapOfStrings Map&lt;String,String&gt;}</li>
- *   <li>IsCompoundFile --&gt; {@link DataOutput#writeByte Int8}</li>
- *   <li>Footer --&gt; {@link CodecUtil#writeFooter CodecFooter}</li>
- * </ul>
- * Field Descriptions:
- * <ul>
- *   <li>SegVersion is the code version that created the segment.</li>
- *   <li>SegSize is the number of documents contained in the segment index.</li>
- *   <li>IsCompoundFile records whether the segment is written as a compound file or
- *       not. If this is -1, the segment is not a compound file. If it is 1, the segment
- *       is a compound file.</li>
- *   <li>The Diagnostics Map is privately written by {@link IndexWriter}, as a debugging aid,
- *       for each segment it creates. It includes metadata like the current Lucene
- *       version, OS, Java version, why the segment was created (merge, flush,
- *       addIndexes), etc.</li>
- *   <li>Files is a list of files referred to by this segment.</li>
- * </ul>
- * 
- * @see SegmentInfos
- * @lucene.experimental
- */
-public class Lucene50SegmentInfoFormat extends SegmentInfoFormat {
-
-  /** Sole constructor. */
-  public Lucene50SegmentInfoFormat() {
-  }
-  
-  @Override
-  public SegmentInfo read(Directory dir, String segment, byte[] segmentID, IOContext context) throws IOException {
-    final String fileName = IndexFileNames.segmentFileName(segment, "", Lucene50SegmentInfoFormat.SI_EXTENSION);
-    try (ChecksumIndexInput input = dir.openChecksumInput(fileName, context)) {
-      Throwable priorE = null;
-      SegmentInfo si = null;
-      try {
-        int format = CodecUtil.checkIndexHeader(input, Lucene50SegmentInfoFormat.CODEC_NAME,
-                                          Lucene50SegmentInfoFormat.VERSION_START,
-                                          Lucene50SegmentInfoFormat.VERSION_CURRENT,
-                                          segmentID, "");
-        final Version version = Version.fromBits(input.readInt(), input.readInt(), input.readInt());
-        
-        final int docCount = input.readInt();
-        if (docCount < 0) {
-          throw new CorruptIndexException("invalid docCount: " + docCount, input);
-        }
-        final boolean isCompoundFile = input.readByte() == SegmentInfo.YES;
-        
-        final Map<String,String> diagnostics;
-        final Set<String> files;
-        final Map<String,String> attributes;
-        
-        if (format >= VERSION_SAFE_MAPS) {
-          diagnostics = input.readMapOfStrings();
-          files = input.readSetOfStrings();
-          attributes = input.readMapOfStrings();
-        } else {
-          diagnostics = Collections.unmodifiableMap(input.readStringStringMap());
-          files = Collections.unmodifiableSet(input.readStringSet());
-          attributes = Collections.unmodifiableMap(input.readStringStringMap());
-        }
-        
-        si = new SegmentInfo(dir, version, segment, docCount, isCompoundFile, null, diagnostics, segmentID, attributes, null);
-        si.setFiles(files);
-      } catch (Throwable exception) {
-        priorE = exception;
-      } finally {
-        CodecUtil.checkFooter(input, priorE);
-      }
-      return si;
-    }
-  }
-
-  @Override
-  public void write(Directory dir, SegmentInfo si, IOContext ioContext) throws IOException {
-    final String fileName = IndexFileNames.segmentFileName(si.name, "", Lucene50SegmentInfoFormat.SI_EXTENSION);
-    // nocommit indexSort
-    if (si.getIndexSort() != null) {
-      throw new IllegalArgumentException("teach me to write indexSort");
-    }
-
-    try (IndexOutput output = dir.createOutput(fileName, ioContext)) {
-      // Only add the file once we've successfully created it, else IFD assert can trip:
-      si.addFile(fileName);
-      CodecUtil.writeIndexHeader(output, 
-                                   Lucene50SegmentInfoFormat.CODEC_NAME, 
-                                   Lucene50SegmentInfoFormat.VERSION_CURRENT,
-                                   si.getId(),
-                                   "");
-      Version version = si.getVersion();
-      if (version.major < 5) {
-        throw new IllegalArgumentException("invalid major version: should be >= 5 but got: " + version.major + " segment=" + si);
-      }
-      // Write the Lucene version that created this segment, since 3.1
-      output.writeInt(version.major);
-      output.writeInt(version.minor);
-      output.writeInt(version.bugfix);
-      assert version.prerelease == 0;
-      output.writeInt(si.maxDoc());
-
-      output.writeByte((byte) (si.getUseCompoundFile() ? SegmentInfo.YES : SegmentInfo.NO));
-      output.writeMapOfStrings(si.getDiagnostics());
-      Set<String> files = si.files();
-      for (String file : files) {
-        if (!IndexFileNames.parseSegmentName(file).equals(si.name)) {
-          throw new IllegalArgumentException("invalid files: expected segment=" + si.name + ", got=" + files);
-        }
-      }
-      output.writeSetOfStrings(files);
-      output.writeMapOfStrings(si.getAttributes());
-      
-      CodecUtil.writeFooter(output);
-    }
-  }
-
-  /** File extension used to store {@link SegmentInfo}. */
-  public final static String SI_EXTENSION = "si";
-  static final String CODEC_NAME = "Lucene50SegmentInfo";
-  static final int VERSION_START = 0;
-  static final int VERSION_SAFE_MAPS = 1;
-  static final int VERSION_CURRENT = VERSION_SAFE_MAPS;
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1530c5a9/lucene/core/src/java/org/apache/lucene/codecs/lucene54/package-info.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene54/package-info.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene54/package-info.java
index ebae849..5dec06b 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene54/package-info.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene54/package-info.java
@@ -135,7 +135,7 @@
  * <p>Each segment index maintains the following:</p>
  * <ul>
  * <li>
- * {@link org.apache.lucene.codecs.lucene50.Lucene50SegmentInfoFormat Segment info}.
+ * {@link org.apache.lucene.codecs.lucene62.Lucene62SegmentInfoFormat Segment info}.
  *    This contains metadata about a segment, such as the number of documents,
  *    what files it uses, 
  * </li>
@@ -235,7 +235,7 @@
  * file.</td>
  * </tr>
  * <tr>
- * <td>{@link org.apache.lucene.codecs.lucene50.Lucene50SegmentInfoFormat Segment Info}</td>
+ * <td>{@link org.apache.lucene.codecs.lucene62.Lucene62SegmentInfoFormat Segment Info}</td>
  * <td>.si</td>
  * <td>Stores metadata about a segment</td>
  * </tr>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1530c5a9/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60Codec.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60Codec.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60Codec.java
deleted file mode 100644
index c696fbe..0000000
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60Codec.java
+++ /dev/null
@@ -1,178 +0,0 @@
-/*
- * 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.
- */
-package org.apache.lucene.codecs.lucene60;
-
-// nocommit if index time sorting is in use, don't try to bulk merge ... later we can make crazy bulk merger that looks for long runs from
-// one sub?
-
-import java.util.Objects;
-
-import org.apache.lucene.codecs.Codec;
-import org.apache.lucene.codecs.CompoundFormat;
-import org.apache.lucene.codecs.DocValuesFormat;
-import org.apache.lucene.codecs.FieldInfosFormat;
-import org.apache.lucene.codecs.FilterCodec;
-import org.apache.lucene.codecs.LiveDocsFormat;
-import org.apache.lucene.codecs.NormsFormat;
-import org.apache.lucene.codecs.PointsFormat;
-import org.apache.lucene.codecs.PostingsFormat;
-import org.apache.lucene.codecs.SegmentInfoFormat;
-import org.apache.lucene.codecs.StoredFieldsFormat;
-import org.apache.lucene.codecs.TermVectorsFormat;
-import org.apache.lucene.codecs.lucene50.Lucene50CompoundFormat;
-import org.apache.lucene.codecs.lucene50.Lucene50LiveDocsFormat;
-import org.apache.lucene.codecs.lucene50.Lucene50SegmentInfoFormat;
-import org.apache.lucene.codecs.lucene50.Lucene50StoredFieldsFormat.Mode;
-import org.apache.lucene.codecs.lucene50.Lucene50StoredFieldsFormat;
-import org.apache.lucene.codecs.lucene50.Lucene50TermVectorsFormat;
-import org.apache.lucene.codecs.lucene53.Lucene53NormsFormat;
-import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat;
-import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
-
-/**
- * Implements the Lucene 6.0 index format, with configurable per-field postings
- * and docvalues formats.
- * <p>
- * If you want to reuse functionality of this codec in another codec, extend
- * {@link FilterCodec}.
- *
- * @see org.apache.lucene.codecs.lucene60 package documentation for file format details.
- *
- * @lucene.experimental
- */
-public class Lucene60Codec extends Codec {
-  private final TermVectorsFormat vectorsFormat = new Lucene50TermVectorsFormat();
-  private final FieldInfosFormat fieldInfosFormat = new Lucene60FieldInfosFormat();
-  private final SegmentInfoFormat segmentInfosFormat = new Lucene50SegmentInfoFormat();
-  private final LiveDocsFormat liveDocsFormat = new Lucene50LiveDocsFormat();
-  private final CompoundFormat compoundFormat = new Lucene50CompoundFormat();
-  
-  private final PostingsFormat postingsFormat = new PerFieldPostingsFormat() {
-    @Override
-    public PostingsFormat getPostingsFormatForField(String field) {
-      return Lucene60Codec.this.getPostingsFormatForField(field);
-    }
-  };
-  
-  private final DocValuesFormat docValuesFormat = new PerFieldDocValuesFormat() {
-    @Override
-    public DocValuesFormat getDocValuesFormatForField(String field) {
-      return Lucene60Codec.this.getDocValuesFormatForField(field);
-    }
-  };
-  
-  private final StoredFieldsFormat storedFieldsFormat;
-
-  /** 
-   * Instantiates a new codec.
-   */
-  public Lucene60Codec() {
-    this(Mode.BEST_SPEED);
-  }
-  
-  /** 
-   * Instantiates a new codec, specifying the stored fields compression
-   * mode to use.
-   * @param mode stored fields compression mode to use for newly 
-   *             flushed/merged segments.
-   */
-  public Lucene60Codec(Mode mode) {
-    super("Lucene60");
-    this.storedFieldsFormat = new Lucene50StoredFieldsFormat(Objects.requireNonNull(mode));
-  }
-  
-  @Override
-  public final StoredFieldsFormat storedFieldsFormat() {
-    return storedFieldsFormat;
-  }
-  
-  @Override
-  public final TermVectorsFormat termVectorsFormat() {
-    return vectorsFormat;
-  }
-
-  @Override
-  public final PostingsFormat postingsFormat() {
-    return postingsFormat;
-  }
-  
-  @Override
-  public final FieldInfosFormat fieldInfosFormat() {
-    return fieldInfosFormat;
-  }
-  
-  @Override
-  public final SegmentInfoFormat segmentInfoFormat() {
-    return segmentInfosFormat;
-  }
-  
-  @Override
-  public final LiveDocsFormat liveDocsFormat() {
-    return liveDocsFormat;
-  }
-
-  @Override
-  public final CompoundFormat compoundFormat() {
-    return compoundFormat;
-  }
-
-  @Override
-  public final PointsFormat pointsFormat() {
-    return new Lucene60PointsFormat();
-  }
-
-  /** Returns the postings format that should be used for writing 
-   *  new segments of <code>field</code>.
-   *  
-   *  The default implementation always returns "Lucene50".
-   *  <p>
-   *  <b>WARNING:</b> if you subclass, you are responsible for index 
-   *  backwards compatibility: future version of Lucene are only 
-   *  guaranteed to be able to read the default implementation. 
-   */
-  public PostingsFormat getPostingsFormatForField(String field) {
-    return defaultFormat;
-  }
-  
-  /** Returns the docvalues format that should be used for writing 
-   *  new segments of <code>field</code>.
-   *  
-   *  The default implementation always returns "Lucene50".
-   *  <p>
-   *  <b>WARNING:</b> if you subclass, you are responsible for index 
-   *  backwards compatibility: future version of Lucene are only 
-   *  guaranteed to be able to read the default implementation. 
-   */
-  public DocValuesFormat getDocValuesFormatForField(String field) {
-    return defaultDVFormat;
-  }
-  
-  @Override
-  public final DocValuesFormat docValuesFormat() {
-    return docValuesFormat;
-  }
-
-  private final PostingsFormat defaultFormat = PostingsFormat.forName("Lucene50");
-  private final DocValuesFormat defaultDVFormat = DocValuesFormat.forName("Lucene54");
-
-  private final NormsFormat normsFormat = new Lucene53NormsFormat();
-
-  @Override
-  public final NormsFormat normsFormat() {
-    return normsFormat;
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1530c5a9/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60PointsWriter.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60PointsWriter.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60PointsWriter.java
index f6c73bd..63308c4 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60PointsWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60PointsWriter.java
@@ -123,6 +123,13 @@ public class Lucene60PointsWriter extends PointsWriter implements Closeable {
 
   @Override
   public void merge(MergeState mergeState) throws IOException {
+    if (mergeState.segmentInfo.getIndexSort() != null) {
+      // TODO: can we gain back some optos even if index is sorted?  E.g. if sort results in large chunks of contiguous docs from one sub
+      // being copied over...?
+      super.merge(mergeState);
+      return;
+    }
+
     for(PointsReader reader : mergeState.pointsReaders) {
       if (reader instanceof Lucene60PointsReader == false) {
         // We can only bulk merge when all to-be-merged segments use our format:

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1530c5a9/lucene/core/src/java/org/apache/lucene/codecs/lucene60/package-info.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene60/package-info.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene60/package-info.java
index 03a17ba..8968a6d 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene60/package-info.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene60/package-info.java
@@ -16,400 +16,7 @@
  */
 
 /**
- * Lucene 6.0 file format.
- * 
- * <h1>Apache Lucene - Index File Formats</h1>
- * <div>
- * <ul>
- * <li><a href="#Introduction">Introduction</a></li>
- * <li><a href="#Definitions">Definitions</a>
- *   <ul>
- *   <li><a href="#Inverted_Indexing">Inverted Indexing</a></li>
- *   <li><a href="#Types_of_Fields">Types of Fields</a></li>
- *   <li><a href="#Segments">Segments</a></li>
- *   <li><a href="#Document_Numbers">Document Numbers</a></li>
- *   </ul>
- * </li>
- * <li><a href="#Overview">Index Structure Overview</a></li>
- * <li><a href="#File_Naming">File Naming</a></li>
- * <li><a href="#file-names">Summary of File Extensions</a>
- *   <ul>
- *   <li><a href="#Lock_File">Lock File</a></li>
- *   <li><a href="#History">History</a></li>
- *   <li><a href="#Limitations">Limitations</a></li>
- *   </ul>
- * </li>
- * </ul>
- * </div>
- * <a name="Introduction"></a>
- * <h2>Introduction</h2>
- * <div>
- * <p>This document defines the index file formats used in this version of Lucene.
- * If you are using a different version of Lucene, please consult the copy of
- * <code>docs/</code> that was distributed with
- * the version you are using.</p>
- * <p>Apache Lucene is written in Java, but several efforts are underway to write
- * <a href="http://wiki.apache.org/lucene-java/LuceneImplementations">versions of
- * Lucene in other programming languages</a>. If these versions are to remain
- * compatible with Apache Lucene, then a language-independent definition of the
- * Lucene index format is required. This document thus attempts to provide a
- * complete and independent definition of the Apache Lucene file formats.</p>
- * <p>As Lucene evolves, this document should evolve. Versions of Lucene in
- * different programming languages should endeavor to agree on file formats, and
- * generate new versions of this document.</p>
- * </div>
- * <a name="Definitions"></a>
- * <h2>Definitions</h2>
- * <div>
- * <p>The fundamental concepts in Lucene are index, document, field and term.</p>
- * <p>An index contains a sequence of documents.</p>
- * <ul>
- * <li>A document is a sequence of fields.</li>
- * <li>A field is a named sequence of terms.</li>
- * <li>A term is a sequence of bytes.</li>
- * </ul>
- * <p>The same sequence of bytes in two different fields is considered a different 
- * term. Thus terms are represented as a pair: the string naming the field, and the
- * bytes within the field.</p>
- * <a name="Inverted_Indexing"></a>
- * <h3>Inverted Indexing</h3>
- * <p>The index stores statistics about terms in order to make term-based search
- * more efficient. Lucene's index falls into the family of indexes known as an
- * <i>inverted index.</i> This is because it can list, for a term, the documents
- * that contain it. This is the inverse of the natural relationship, in which
- * documents list terms.</p>
- * <a name="Types_of_Fields"></a>
- * <h3>Types of Fields</h3>
- * <p>In Lucene, fields may be <i>stored</i>, in which case their text is stored
- * in the index literally, in a non-inverted manner. Fields that are inverted are
- * called <i>indexed</i>. A field may be both stored and indexed.</p>
- * <p>The text of a field may be <i>tokenized</i> into terms to be indexed, or the
- * text of a field may be used literally as a term to be indexed. Most fields are
- * tokenized, but sometimes it is useful for certain identifier fields to be
- * indexed literally.</p>
- * <p>See the {@link org.apache.lucene.document.Field Field}
- * java docs for more information on Fields.</p>
- * <a name="Segments"></a>
- * <h3>Segments</h3>
- * <p>Lucene indexes may be composed of multiple sub-indexes, or <i>segments</i>.
- * Each segment is a fully independent index, which could be searched separately.
- * Indexes evolve by:</p>
- * <ol>
- * <li>Creating new segments for newly added documents.</li>
- * <li>Merging existing segments.</li>
- * </ol>
- * <p>Searches may involve multiple segments and/or multiple indexes, each index
- * potentially composed of a set of segments.</p>
- * <a name="Document_Numbers"></a>
- * <h3>Document Numbers</h3>
- * <p>Internally, Lucene refers to documents by an integer <i>document number</i>.
- * The first document added to an index is numbered zero, and each subsequent
- * document added gets a number one greater than the previous.</p>
- * <p>Note that a document's number may change, so caution should be taken when
- * storing these numbers outside of Lucene. In particular, numbers may change in
- * the following situations:</p>
- * <ul>
- * <li>
- * <p>The numbers stored in each segment are unique only within the segment, and
- * must be converted before they can be used in a larger context. The standard
- * technique is to allocate each segment a range of values, based on the range of
- * numbers used in that segment. To convert a document number from a segment to an
- * external value, the segment's <i>base</i> document number is added. To convert
- * an external value back to a segment-specific value, the segment is identified
- * by the range that the external value is in, and the segment's base value is
- * subtracted. For example two five document segments might be combined, so that
- * the first segment has a base value of zero, and the second of five. Document
- * three from the second segment would have an external value of eight.</p>
- * </li>
- * <li>
- * <p>When documents are deleted, gaps are created in the numbering. These are
- * eventually removed as the index evolves through merging. Deleted documents are
- * dropped when segments are merged. A freshly-merged segment thus has no gaps in
- * its numbering.</p>
- * </li>
- * </ul>
- * </div>
- * <a name="Overview"></a>
- * <h2>Index Structure Overview</h2>
- * <div>
- * <p>Each segment index maintains the following:</p>
- * <ul>
- * <li>
- * {@link org.apache.lucene.codecs.lucene50.Lucene50SegmentInfoFormat Segment info}.
- *    This contains metadata about a segment, such as the number of documents,
- *    what files it uses, 
- * </li>
- * <li>
- * {@link org.apache.lucene.codecs.lucene50.Lucene50FieldInfosFormat Field names}. 
- *    This contains the set of field names used in the index.
- * </li>
- * <li>
- * {@link org.apache.lucene.codecs.lucene50.Lucene50StoredFieldsFormat Stored Field values}. 
- * This contains, for each document, a list of attribute-value pairs, where the attributes 
- * are field names. These are used to store auxiliary information about the document, such as 
- * its title, url, or an identifier to access a database. The set of stored fields are what is 
- * returned for each hit when searching. This is keyed by document number.
- * </li>
- * <li>
- * {@link org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat Term dictionary}. 
- * A dictionary containing all of the terms used in all of the
- * indexed fields of all of the documents. The dictionary also contains the number
- * of documents which contain the term, and pointers to the term's frequency and
- * proximity data.
- * </li>
- * <li>
- * {@link org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat Term Frequency data}. 
- * For each term in the dictionary, the numbers of all the
- * documents that contain that term, and the frequency of the term in that
- * document, unless frequencies are omitted (IndexOptions.DOCS_ONLY)
- * </li>
- * <li>
- * {@link org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat Term Proximity data}. 
- * For each term in the dictionary, the positions that the
- * term occurs in each document. Note that this will not exist if all fields in
- * all documents omit position data.
- * </li>
- * <li>
- * {@link org.apache.lucene.codecs.lucene53.Lucene53NormsFormat Normalization factors}. 
- * For each field in each document, a value is stored
- * that is multiplied into the score for hits on that field.
- * </li>
- * <li>
- * {@link org.apache.lucene.codecs.lucene50.Lucene50TermVectorsFormat Term Vectors}. 
- * For each field in each document, the term vector (sometimes
- * called document vector) may be stored. A term vector consists of term text and
- * term frequency. To add Term Vectors to your index see the 
- * {@link org.apache.lucene.document.Field Field} constructors
- * </li>
- * <li>
- * {@link org.apache.lucene.codecs.lucene54.Lucene54DocValuesFormat Per-document values}. 
- * Like stored values, these are also keyed by document
- * number, but are generally intended to be loaded into main memory for fast
- * access. Whereas stored values are generally intended for summary results from
- * searches, per-document values are useful for things like scoring factors.
- * </li>
- * <li>
- * {@link org.apache.lucene.codecs.lucene50.Lucene50LiveDocsFormat Live documents}. 
- * An optional file indicating which documents are live.
- * </li>
- * <li>
- * {@link org.apache.lucene.codecs.lucene60.Lucene60PointsFormat Point values}.
- * Optional pair of files, recording dimensionally indexed fields, to enable fast
- * numeric range filtering and large numeric values like BigInteger and BigDecimal (1D)
- * and geographic shape intersection (2D, 3D).
- * </li>
- * </ul>
- * <p>Details on each of these are provided in their linked pages.</p>
- * </div>
- * <a name="File_Naming"></a>
- * <h2>File Naming</h2>
- * <div>
- * <p>All files belonging to a segment have the same name with varying extensions.
- * The extensions correspond to the different file formats described below. When
- * using the Compound File format (default in 1.4 and greater) these files (except
- * for the Segment info file, the Lock file, and Deleted documents file) are collapsed 
- * into a single .cfs file (see below for details)</p>
- * <p>Typically, all segments in an index are stored in a single directory,
- * although this is not required.</p>
- * <p>As of version 2.1 (lock-less commits), file names are never re-used.
- * That is, when any file is saved
- * to the Directory it is given a never before used filename. This is achieved
- * using a simple generations approach. For example, the first segments file is
- * segments_1, then segments_2, etc. The generation is a sequential long integer
- * represented in alpha-numeric (base 36) form.</p>
- * </div>
- * <a name="file-names"></a>
- * <h2>Summary of File Extensions</h2>
- * <div>
- * <p>The following table summarizes the names and extensions of the files in
- * Lucene:</p>
- * <table cellspacing="1" cellpadding="4" summary="lucene filenames by extension">
- * <tr>
- * <th>Name</th>
- * <th>Extension</th>
- * <th>Brief Description</th>
- * </tr>
- * <tr>
- * <td>{@link org.apache.lucene.index.SegmentInfos Segments File}</td>
- * <td>segments_N</td>
- * <td>Stores information about a commit point</td>
- * </tr>
- * <tr>
- * <td><a href="#Lock_File">Lock File</a></td>
- * <td>write.lock</td>
- * <td>The Write lock prevents multiple IndexWriters from writing to the same
- * file.</td>
- * </tr>
- * <tr>
- * <td>{@link org.apache.lucene.codecs.lucene50.Lucene50SegmentInfoFormat Segment Info}</td>
- * <td>.si</td>
- * <td>Stores metadata about a segment</td>
- * </tr>
- * <tr>
- * <td>{@link org.apache.lucene.codecs.lucene50.Lucene50CompoundFormat Compound File}</td>
- * <td>.cfs, .cfe</td>
- * <td>An optional "virtual" file consisting of all the other index files for
- * systems that frequently run out of file handles.</td>
- * </tr>
- * <tr>
- * <td>{@link org.apache.lucene.codecs.lucene50.Lucene50FieldInfosFormat Fields}</td>
- * <td>.fnm</td>
- * <td>Stores information about the fields</td>
- * </tr>
- * <tr>
- * <td>{@link org.apache.lucene.codecs.lucene50.Lucene50StoredFieldsFormat Field Index}</td>
- * <td>.fdx</td>
- * <td>Contains pointers to field data</td>
- * </tr>
- * <tr>
- * <td>{@link org.apache.lucene.codecs.lucene50.Lucene50StoredFieldsFormat Field Data}</td>
- * <td>.fdt</td>
- * <td>The stored fields for documents</td>
- * </tr>
- * <tr>
- * <td>{@link org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat Term Dictionary}</td>
- * <td>.tim</td>
- * <td>The term dictionary, stores term info</td>
- * </tr>
- * <tr>
- * <td>{@link org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat Term Index}</td>
- * <td>.tip</td>
- * <td>The index into the Term Dictionary</td>
- * </tr>
- * <tr>
- * <td>{@link org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat Frequencies}</td>
- * <td>.doc</td>
- * <td>Contains the list of docs which contain each term along with frequency</td>
- * </tr>
- * <tr>
- * <td>{@link org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat Positions}</td>
- * <td>.pos</td>
- * <td>Stores position information about where a term occurs in the index</td>
- * </tr>
- * <tr>
- * <td>{@link org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat Payloads}</td>
- * <td>.pay</td>
- * <td>Stores additional per-position metadata information such as character offsets and user payloads</td>
- * </tr>
- * <tr>
- * <td>{@link org.apache.lucene.codecs.lucene53.Lucene53NormsFormat Norms}</td>
- * <td>.nvd, .nvm</td>
- * <td>Encodes length and boost factors for docs and fields</td>
- * </tr>
- * <tr>
- * <td>{@link org.apache.lucene.codecs.lucene54.Lucene54DocValuesFormat Per-Document Values}</td>
- * <td>.dvd, .dvm</td>
- * <td>Encodes additional scoring factors or other per-document information.</td>
- * </tr>
- * <tr>
- * <td>{@link org.apache.lucene.codecs.lucene50.Lucene50TermVectorsFormat Term Vector Index}</td>
- * <td>.tvx</td>
- * <td>Stores offset into the document data file</td>
- * </tr>
- * <tr>
- * <td>{@link org.apache.lucene.codecs.lucene50.Lucene50TermVectorsFormat Term Vector Documents}</td>
- * <td>.tvd</td>
- * <td>Contains information about each document that has term vectors</td>
- * </tr>
- * <tr>
- * <td>{@link org.apache.lucene.codecs.lucene50.Lucene50TermVectorsFormat Term Vector Fields}</td>
- * <td>.tvf</td>
- * <td>The field level info about term vectors</td>
- * </tr>
- * <tr>
- * <td>{@link org.apache.lucene.codecs.lucene50.Lucene50LiveDocsFormat Live Documents}</td>
- * <td>.liv</td>
- * <td>Info about what files are live</td>
- * </tr>
- * <tr>
- * <td>{@link org.apache.lucene.codecs.lucene60.Lucene60PointsFormat Point values}</td>
- * <td>.dii, .dim</td>
- * <td>Holds indexed points, if any</td>
- * </tr>
- * </table>
- * </div>
- * <a name="Lock_File"></a>
- * <h2>Lock File</h2>
- * The write lock, which is stored in the index directory by default, is named
- * "write.lock". If the lock directory is different from the index directory then
- * the write lock will be named "XXXX-write.lock" where XXXX is a unique prefix
- * derived from the full path to the index directory. When this file is present, a
- * writer is currently modifying the index (adding or removing documents). This
- * lock file ensures that only one writer is modifying the index at a time.
- * <a name="History"></a>
- * <h2>History</h2>
- * <p>Compatibility notes are provided in this document, describing how file
- * formats have changed from prior versions:</p>
- * <ul>
- * <li>In version 2.1, the file format was changed to allow lock-less commits (ie,
- * no more commit lock). The change is fully backwards compatible: you can open a
- * pre-2.1 index for searching or adding/deleting of docs. When the new segments
- * file is saved (committed), it will be written in the new file format (meaning
- * no specific "upgrade" process is needed). But note that once a commit has
- * occurred, pre-2.1 Lucene will not be able to read the index.</li>
- * <li>In version 2.3, the file format was changed to allow segments to share a
- * single set of doc store (vectors &amp; stored fields) files. This allows for
- * faster indexing in certain cases. The change is fully backwards compatible (in
- * the same way as the lock-less commits change in 2.1).</li>
- * <li>In version 2.4, Strings are now written as true UTF-8 byte sequence, not
- * Java's modified UTF-8. See <a href="http://issues.apache.org/jira/browse/LUCENE-510">
- * LUCENE-510</a> for details.</li>
- * <li>In version 2.9, an optional opaque Map&lt;String,String&gt; CommitUserData
- * may be passed to IndexWriter's commit methods (and later retrieved), which is
- * recorded in the segments_N file. See <a href="http://issues.apache.org/jira/browse/LUCENE-1382">
- * LUCENE-1382</a> for details. Also,
- * diagnostics were added to each segment written recording details about why it
- * was written (due to flush, merge; which OS/JRE was used; etc.). See issue
- * <a href="http://issues.apache.org/jira/browse/LUCENE-1654">LUCENE-1654</a> for details.</li>
- * <li>In version 3.0, compressed fields are no longer written to the index (they
- * can still be read, but on merge the new segment will write them, uncompressed).
- * See issue <a href="http://issues.apache.org/jira/browse/LUCENE-1960">LUCENE-1960</a> 
- * for details.</li>
- * <li>In version 3.1, segments records the code version that created them. See
- * <a href="http://issues.apache.org/jira/browse/LUCENE-2720">LUCENE-2720</a> for details. 
- * Additionally segments track explicitly whether or not they have term vectors. 
- * See <a href="http://issues.apache.org/jira/browse/LUCENE-2811">LUCENE-2811</a> 
- * for details.</li>
- * <li>In version 3.2, numeric fields are written as natively to stored fields
- * file, previously they were stored in text format only.</li>
- * <li>In version 3.4, fields can omit position data while still indexing term
- * frequencies.</li>
- * <li>In version 4.0, the format of the inverted index became extensible via
- * the {@link org.apache.lucene.codecs.Codec Codec} api. Fast per-document storage
- * ({@code DocValues}) was introduced. Normalization factors need no longer be a 
- * single byte, they can be any {@link org.apache.lucene.index.NumericDocValues NumericDocValues}. 
- * Terms need not be unicode strings, they can be any byte sequence. Term offsets 
- * can optionally be indexed into the postings lists. Payloads can be stored in the 
- * term vectors.</li>
- * <li>In version 4.1, the format of the postings list changed to use either
- * of FOR compression or variable-byte encoding, depending upon the frequency
- * of the term. Terms appearing only once were changed to inline directly into
- * the term dictionary. Stored fields are compressed by default. </li>
- * <li>In version 4.2, term vectors are compressed by default. DocValues has 
- * a new multi-valued type (SortedSet), that can be used for faceting/grouping/joining
- * on multi-valued fields.</li>
- * <li>In version 4.5, DocValues were extended to explicitly represent missing values.</li>
- * <li>In version 4.6, FieldInfos were extended to support per-field DocValues generation, to 
- * allow updating NumericDocValues fields.</li>
- * <li>In version 4.8, checksum footers were added to the end of each index file 
- * for improved data integrity. Specifically, the last 8 bytes of every index file
- * contain the zlib-crc32 checksum of the file.</li>
- * <li>In version 4.9, DocValues has a new multi-valued numeric type (SortedNumeric)
- * that is suitable for faceting/sorting/analytics.
- * <li>In version 5.4, DocValues have been improved to store more information on disk:
- * addresses for binary fields and ord indexes for multi-valued fields.
- * <li>In version 6.0, Points were added, for multi-dimensional range/distance search.
- * </li>
- * </ul>
- * <a name="Limitations"></a>
- * <h2>Limitations</h2>
- * <div>
- * <p>Lucene uses a Java <code>int</code> to refer to
- * document numbers, and the index file format uses an <code>Int32</code>
- * on-disk to store document numbers. This is a limitation
- * of both the index file format and the current implementation. Eventually these
- * should be replaced with either <code>UInt64</code> values, or
- * better yet, {@link org.apache.lucene.store.DataOutput#writeVInt VInt} values which have no limit.</p>
- * </div>
+ * Components from the Lucene 6.0 index format.  See {@link org.apache.lucene.codecs.lucene62}
+ * for an overview of the index format.
  */
 package org.apache.lucene.codecs.lucene60;