You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by jp...@apache.org on 2015/10/27 20:49:47 UTC

svn commit: r1710876 [2/4] - in /lucene/dev/trunk: lucene/ lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene50/ lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene53/ lucene/backward-codecs/src/resources/META-INF/services/ luc...

Added: lucene/dev/trunk/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene53/Lucene53Codec.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene53/Lucene53Codec.java?rev=1710876&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene53/Lucene53Codec.java (added)
+++ lucene/dev/trunk/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene53/Lucene53Codec.java Tue Oct 27 19:49:47 2015
@@ -0,0 +1,176 @@
+package org.apache.lucene.codecs.lucene53;
+
+/*
+ * 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.util.Objects;
+
+import org.apache.lucene.codecs.Codec;
+import org.apache.lucene.codecs.CompoundFormat;
+import org.apache.lucene.codecs.DimensionalFormat;
+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.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.Lucene50FieldInfosFormat;
+import org.apache.lucene.codecs.lucene50.Lucene50LiveDocsFormat;
+import org.apache.lucene.codecs.lucene50.Lucene50SegmentInfoFormat;
+import org.apache.lucene.codecs.lucene50.Lucene50StoredFieldsFormat;
+import org.apache.lucene.codecs.lucene50.Lucene50StoredFieldsFormat.Mode;
+import org.apache.lucene.codecs.lucene50.Lucene50TermVectorsFormat;
+import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat;
+import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
+
+/**
+ * Implements the Lucene 5.3 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.lucene53 package documentation for file format details.
+ * @deprecated Only for reading old 5.3 segments
+ */
+@Deprecated
+public class Lucene53Codec extends Codec {
+  private final TermVectorsFormat vectorsFormat = new Lucene50TermVectorsFormat();
+  private final FieldInfosFormat fieldInfosFormat = new Lucene50FieldInfosFormat();
+  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 Lucene53Codec.this.getPostingsFormatForField(field);
+    }
+  };
+  
+  private final DocValuesFormat docValuesFormat = new PerFieldDocValuesFormat() {
+    @Override
+    public DocValuesFormat getDocValuesFormatForField(String field) {
+      return Lucene53Codec.this.getDocValuesFormatForField(field);
+    }
+  };
+  
+  private final StoredFieldsFormat storedFieldsFormat;
+
+  /** 
+   * Instantiates a new codec.
+   */
+  public Lucene53Codec() {
+    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 Lucene53Codec(Mode mode) {
+    super("Lucene53");
+    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;
+  }
+
+  /** 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;
+  }
+
+  @Override
+  public final DimensionalFormat dimensionalFormat() {
+    return DimensionalFormat.EMPTY;
+  }
+
+  private final PostingsFormat defaultFormat = PostingsFormat.forName("Lucene50");
+  private final DocValuesFormat defaultDVFormat = DocValuesFormat.forName("Lucene50");
+
+  private final NormsFormat normsFormat = new Lucene53NormsFormat();
+
+  @Override
+  public final NormsFormat normsFormat() {
+    return normsFormat;
+  }
+}

Added: lucene/dev/trunk/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene53/package.html
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene53/package.html?rev=1710876&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene53/package.html (added)
+++ lucene/dev/trunk/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene53/package.html Tue Oct 27 19:49:47 2015
@@ -0,0 +1,25 @@
+<!doctype html public "-//w3c//dtd html 4.0 transitional//en">
+<!--
+ 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.
+-->
+<html>
+<head>
+   <meta http-equiv="Content-Type" content="text/html; charset=iso-8859-1">
+</head>
+<body>
+Lucene 5.3 file format.
+</body>
+</html>

Modified: lucene/dev/trunk/lucene/backward-codecs/src/resources/META-INF/services/org.apache.lucene.codecs.Codec
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/backward-codecs/src/resources/META-INF/services/org.apache.lucene.codecs.Codec?rev=1710876&r1=1710875&r2=1710876&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/backward-codecs/src/resources/META-INF/services/org.apache.lucene.codecs.Codec (original)
+++ lucene/dev/trunk/lucene/backward-codecs/src/resources/META-INF/services/org.apache.lucene.codecs.Codec Tue Oct 27 19:49:47 2015
@@ -14,3 +14,4 @@
 #  limitations under the License.
 
 org.apache.lucene.codecs.lucene50.Lucene50Codec
+org.apache.lucene.codecs.lucene53.Lucene53Codec

Modified: lucene/dev/trunk/lucene/backward-codecs/src/resources/META-INF/services/org.apache.lucene.codecs.DocValuesFormat
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/backward-codecs/src/resources/META-INF/services/org.apache.lucene.codecs.DocValuesFormat?rev=1710876&r1=1710875&r2=1710876&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/backward-codecs/src/resources/META-INF/services/org.apache.lucene.codecs.DocValuesFormat (original)
+++ lucene/dev/trunk/lucene/backward-codecs/src/resources/META-INF/services/org.apache.lucene.codecs.DocValuesFormat Tue Oct 27 19:49:47 2015
@@ -13,4 +13,4 @@
 #  See the License for the specific language governing permissions and
 #  limitations under the License.
 
-
+org.apache.lucene.codecs.lucene50.Lucene50DocValuesFormat

Added: lucene/dev/trunk/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene50/TestLucene50DocValuesFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene50/TestLucene50DocValuesFormat.java?rev=1710876&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene50/TestLucene50DocValuesFormat.java (added)
+++ lucene/dev/trunk/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene50/TestLucene50DocValuesFormat.java Tue Oct 27 19:49:47 2015
@@ -0,0 +1,281 @@
+package org.apache.lucene.codecs.lucene50;
+
+/*
+ * 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.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.lucene.analysis.MockAnalyzer;
+import org.apache.lucene.codecs.Codec;
+import org.apache.lucene.codecs.DocValuesFormat;
+import org.apache.lucene.codecs.PostingsFormat;
+import org.apache.lucene.codecs.asserting.AssertingCodec;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.document.SortedSetDocValuesField;
+import org.apache.lucene.document.StringField;
+import org.apache.lucene.index.BaseCompressingDocValuesFormatTestCase;
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.IndexWriterConfig;
+import org.apache.lucene.index.LeafReader;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.RandomIndexWriter;
+import org.apache.lucene.index.SerialMergeScheduler;
+import org.apache.lucene.index.SortedSetDocValues;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.index.Terms;
+import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.index.TermsEnum.SeekStatus;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.TestUtil;
+
+/**
+ * Tests Lucene50DocValuesFormat
+ */
+public class TestLucene50DocValuesFormat extends BaseCompressingDocValuesFormatTestCase {
+  private final Codec codec = TestUtil.alwaysDocValuesFormat(new Lucene50DocValuesFormat());
+
+  @Override
+  protected Codec getCodec() {
+    return codec;
+  }
+  
+  // TODO: these big methods can easily blow up some of the other ram-hungry codecs...
+  // for now just keep them here, as we want to test this for this format.
+  
+  @Slow
+  public void testSortedSetVariableLengthBigVsStoredFields() throws Exception {
+    int numIterations = atLeast(1);
+    for (int i = 0; i < numIterations; i++) {
+      doTestSortedSetVsStoredFields(atLeast(300), 1, 32766, 16, 100);
+    }
+  }
+  
+  @Nightly
+  public void testSortedSetVariableLengthManyVsStoredFields() throws Exception {
+    int numIterations = atLeast(1);
+    for (int i = 0; i < numIterations; i++) {
+      doTestSortedSetVsStoredFields(TestUtil.nextInt(random(), 1024, 2049), 1, 500, 16, 100);
+    }
+  }
+  
+  @Slow
+  public void testSortedVariableLengthBigVsStoredFields() throws Exception {
+    int numIterations = atLeast(1);
+    for (int i = 0; i < numIterations; i++) {
+      doTestSortedVsStoredFields(atLeast(300), 1, 32766);
+    }
+  }
+  
+  @Nightly
+  public void testSortedVariableLengthManyVsStoredFields() throws Exception {
+    int numIterations = atLeast(1);
+    for (int i = 0; i < numIterations; i++) {
+      doTestSortedVsStoredFields(TestUtil.nextInt(random(), 1024, 2049), 1, 500);
+    }
+  }
+  
+  @Slow
+  public void testTermsEnumFixedWidth() throws Exception {
+    int numIterations = atLeast(1);
+    for (int i = 0; i < numIterations; i++) {
+      doTestTermsEnumRandom(TestUtil.nextInt(random(), 1025, 5121), 10, 10);
+    }
+  }
+  
+  @Slow
+  public void testTermsEnumVariableWidth() throws Exception {
+    int numIterations = atLeast(1);
+    for (int i = 0; i < numIterations; i++) {
+      doTestTermsEnumRandom(TestUtil.nextInt(random(), 1025, 5121), 1, 500);
+    }
+  }
+  
+  @Nightly
+  public void testTermsEnumRandomMany() throws Exception {
+    int numIterations = atLeast(1);
+    for (int i = 0; i < numIterations; i++) {
+      doTestTermsEnumRandom(TestUtil.nextInt(random(), 1025, 8121), 1, 500);
+    }
+  }
+  
+  // TODO: try to refactor this and some termsenum tests into the base class.
+  // to do this we need to fix the test class to get a DVF not a Codec so we can setup
+  // the postings format correctly.
+  private void doTestTermsEnumRandom(int numDocs, int minLength, int maxLength) throws Exception {
+    Directory dir = newFSDirectory(createTempDir());
+    IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random()));
+    conf.setMergeScheduler(new SerialMergeScheduler());
+    // set to duel against a codec which has ordinals:
+    final PostingsFormat pf = TestUtil.getPostingsFormatWithOrds(random());
+    final DocValuesFormat dv = new Lucene50DocValuesFormat();
+    conf.setCodec(new AssertingCodec() {
+      @Override
+      public PostingsFormat getPostingsFormatForField(String field) {
+        return pf;
+      }
+
+      @Override
+      public DocValuesFormat getDocValuesFormatForField(String field) {
+        return dv;
+      }
+    });
+    RandomIndexWriter writer = new RandomIndexWriter(random(), dir, conf);
+    
+    // index some docs
+    for (int i = 0; i < numDocs; i++) {
+      Document doc = new Document();
+      Field idField = new StringField("id", Integer.toString(i), Field.Store.NO);
+      doc.add(idField);
+      final int length = TestUtil.nextInt(random(), minLength, maxLength);
+      int numValues = random().nextInt(17);
+      // create a random list of strings
+      List<String> values = new ArrayList<>();
+      for (int v = 0; v < numValues; v++) {
+        values.add(TestUtil.randomSimpleString(random(), minLength, length));
+      }
+      
+      // add in any order to the indexed field
+      ArrayList<String> unordered = new ArrayList<>(values);
+      Collections.shuffle(unordered, random());
+      for (String v : values) {
+        doc.add(newStringField("indexed", v, Field.Store.NO));
+      }
+
+      // add in any order to the dv field
+      ArrayList<String> unordered2 = new ArrayList<>(values);
+      Collections.shuffle(unordered2, random());
+      for (String v : unordered2) {
+        doc.add(new SortedSetDocValuesField("dv", new BytesRef(v)));
+      }
+
+      writer.addDocument(doc);
+      if (random().nextInt(31) == 0) {
+        writer.commit();
+      }
+    }
+    
+    // delete some docs
+    int numDeletions = random().nextInt(numDocs/10);
+    for (int i = 0; i < numDeletions; i++) {
+      int id = random().nextInt(numDocs);
+      writer.deleteDocuments(new Term("id", Integer.toString(id)));
+    }
+    
+    // compare per-segment
+    DirectoryReader ir = writer.getReader();
+    for (LeafReaderContext context : ir.leaves()) {
+      LeafReader r = context.reader();
+      Terms terms = r.terms("indexed");
+      if (terms != null) {
+        SortedSetDocValues ssdv = r.getSortedSetDocValues("dv");
+        assertEquals(terms.size(), ssdv.getValueCount());
+        TermsEnum expected = terms.iterator();
+        TermsEnum actual = r.getSortedSetDocValues("dv").termsEnum();
+        assertEquals(terms.size(), expected, actual);
+
+        doTestSortedSetEnumAdvanceIndependently(ssdv);
+      }
+    }
+    ir.close();
+    
+    writer.forceMerge(1);
+    
+    // now compare again after the merge
+    ir = writer.getReader();
+    LeafReader ar = getOnlySegmentReader(ir);
+    Terms terms = ar.terms("indexed");
+    if (terms != null) {
+      assertEquals(terms.size(), ar.getSortedSetDocValues("dv").getValueCount());
+      TermsEnum expected = terms.iterator();
+      TermsEnum actual = ar.getSortedSetDocValues("dv").termsEnum();
+      assertEquals(terms.size(), expected, actual);
+    }
+    ir.close();
+    
+    writer.close();
+    dir.close();
+  }
+  
+  private void assertEquals(long numOrds, TermsEnum expected, TermsEnum actual) throws Exception {
+    BytesRef ref;
+    
+    // sequential next() through all terms
+    while ((ref = expected.next()) != null) {
+      assertEquals(ref, actual.next());
+      assertEquals(expected.ord(), actual.ord());
+      assertEquals(expected.term(), actual.term());
+    }
+    assertNull(actual.next());
+    
+    // sequential seekExact(ord) through all terms
+    for (long i = 0; i < numOrds; i++) {
+      expected.seekExact(i);
+      actual.seekExact(i);
+      assertEquals(expected.ord(), actual.ord());
+      assertEquals(expected.term(), actual.term());
+    }
+    
+    // sequential seekExact(BytesRef) through all terms
+    for (long i = 0; i < numOrds; i++) {
+      expected.seekExact(i);
+      assertTrue(actual.seekExact(expected.term()));
+      assertEquals(expected.ord(), actual.ord());
+      assertEquals(expected.term(), actual.term());
+    }
+    
+    // sequential seekCeil(BytesRef) through all terms
+    for (long i = 0; i < numOrds; i++) {
+      expected.seekExact(i);
+      assertEquals(SeekStatus.FOUND, actual.seekCeil(expected.term()));
+      assertEquals(expected.ord(), actual.ord());
+      assertEquals(expected.term(), actual.term());
+    }
+    
+    // random seekExact(ord)
+    for (long i = 0; i < numOrds; i++) {
+      long randomOrd = TestUtil.nextLong(random(), 0, numOrds - 1);
+      expected.seekExact(randomOrd);
+      actual.seekExact(randomOrd);
+      assertEquals(expected.ord(), actual.ord());
+      assertEquals(expected.term(), actual.term());
+    }
+    
+    // random seekExact(BytesRef)
+    for (long i = 0; i < numOrds; i++) {
+      long randomOrd = TestUtil.nextLong(random(), 0, numOrds - 1);
+      expected.seekExact(randomOrd);
+      actual.seekExact(expected.term());
+      assertEquals(expected.ord(), actual.ord());
+      assertEquals(expected.term(), actual.term());
+    }
+    
+    // random seekCeil(BytesRef)
+    for (long i = 0; i < numOrds; i++) {
+      BytesRef target = new BytesRef(TestUtil.randomUnicodeString(random()));
+      SeekStatus expectedStatus = expected.seekCeil(target);
+      assertEquals(expectedStatus, actual.seekCeil(target));
+      if (expectedStatus != SeekStatus.END) {
+        assertEquals(expected.ord(), actual.ord());
+        assertEquals(expected.term(), actual.term());
+      }
+    }
+  }
+}

Modified: lucene/dev/trunk/lucene/benchmark/src/java/org/apache/lucene/benchmark/byTask/tasks/CreateIndexTask.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/benchmark/src/java/org/apache/lucene/benchmark/byTask/tasks/CreateIndexTask.java?rev=1710876&r1=1710875&r2=1710876&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/benchmark/src/java/org/apache/lucene/benchmark/byTask/tasks/CreateIndexTask.java (original)
+++ lucene/dev/trunk/lucene/benchmark/src/java/org/apache/lucene/benchmark/byTask/tasks/CreateIndexTask.java Tue Oct 27 19:49:47 2015
@@ -29,7 +29,7 @@ import org.apache.lucene.benchmark.byTas
 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.lucene53.Lucene53Codec;
+import org.apache.lucene.codecs.lucene54.Lucene54Codec;
 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 Per
     if (defaultCodec == null && postingsFormat != null) {
       try {
         final PostingsFormat postingsFormatChosen = PostingsFormat.forName(postingsFormat);
-        iwConf.setCodec(new Lucene53Codec() {
+        iwConf.setCodec(new Lucene54Codec() {
           @Override
           public PostingsFormat getPostingsFormatForField(String field) {
             return postingsFormatChosen;

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/Codec.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/Codec.java?rev=1710876&r1=1710875&r2=1710876&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/Codec.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/Codec.java Tue Oct 27 19:49:47 2015
@@ -57,7 +57,7 @@ public abstract class Codec implements N
     }
     
     // TODO: should we use this, or maybe a system property is better?
-    static Codec defaultCodec = LOADER.lookup("Lucene53");
+    static Codec defaultCodec = LOADER.lookup("Lucene54");
   }
 
   private final String name;

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50StoredFieldsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50StoredFieldsFormat.java?rev=1710876&r1=1710875&r2=1710876&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50StoredFieldsFormat.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50StoredFieldsFormat.java Tue Oct 27 19:49:47 2015
@@ -53,9 +53,9 @@ import org.apache.lucene.util.packed.Pac
  * These two options can be configured like this:
  * <pre class="prettyprint">
  *   // the default: for high performance
- *   indexWriterConfig.setCodec(new Lucene53Codec(Mode.BEST_SPEED));
+ *   indexWriterConfig.setCodec(new Lucene54Codec(Mode.BEST_SPEED));
  *   // instead for higher performance (but slower):
- *   // indexWriterConfig.setCodec(new Lucene53Codec(Mode.BEST_COMPRESSION));
+ *   // indexWriterConfig.setCodec(new Lucene54Codec(Mode.BEST_COMPRESSION));
  * </pre>
  * <p><b>File formats</b>
  * <p>Stored fields are represented by two files:

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene53/package-info.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene53/package-info.java?rev=1710876&r1=1710875&r2=1710876&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene53/package-info.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene53/package-info.java Tue Oct 27 19:49:47 2015
@@ -16,386 +16,8 @@
  */
 
 /**
- * Lucene 5.3 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.lucene50.Lucene50DocValuesFormat 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>
- * </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.lucene50.Lucene50DocValuesFormat 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>
- * </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>
- * </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 5.3 index format
+ * See {@link org.apache.lucene.codecs.lucene54} for an overview
+ * of the index format.
  */
 package org.apache.lucene.codecs.lucene53;

Added: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene54/Lucene54Codec.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene54/Lucene54Codec.java?rev=1710876&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene54/Lucene54Codec.java (added)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene54/Lucene54Codec.java Tue Oct 27 19:49:47 2015
@@ -0,0 +1,176 @@
+package org.apache.lucene.codecs.lucene54;
+
+/*
+ * 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.util.Objects;
+
+import org.apache.lucene.codecs.Codec;
+import org.apache.lucene.codecs.CompoundFormat;
+import org.apache.lucene.codecs.DimensionalFormat;
+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.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.Lucene50FieldInfosFormat;
+import org.apache.lucene.codecs.lucene50.Lucene50LiveDocsFormat;
+import org.apache.lucene.codecs.lucene50.Lucene50SegmentInfoFormat;
+import org.apache.lucene.codecs.lucene50.Lucene50StoredFieldsFormat;
+import org.apache.lucene.codecs.lucene50.Lucene50StoredFieldsFormat.Mode;
+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 5.4 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.lucene54 package documentation for file format details.
+ * @lucene.experimental
+ */
+public class Lucene54Codec extends Codec {
+  private final TermVectorsFormat vectorsFormat = new Lucene50TermVectorsFormat();
+  private final FieldInfosFormat fieldInfosFormat = new Lucene50FieldInfosFormat();
+  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 Lucene54Codec.this.getPostingsFormatForField(field);
+    }
+  };
+
+  private final DocValuesFormat docValuesFormat = new PerFieldDocValuesFormat() {
+    @Override
+    public DocValuesFormat getDocValuesFormatForField(String field) {
+      return Lucene54Codec.this.getDocValuesFormatForField(field);
+    }
+  };
+
+  private final StoredFieldsFormat storedFieldsFormat;
+
+  /**
+   * Instantiates a new codec.
+   */
+  public Lucene54Codec() {
+    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 Lucene54Codec(Mode mode) {
+    super("Lucene54");
+    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;
+  }
+
+  /** 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;
+  }
+
+  @Override
+  public final DimensionalFormat dimensionalFormat() {
+    return DimensionalFormat.EMPTY;
+  }
+
+  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;
+  }
+}

Added: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene54/Lucene54DocValuesConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene54/Lucene54DocValuesConsumer.java?rev=1710876&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene54/Lucene54DocValuesConsumer.java (added)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene54/Lucene54DocValuesConsumer.java Tue Oct 27 19:49:47 2015
@@ -0,0 +1,658 @@
+package org.apache.lucene.codecs.lucene54;
+
+/*
+ * 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; // javadocs
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Set;
+import java.util.SortedSet;
+import java.util.TreeSet;
+
+import org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.codecs.DocValuesConsumer;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.index.SegmentWriteState;
+import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.store.RAMOutputStream;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefBuilder;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.LongsRef;
+import org.apache.lucene.util.MathUtil;
+import org.apache.lucene.util.PagedBytes;
+import org.apache.lucene.util.PagedBytes.PagedBytesDataInput;
+import org.apache.lucene.util.StringHelper;
+import org.apache.lucene.util.packed.DirectMonotonicWriter;
+import org.apache.lucene.util.packed.DirectWriter;
+import org.apache.lucene.util.packed.MonotonicBlockPackedWriter;
+import org.apache.lucene.util.packed.PackedInts;
+
+import static org.apache.lucene.codecs.lucene54.Lucene54DocValuesFormat.*;
+
+/** writer for {@link Lucene54DocValuesFormat} */
+final class Lucene54DocValuesConsumer extends DocValuesConsumer implements Closeable {
+
+  IndexOutput data, meta;
+  final int maxDoc;
+  
+  /** expert: Creates a new writer */
+  public Lucene54DocValuesConsumer(SegmentWriteState state, String dataCodec, String dataExtension, String metaCodec, String metaExtension) throws IOException {
+    boolean success = false;
+    try {
+      String dataName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, dataExtension);
+      data = state.directory.createOutput(dataName, state.context);
+      CodecUtil.writeIndexHeader(data, dataCodec, Lucene54DocValuesFormat.VERSION_CURRENT, state.segmentInfo.getId(), state.segmentSuffix);
+      String metaName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, metaExtension);
+      meta = state.directory.createOutput(metaName, state.context);
+      CodecUtil.writeIndexHeader(meta, metaCodec, Lucene54DocValuesFormat.VERSION_CURRENT, state.segmentInfo.getId(), state.segmentSuffix);
+      maxDoc = state.segmentInfo.maxDoc();
+      success = true;
+    } finally {
+      if (!success) {
+        IOUtils.closeWhileHandlingException(this);
+      }
+    }
+  }
+  
+  @Override
+  public void addNumericField(FieldInfo field, Iterable<Number> values) throws IOException {
+    addNumericField(field, values, true);
+  }
+
+  void addNumericField(FieldInfo field, Iterable<Number> values, boolean optimizeStorage) throws IOException {
+    long count = 0;
+    long minValue = Long.MAX_VALUE;
+    long maxValue = Long.MIN_VALUE;
+    long gcd = 0;
+    long missingCount = 0;
+    long zeroCount = 0;
+    // TODO: more efficient?
+    HashSet<Long> uniqueValues = null;
+    if (optimizeStorage) {
+      uniqueValues = new HashSet<>();
+
+      for (Number nv : values) {
+        final long v;
+        if (nv == null) {
+          v = 0;
+          missingCount++;
+          zeroCount++;
+        } else {
+          v = nv.longValue();
+          if (v == 0) {
+            zeroCount++;
+          }
+        }
+
+        if (gcd != 1) {
+          if (v < Long.MIN_VALUE / 2 || v > Long.MAX_VALUE / 2) {
+            // in that case v - minValue might overflow and make the GCD computation return
+            // wrong results. Since these extreme values are unlikely, we just discard
+            // GCD computation for them
+            gcd = 1;
+          } else if (count != 0) { // minValue needs to be set first
+            gcd = MathUtil.gcd(gcd, v - minValue);
+          }
+        }
+
+        minValue = Math.min(minValue, v);
+        maxValue = Math.max(maxValue, v);
+
+        if (uniqueValues != null) {
+          if (uniqueValues.add(v)) {
+            if (uniqueValues.size() > 256) {
+              uniqueValues = null;
+            }
+          }
+        }
+
+        ++count;
+      }
+    } else {
+      for (Number nv : values) {
+        long v = nv.longValue();
+        minValue = Math.min(minValue, v);
+        maxValue = Math.max(maxValue, v);
+        ++count;
+      }
+    }
+    
+    final long delta = maxValue - minValue;
+    final int deltaBitsRequired = DirectWriter.unsignedBitsRequired(delta);
+    final int tableBitsRequired = uniqueValues == null
+        ? Integer.MAX_VALUE
+        : DirectWriter.bitsRequired(uniqueValues.size() - 1);
+
+    final int format;
+    if (uniqueValues != null 
+        && count <= Integer.MAX_VALUE
+        && (uniqueValues.size() == 1
+           || (uniqueValues.size() == 2 && missingCount > 0 && zeroCount == missingCount))) {
+      // either one unique value C or two unique values: "missing" and C
+      format = CONST_COMPRESSED;
+    } else if (uniqueValues != null && tableBitsRequired < deltaBitsRequired) {
+      format = TABLE_COMPRESSED;
+    } else if (gcd != 0 && gcd != 1) {
+      final long gcdDelta = (maxValue - minValue) / gcd;
+      final long gcdBitsRequired = DirectWriter.unsignedBitsRequired(gcdDelta);
+      format = gcdBitsRequired < deltaBitsRequired ? GCD_COMPRESSED : DELTA_COMPRESSED;
+    } else {
+      format = DELTA_COMPRESSED;
+    }
+    meta.writeVInt(field.number);
+    meta.writeByte(Lucene54DocValuesFormat.NUMERIC);
+    meta.writeVInt(format);
+    if (missingCount == 0) {
+      meta.writeLong(ALL_LIVE);
+    } else if (missingCount == count) {
+      meta.writeLong(ALL_MISSING);
+    } else {
+      meta.writeLong(data.getFilePointer());
+      writeMissingBitset(values);
+    }
+    meta.writeLong(data.getFilePointer());
+    meta.writeVLong(count);
+
+    switch (format) {
+      case CONST_COMPRESSED:
+        // write the constant (nonzero value in the n=2 case, singleton value otherwise)
+        meta.writeLong(minValue < 0 ? Collections.min(uniqueValues) : Collections.max(uniqueValues));
+        break;
+      case GCD_COMPRESSED:
+        meta.writeLong(minValue);
+        meta.writeLong(gcd);
+        final long maxDelta = (maxValue - minValue) / gcd;
+        final int bits = DirectWriter.unsignedBitsRequired(maxDelta);
+        meta.writeVInt(bits);
+        final DirectWriter quotientWriter = DirectWriter.getInstance(data, count, bits);
+        for (Number nv : values) {
+          long value = nv == null ? 0 : nv.longValue();
+          quotientWriter.add((value - minValue) / gcd);
+        }
+        quotientWriter.finish();
+        break;
+      case DELTA_COMPRESSED:
+        final long minDelta = delta < 0 ? 0 : minValue;
+        meta.writeLong(minDelta);
+        meta.writeVInt(deltaBitsRequired);
+        final DirectWriter writer = DirectWriter.getInstance(data, count, deltaBitsRequired);
+        for (Number nv : values) {
+          long v = nv == null ? 0 : nv.longValue();
+          writer.add(v - minDelta);
+        }
+        writer.finish();
+        break;
+      case TABLE_COMPRESSED:
+        final Long[] decode = uniqueValues.toArray(new Long[uniqueValues.size()]);
+        Arrays.sort(decode);
+        final HashMap<Long,Integer> encode = new HashMap<>();
+        meta.writeVInt(decode.length);
+        for (int i = 0; i < decode.length; i++) {
+          meta.writeLong(decode[i]);
+          encode.put(decode[i], i);
+        }
+        meta.writeVInt(tableBitsRequired);
+        final DirectWriter ordsWriter = DirectWriter.getInstance(data, count, tableBitsRequired);
+        for (Number nv : values) {
+          ordsWriter.add(encode.get(nv == null ? 0 : nv.longValue()));
+        }
+        ordsWriter.finish();
+        break;
+      default:
+        throw new AssertionError();
+    }
+    meta.writeLong(data.getFilePointer());
+  }
+  
+  // TODO: in some cases representing missing with minValue-1 wouldn't take up additional space and so on,
+  // but this is very simple, and algorithms only check this for values of 0 anyway (doesnt slow down normal decode)
+  void writeMissingBitset(Iterable<?> values) throws IOException {
+    byte bits = 0;
+    int count = 0;
+    for (Object v : values) {
+      if (count == 8) {
+        data.writeByte(bits);
+        count = 0;
+        bits = 0;
+      }
+      if (v != null) {
+        bits |= 1 << (count & 7);
+      }
+      count++;
+    }
+    if (count > 0) {
+      data.writeByte(bits);
+    }
+  }
+
+  @Override
+  public void addBinaryField(FieldInfo field, Iterable<BytesRef> values) throws IOException {
+    // write the byte[] data
+    meta.writeVInt(field.number);
+    meta.writeByte(Lucene54DocValuesFormat.BINARY);
+    int minLength = Integer.MAX_VALUE;
+    int maxLength = Integer.MIN_VALUE;
+    final long startFP = data.getFilePointer();
+    long count = 0;
+    long missingCount = 0;
+    for(BytesRef v : values) {
+      final int length;
+      if (v == null) {
+        length = 0;
+        missingCount++;
+      } else {
+        length = v.length;
+      }
+      minLength = Math.min(minLength, length);
+      maxLength = Math.max(maxLength, length);
+      if (v != null) {
+        data.writeBytes(v.bytes, v.offset, v.length);
+      }
+      count++;
+    }
+    meta.writeVInt(minLength == maxLength ? BINARY_FIXED_UNCOMPRESSED : BINARY_VARIABLE_UNCOMPRESSED);
+    if (missingCount == 0) {
+      meta.writeLong(ALL_LIVE);
+    } else if (missingCount == count) {
+      meta.writeLong(ALL_MISSING);
+    } else {
+      meta.writeLong(data.getFilePointer());
+      writeMissingBitset(values);
+    }
+    meta.writeVInt(minLength);
+    meta.writeVInt(maxLength);
+    meta.writeVLong(count);
+    meta.writeLong(startFP);
+    
+    // if minLength == maxLength, it's a fixed-length byte[], we are done (the addresses are implicit)
+    // otherwise, we need to record the length fields...
+    if (minLength != maxLength) {
+      meta.writeLong(data.getFilePointer());
+      meta.writeVInt(DIRECT_MONOTONIC_BLOCK_SHIFT);
+
+      final DirectMonotonicWriter writer = DirectMonotonicWriter.getInstance(meta, data, count + 1, DIRECT_MONOTONIC_BLOCK_SHIFT);
+      long addr = 0;
+      writer.add(addr);
+      for (BytesRef v : values) {
+        if (v != null) {
+          addr += v.length;
+        }
+        writer.add(addr);
+      }
+      writer.finish();
+      meta.writeLong(data.getFilePointer());
+    }
+  }
+  
+  /** expert: writes a value dictionary for a sorted/sortedset field */
+  private void addTermsDict(FieldInfo field, final Iterable<BytesRef> values) throws IOException {
+    // first check if it's a "fixed-length" terms dict
+    int minLength = Integer.MAX_VALUE;
+    int maxLength = Integer.MIN_VALUE;
+    long numValues = 0;
+    for (BytesRef v : values) {
+      minLength = Math.min(minLength, v.length);
+      maxLength = Math.max(maxLength, v.length);
+      numValues++;
+    }
+    if (minLength == maxLength) {
+      // no index needed: direct addressing by mult
+      addBinaryField(field, values);
+    } else if (numValues < REVERSE_INTERVAL_COUNT) {
+      // low cardinality: waste a few KB of ram, but can't really use fancy index etc
+      addBinaryField(field, values);
+    } else {
+      assert numValues > 0; // we don't have to handle the empty case
+      // header
+      meta.writeVInt(field.number);
+      meta.writeByte(Lucene54DocValuesFormat.BINARY);
+      meta.writeVInt(BINARY_PREFIX_COMPRESSED);
+      meta.writeLong(-1L);
+      // now write the bytes: sharing prefixes within a block
+      final long startFP = data.getFilePointer();
+      // currently, we have to store the delta from expected for every 1/nth term
+      // we could avoid this, but it's not much and less overall RAM than the previous approach!
+      RAMOutputStream addressBuffer = new RAMOutputStream();
+      MonotonicBlockPackedWriter termAddresses = new MonotonicBlockPackedWriter(addressBuffer, MONOTONIC_BLOCK_SIZE);
+      // buffers up 16 terms
+      RAMOutputStream bytesBuffer = new RAMOutputStream();
+      // buffers up block header
+      RAMOutputStream headerBuffer = new RAMOutputStream();
+      BytesRefBuilder lastTerm = new BytesRefBuilder();
+      lastTerm.grow(maxLength);
+      long count = 0;
+      int suffixDeltas[] = new int[INTERVAL_COUNT];
+      for (BytesRef v : values) {
+        int termPosition = (int) (count & INTERVAL_MASK);
+        if (termPosition == 0) {
+          termAddresses.add(data.getFilePointer() - startFP);
+          // abs-encode first term
+          headerBuffer.writeVInt(v.length);
+          headerBuffer.writeBytes(v.bytes, v.offset, v.length);
+          lastTerm.copyBytes(v);
+        } else {
+          // prefix-code: we only share at most 255 characters, to encode the length as a single
+          // byte and have random access. Larger terms just get less compression.
+          int sharedPrefix = Math.min(255, StringHelper.bytesDifference(lastTerm.get(), v));
+          bytesBuffer.writeByte((byte) sharedPrefix);
+          bytesBuffer.writeBytes(v.bytes, v.offset + sharedPrefix, v.length - sharedPrefix);
+          // we can encode one smaller, because terms are unique.
+          suffixDeltas[termPosition] = v.length - sharedPrefix - 1;
+        }
+        
+        count++;
+        // flush block
+        if ((count & INTERVAL_MASK) == 0) {
+          flushTermsDictBlock(headerBuffer, bytesBuffer, suffixDeltas);
+        }
+      }
+      // flush trailing crap
+      int leftover = (int) (count & INTERVAL_MASK);
+      if (leftover > 0) {
+        Arrays.fill(suffixDeltas, leftover, suffixDeltas.length, 0);
+        flushTermsDictBlock(headerBuffer, bytesBuffer, suffixDeltas);
+      }
+      final long indexStartFP = data.getFilePointer();
+      // write addresses of indexed terms
+      termAddresses.finish();
+      addressBuffer.writeTo(data);
+      addressBuffer = null;
+      termAddresses = null;
+      meta.writeVInt(minLength);
+      meta.writeVInt(maxLength);
+      meta.writeVLong(count);
+      meta.writeLong(startFP);
+      meta.writeLong(indexStartFP);
+      meta.writeVInt(PackedInts.VERSION_CURRENT);
+      meta.writeVInt(MONOTONIC_BLOCK_SIZE);
+      addReverseTermIndex(field, values, maxLength);
+    }
+  }
+  
+  // writes term dictionary "block"
+  // first term is absolute encoded as vint length + bytes.
+  // lengths of subsequent N terms are encoded as either N bytes or N shorts.
+  // in the double-byte case, the first byte is indicated with -1.
+  // subsequent terms are encoded as byte suffixLength + bytes.
+  private void flushTermsDictBlock(RAMOutputStream headerBuffer, RAMOutputStream bytesBuffer, int suffixDeltas[]) throws IOException {
+    boolean twoByte = false;
+    for (int i = 1; i < suffixDeltas.length; i++) {
+      if (suffixDeltas[i] > 254) {
+        twoByte = true;
+      }
+    }
+    if (twoByte) {
+      headerBuffer.writeByte((byte)255);
+      for (int i = 1; i < suffixDeltas.length; i++) {
+        headerBuffer.writeShort((short) suffixDeltas[i]);
+      }
+    } else {
+      for (int i = 1; i < suffixDeltas.length; i++) {
+        headerBuffer.writeByte((byte) suffixDeltas[i]);
+      }
+    }
+    headerBuffer.writeTo(data);
+    headerBuffer.reset();
+    bytesBuffer.writeTo(data);
+    bytesBuffer.reset();
+  }
+  
+  // writes reverse term index: used for binary searching a term into a range of 64 blocks
+  // for every 64 blocks (1024 terms) we store a term, trimming any suffix unnecessary for comparison
+  // terms are written as a contiguous byte[], but never spanning 2^15 byte boundaries.
+  private void addReverseTermIndex(FieldInfo field, final Iterable<BytesRef> values, int maxLength) throws IOException {
+    long count = 0;
+    BytesRefBuilder priorTerm = new BytesRefBuilder();
+    priorTerm.grow(maxLength);
+    BytesRef indexTerm = new BytesRef();
+    long startFP = data.getFilePointer();
+    PagedBytes pagedBytes = new PagedBytes(15);
+    MonotonicBlockPackedWriter addresses = new MonotonicBlockPackedWriter(data, MONOTONIC_BLOCK_SIZE);
+    
+    for (BytesRef b : values) {
+      int termPosition = (int) (count & REVERSE_INTERVAL_MASK);
+      if (termPosition == 0) {
+        int len = StringHelper.sortKeyLength(priorTerm.get(), b);
+        indexTerm.bytes = b.bytes;
+        indexTerm.offset = b.offset;
+        indexTerm.length = len;
+        addresses.add(pagedBytes.copyUsingLengthPrefix(indexTerm));
+      } else if (termPosition == REVERSE_INTERVAL_MASK) {
+        priorTerm.copyBytes(b);
+      }
+      count++;
+    }
+    addresses.finish();
+    long numBytes = pagedBytes.getPointer();
+    pagedBytes.freeze(true);
+    PagedBytesDataInput in = pagedBytes.getDataInput();
+    meta.writeLong(startFP);
+    data.writeVLong(numBytes);
+    data.copyBytes(in, numBytes);
+  }
+
+  @Override
+  public void addSortedField(FieldInfo field, Iterable<BytesRef> values, Iterable<Number> docToOrd) throws IOException {
+    meta.writeVInt(field.number);
+    meta.writeByte(Lucene54DocValuesFormat.SORTED);
+    addTermsDict(field, values);
+    addNumericField(field, docToOrd, false);
+  }
+
+  @Override
+  public void addSortedNumericField(FieldInfo field, final Iterable<Number> docToValueCount, final Iterable<Number> values) throws IOException {
+    meta.writeVInt(field.number);
+    meta.writeByte(Lucene54DocValuesFormat.SORTED_NUMERIC);
+    if (isSingleValued(docToValueCount)) {
+      meta.writeVInt(SORTED_SINGLE_VALUED);
+      // The field is single-valued, we can encode it as NUMERIC
+      addNumericField(field, singletonView(docToValueCount, values, null));
+    } else {
+      final SortedSet<LongsRef> uniqueValueSets = uniqueValueSets(docToValueCount, values);
+      if (uniqueValueSets != null) {
+        meta.writeVInt(SORTED_SET_TABLE);
+
+        // write the set_id -> values mapping
+        writeDictionary(uniqueValueSets);
+
+        // write the doc -> set_id as a numeric field
+        addNumericField(field, docToSetId(uniqueValueSets, docToValueCount, values), false);
+      } else {
+        meta.writeVInt(SORTED_WITH_ADDRESSES);
+        // write the stream of values as a numeric field
+        addNumericField(field, values, true);
+        // write the doc -> ord count as a absolute index to the stream
+        addOrdIndex(field, docToValueCount);
+      }
+    }
+  }
+
+  @Override
+  public void addSortedSetField(FieldInfo field, Iterable<BytesRef> values, final Iterable<Number> docToOrdCount, final Iterable<Number> ords) throws IOException {
+    meta.writeVInt(field.number);
+    meta.writeByte(Lucene54DocValuesFormat.SORTED_SET);
+
+    if (isSingleValued(docToOrdCount)) {
+      meta.writeVInt(SORTED_SINGLE_VALUED);
+      // The field is single-valued, we can encode it as SORTED
+      addSortedField(field, values, singletonView(docToOrdCount, ords, -1L));
+    } else {
+      final SortedSet<LongsRef> uniqueValueSets = uniqueValueSets(docToOrdCount, ords);
+      if (uniqueValueSets != null) {
+        meta.writeVInt(SORTED_SET_TABLE);
+
+        // write the set_id -> ords mapping
+        writeDictionary(uniqueValueSets);
+
+        // write the ord -> byte[] as a binary field
+        addTermsDict(field, values);
+
+        // write the doc -> set_id as a numeric field
+        addNumericField(field, docToSetId(uniqueValueSets, docToOrdCount, ords), false);
+      } else {
+        meta.writeVInt(SORTED_WITH_ADDRESSES);
+
+        // write the ord -> byte[] as a binary field
+        addTermsDict(field, values);
+
+        // write the stream of ords as a numeric field
+        // NOTE: we could return an iterator that delta-encodes these within a doc
+        addNumericField(field, ords, false);
+
+        // write the doc -> ord count as a absolute index to the stream
+        addOrdIndex(field, docToOrdCount);
+      }
+    }
+  }
+
+  private SortedSet<LongsRef> uniqueValueSets(Iterable<Number> docToValueCount, Iterable<Number> values) {
+    Set<LongsRef> uniqueValueSet = new HashSet<>();
+    LongsRef docValues = new LongsRef(256);
+
+    Iterator<Number> valueCountIterator = docToValueCount.iterator();
+    Iterator<Number> valueIterator = values.iterator();
+    int totalDictSize = 0;
+    while (valueCountIterator.hasNext()) {
+      docValues.length = valueCountIterator.next().intValue();
+      if (docValues.length > 256) {
+        return null;
+      }
+      for (int i = 0; i < docValues.length; ++i) {
+        docValues.longs[i] = valueIterator.next().longValue();
+      }
+      if (uniqueValueSet.contains(docValues)) {
+        continue;
+      }
+      totalDictSize += docValues.length;
+      if (totalDictSize > 256) {
+        return null;
+      }
+      uniqueValueSet.add(new LongsRef(Arrays.copyOf(docValues.longs, docValues.length), 0, docValues.length));
+    }
+    assert valueIterator.hasNext() == false;
+    return new TreeSet<>(uniqueValueSet);
+  }
+
+  private void writeDictionary(SortedSet<LongsRef> uniqueValueSets) throws IOException {
+    int lengthSum = 0;
+    for (LongsRef longs : uniqueValueSets) {
+      lengthSum += longs.length;
+    }
+
+    meta.writeInt(lengthSum);
+    for (LongsRef valueSet : uniqueValueSets) {
+      for (int  i = 0; i < valueSet.length; ++i) {
+        meta.writeLong(valueSet.longs[valueSet.offset + i]);
+      }
+    }
+
+    meta.writeInt(uniqueValueSets.size());
+    for (LongsRef valueSet : uniqueValueSets) {
+      meta.writeInt(valueSet.length);
+    }
+  }
+
+  private Iterable<Number> docToSetId(SortedSet<LongsRef> uniqueValueSets, Iterable<Number> docToValueCount, Iterable<Number> values) {
+    final Map<LongsRef, Integer> setIds = new HashMap<>();
+    int i = 0;
+    for (LongsRef set : uniqueValueSets) {
+      setIds.put(set, i++);
+    }
+    assert i == uniqueValueSets.size();
+
+    return new Iterable<Number>() {
+
+      @Override
+      public Iterator<Number> iterator() {
+        final Iterator<Number> valueCountIterator = docToValueCount.iterator();
+        final Iterator<Number> valueIterator = values.iterator();
+        final LongsRef docValues = new LongsRef(256);
+        return new Iterator<Number>() {
+
+          @Override
+          public boolean hasNext() {
+            return valueCountIterator.hasNext();
+          }
+
+          @Override
+          public Number next() {
+            docValues.length = valueCountIterator.next().intValue();
+            for (int i = 0; i < docValues.length; ++i) {
+              docValues.longs[i] = valueIterator.next().longValue();
+            }
+            final Integer id = setIds.get(docValues);
+            assert id != null;
+            return id;
+          }
+
+        };
+
+      }
+    };
+  }
+
+  // writes addressing information as MONOTONIC_COMPRESSED integer
+  private void addOrdIndex(FieldInfo field, Iterable<Number> values) throws IOException {
+    meta.writeVInt(field.number);
+    meta.writeByte(Lucene54DocValuesFormat.NUMERIC);
+    meta.writeVInt(MONOTONIC_COMPRESSED);
+    meta.writeLong(-1L);
+    meta.writeLong(data.getFilePointer());
+    meta.writeVLong(maxDoc);
+    meta.writeVInt(DIRECT_MONOTONIC_BLOCK_SHIFT);
+
+    final DirectMonotonicWriter writer = DirectMonotonicWriter.getInstance(meta, data, maxDoc + 1, DIRECT_MONOTONIC_BLOCK_SHIFT);
+    long addr = 0;
+    writer.add(addr);
+    for (Number v : values) {
+      addr += v.longValue();
+      writer.add(addr);
+    }
+    writer.finish();
+    meta.writeLong(data.getFilePointer());
+  }
+
+  @Override
+  public void close() throws IOException {
+    boolean success = false;
+    try {
+      if (meta != null) {
+        meta.writeVInt(-1); // write EOF marker
+        CodecUtil.writeFooter(meta); // write checksum
+      }
+      if (data != null) {
+        CodecUtil.writeFooter(data); // write checksum
+      }
+      success = true;
+    } finally {
+      if (success) {
+        IOUtils.close(data, meta);
+      } else {
+        IOUtils.closeWhileHandlingException(data, meta);
+      }
+      meta = data = null;
+    }
+  }
+}