You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ab...@apache.org on 2017/07/04 17:59:51 UTC

[01/23] lucene-solr:jira/solr-10879: LUCENE-7871: fix false positive match in BlockJoinSelector when children have no value.

Repository: lucene-solr
Updated Branches:
  refs/heads/jira/solr-10879 97ebecedb -> c80c8cc59


LUCENE-7871: fix false positive match in BlockJoinSelector when children have no value.

- introducing  BlockJoinSelector.wrap methods accepting children as DISI.
- extracting ToParentDocValues


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

Branch: refs/heads/jira/solr-10879
Commit: 706d2018153d907642c77ae3673b99142b0d1734
Parents: d070ca6
Author: Mikhail Khludnev <mk...@apache.org>
Authored: Thu Jun 15 10:50:46 2017 +0300
Committer: Mikhail Khludnev <mk...@apache.org>
Committed: Tue Jun 20 16:31:09 2017 +0300

----------------------------------------------------------------------
 .../lucene/search/join/BlockJoinSelector.java   | 328 +++----------------
 .../lucene/search/join/ToParentDocValues.java   | 296 +++++++++++++++++
 .../search/join/TestBlockJoinSelector.java      |  76 ++++-
 solr/CHANGES.txt                                |   3 +
 4 files changed, 401 insertions(+), 302 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/706d2018/lucene/join/src/java/org/apache/lucene/search/join/BlockJoinSelector.java
----------------------------------------------------------------------
diff --git a/lucene/join/src/java/org/apache/lucene/search/join/BlockJoinSelector.java b/lucene/join/src/java/org/apache/lucene/search/join/BlockJoinSelector.java
index 359b3cb..a81230d 100644
--- a/lucene/join/src/java/org/apache/lucene/search/join/BlockJoinSelector.java
+++ b/lucene/join/src/java/org/apache/lucene/search/join/BlockJoinSelector.java
@@ -16,18 +16,17 @@
  */
 package org.apache.lucene.search.join;
 
-import java.io.IOException;
-
 import org.apache.lucene.index.NumericDocValues;
 import org.apache.lucene.index.SortedDocValues;
 import org.apache.lucene.index.SortedNumericDocValues;
 import org.apache.lucene.index.SortedSetDocValues;
+import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.search.SortField;
 import org.apache.lucene.search.SortedNumericSelector;
 import org.apache.lucene.search.SortedSetSelector;
 import org.apache.lucene.util.BitSet;
+import org.apache.lucene.util.BitSetIterator;
 import org.apache.lucene.util.Bits;
-import org.apache.lucene.util.BytesRef;
 
 /** Select a value from a block of documents.
  *  @lucene.internal */
@@ -78,7 +77,15 @@ public class BlockJoinSelector {
   /** Wraps the provided {@link SortedSetDocValues} in order to only select
    *  one value per parent among its {@code children} using the configured
    *  {@code selection} type. */
+  @Deprecated
   public static SortedDocValues wrap(SortedSetDocValues sortedSet, Type selection, BitSet parents, BitSet children) {
+    return wrap(sortedSet, selection, parents, toIter(children));
+  }
+
+  /** Wraps the provided {@link SortedSetDocValues} in order to only select
+   *  one value per parent among its {@code children} using the configured
+   *  {@code selection} type. */
+  public static SortedDocValues wrap(SortedSetDocValues sortedSet, Type selection, BitSet parents, DocIdSetIterator children) {
     SortedDocValues values;
     switch (selection) {
       case MIN:
@@ -92,162 +99,42 @@ public class BlockJoinSelector {
     }
     return wrap(values, selection, parents, children);
   }
-
+  
   /** Wraps the provided {@link SortedDocValues} in order to only select
    *  one value per parent among its {@code children} using the configured
    *  {@code selection} type. */
+  @Deprecated
   public static SortedDocValues wrap(final SortedDocValues values, Type selection, BitSet parents, BitSet children) {
+    return wrap(values, selection, parents, toIter(children));
+  }
+
+  /** Wraps the provided {@link SortedDocValues} in order to only select
+   *  one value per parent among its {@code children} using the configured
+   *  {@code selection} type. */
+  public static SortedDocValues wrap(final SortedDocValues values, Type selection, BitSet parents, DocIdSetIterator children) {
     if (values.docID() != -1) {
       throw new IllegalArgumentException("values iterator was already consumed: values.docID=" + values.docID());
     }
-    return new SortedDocValues() {
-
-      private int ord = -1;
-      private int docID = -1;
-
-      @Override
-      public int docID() {
-        return docID;
-      }
-
-      @Override
-      public int nextDoc() throws IOException {
-        assert docID != NO_MORE_DOCS;
-        
-        if (values.docID() == -1) {
-          if (values.nextDoc() == NO_MORE_DOCS) {
-            docID = NO_MORE_DOCS;
-            return docID;
-          }
-        }
-
-        if (values.docID() == NO_MORE_DOCS) {
-          docID = NO_MORE_DOCS;
-          return docID;
-        }
-        
-        int nextParentDocID = parents.nextSetBit(values.docID());
-        ord = values.ordValue();
-
-        while (true) {
-          int childDocID = values.nextDoc();
-          assert childDocID != nextParentDocID;
-          if (childDocID > nextParentDocID) {
-            break;
-          }
-          if (children.get(childDocID) == false) {
-            continue;
-          }
-          if (selection == Type.MIN) {
-            ord = Math.min(ord, values.ordValue());
-          } else if (selection == Type.MAX) {
-            ord = Math.max(ord, values.ordValue());
-          } else {
-            throw new AssertionError();
-          }
-        }
-
-        docID = nextParentDocID;
-        return docID;
-      }
-
-      @Override
-      public int advance(int target) throws IOException {
-        if (target >= parents.length()) {
-          docID = NO_MORE_DOCS;
-          return docID;
-        }
-        if (target == 0) {
-          assert docID() == -1;
-          return nextDoc();
-        }
-        int prevParentDocID = parents.prevSetBit(target-1);
-        if (values.docID() <= prevParentDocID) {
-          values.advance(prevParentDocID+1);
-        }
-        return nextDoc();
-      }
-
-      @Override
-      public boolean advanceExact(int targetParentDocID) throws IOException {
-        if (targetParentDocID < docID) {
-          throw new IllegalArgumentException("target must be after the current document: current=" + docID + " target=" + targetParentDocID);
-        }
-        int previousDocId = docID;
-        docID = targetParentDocID;
-        if (targetParentDocID == previousDocId) {
-          return ord != -1;
-        }
-        docID = targetParentDocID;
-        ord = -1;
-        if (parents.get(targetParentDocID) == false) {
-          return false;
-        }
-        int prevParentDocId = docID == 0 ? -1 : parents.prevSetBit(docID - 1);
-        int childDoc = values.docID();
-        if (childDoc <= prevParentDocId) {
-          childDoc = values.advance(prevParentDocId + 1);
-        }
-        if (childDoc >= docID) {
-          return false;
-        }
-        
-        boolean hasValue = false;
-        for (int doc = values.docID(); doc < docID; doc = values.nextDoc()) {
-          if (children.get(doc)) {
-            ord = values.ordValue();
-            hasValue = true;
-            values.nextDoc();
-            break;
-          }
-        }
-        if (hasValue == false) {
-          return false;
-        }
-
-        for (int doc = values.docID(); doc < docID; doc = values.nextDoc()) {
-          if (children.get(doc)) {
-            switch (selection) {
-              case MIN:
-                ord = Math.min(ord, values.ordValue());
-                break;
-              case MAX:
-                ord = Math.max(ord, values.ordValue());
-                break;
-              default:
-                throw new AssertionError();
-            }
-          }
-        }
-        return true;
-      }
-
-      @Override
-      public int ordValue() {
-        return ord;
-      }
-
-      @Override
-      public BytesRef lookupOrd(int ord) throws IOException {
-        return values.lookupOrd(ord);
-      }
-
-      @Override
-      public int getValueCount() {
-        return values.getValueCount();
-      }
-
-      @Override
-      public long cost() {
-        return values.cost();
-      }
-    };
+    return ToParentDocValues.wrap(values, selection, parents, children);
   }
 
   /** Wraps the provided {@link SortedNumericDocValues} in order to only select
    *  one value per parent among its {@code children} using the configured
    *  {@code selection} type. */
+  @Deprecated
   public static NumericDocValues wrap(SortedNumericDocValues sortedNumerics, Type selection, BitSet parents, BitSet children) {
+    return wrap(sortedNumerics, selection, parents, toIter(children));
+  }
+
+  /** creates an interator for the given bitset */
+  protected static BitSetIterator toIter(BitSet children) {
+    return new BitSetIterator(children, 0);
+  }
+  
+  /** Wraps the provided {@link SortedNumericDocValues} in order to only select
+   *  one value per parent among its {@code children} using the configured
+   *  {@code selection} type. */
+  public static NumericDocValues wrap(SortedNumericDocValues sortedNumerics, Type selection, BitSet parents, DocIdSetIterator children) {
     NumericDocValues values;
     switch (selection) {
       case MIN:
@@ -265,145 +152,18 @@ public class BlockJoinSelector {
   /** Wraps the provided {@link NumericDocValues}, iterating over only
    *  child documents, in order to only select one value per parent among
    *  its {@code children} using the configured {@code selection} type. */
+  @Deprecated
   public static NumericDocValues wrap(final NumericDocValues values, Type selection, BitSet parents, BitSet children) {
-    return new NumericDocValues() {
-
-      private int parentDocID = -1;
-      private long value;
-
-      @Override
-      public int nextDoc() throws IOException {
-
-        if (parentDocID == -1) {
-          values.nextDoc();
-        }
-
-        while (true) {
-
-          // TODO: make this crazy loop more efficient
-
-          int childDocID = values.docID();
-          if (childDocID == NO_MORE_DOCS) {
-            parentDocID = NO_MORE_DOCS;
-            return parentDocID;
-          }
-          if (children.get(childDocID) == false) {
-            values.nextDoc();
-            continue;
-          }
-
-          assert parents.get(childDocID) == false;
-        
-          parentDocID = parents.nextSetBit(childDocID);
-          value = values.longValue();
-
-          while (true) {
-            childDocID = values.nextDoc();
-            assert childDocID != parentDocID;
-            if (childDocID > parentDocID) {
-              break;
-            }
-
-            switch (selection) {
-            case MIN:
-              value = Math.min(value, values.longValue());
-              break;
-            case MAX:
-              value = Math.max(value, values.longValue());
-              break;
-            default:
-              throw new AssertionError();
-            }
-          }
-
-          break;
-        }
-
-        return parentDocID;
-      }
-
-      @Override
-      public int advance(int targetParentDocID) throws IOException {
-        if (targetParentDocID <= parentDocID) {
-          throw new IllegalArgumentException("target must be after the current document: current=" + parentDocID + " target=" + targetParentDocID);
-        }
-
-        if (targetParentDocID == 0) {
-          return nextDoc();
-        }
-        
-        int firstChild = parents.prevSetBit(targetParentDocID - 1) + 1;
-        if (values.advance(firstChild) == NO_MORE_DOCS) {
-          parentDocID = NO_MORE_DOCS;
-          return parentDocID;
-        } else {
-          return nextDoc();
-        }
-      }
-
-      @Override
-      public boolean advanceExact(int targetParentDocID) throws IOException {
-        if (targetParentDocID <= parentDocID) {
-          throw new IllegalArgumentException("target must be after the current document: current=" + parentDocID + " target=" + targetParentDocID);
-        }
-        parentDocID = targetParentDocID;
-        if (parents.get(targetParentDocID) == false) {
-          return false;
-        }
-        int prevParentDocId = parentDocID == 0 ? -1 : parents.prevSetBit(parentDocID - 1);
-        int childDoc = values.docID();
-        if (childDoc <= prevParentDocId) {
-          childDoc = values.advance(prevParentDocId + 1);
-        }
-        if (childDoc >= parentDocID) {
-          return false;
-        }
-        
-        boolean hasValue = false;
-        for (int doc = values.docID(); doc < parentDocID; doc = values.nextDoc()) {
-          if (children.get(doc)) {
-            value = values.longValue();
-            hasValue = true;
-            values.nextDoc();
-            break;
-          }
-        }
-        if (hasValue == false) {
-          return false;
-        }
-
-        for (int doc = values.docID(); doc < parentDocID; doc = values.nextDoc()) {
-          if (children.get(doc)) {
-            switch (selection) {
-              case MIN:
-                value = Math.min(value, values.longValue());
-                break;
-              case MAX:
-                value = Math.max(value, values.longValue());
-                break;
-              default:
-                throw new AssertionError();
-            }
-          }
-        }
-        return true;
-      }
-
-      @Override
-      public long longValue() {
-        return value;
-      }
-      
-      @Override
-      public int docID() {
-        return parentDocID;
-      }      
-
-      @Override
-      public long cost() {
-        return values.cost();
-      }      
-    };
+    return wrap(values,selection, parents, toIter(children));
   }
 
+  /** Wraps the provided {@link NumericDocValues}, iterating over only
+   *  child documents, in order to only select one value per parent among
+   *  its {@code children} using the configured {@code selection} type. */
+  public static NumericDocValues wrap(final NumericDocValues values, Type selection, BitSet parents, DocIdSetIterator children) {
+    if (values.docID() != -1) {
+      throw new IllegalArgumentException("values iterator was already consumed: values.docID=" + values.docID());
+    }
+    return ToParentDocValues.wrap(values,selection, parents, children);
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/706d2018/lucene/join/src/java/org/apache/lucene/search/join/ToParentDocValues.java
----------------------------------------------------------------------
diff --git a/lucene/join/src/java/org/apache/lucene/search/join/ToParentDocValues.java b/lucene/join/src/java/org/apache/lucene/search/join/ToParentDocValues.java
new file mode 100644
index 0000000..4e227ed
--- /dev/null
+++ b/lucene/join/src/java/org/apache/lucene/search/join/ToParentDocValues.java
@@ -0,0 +1,296 @@
+/*
+ * 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.search.join;
+
+import java.io.IOException;
+import java.util.Arrays;
+
+import org.apache.lucene.index.NumericDocValues;
+import org.apache.lucene.index.SortedDocValues;
+import org.apache.lucene.search.ConjunctionDISI;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.join.BlockJoinSelector.Type;
+import org.apache.lucene.util.BitSet;
+import org.apache.lucene.util.BytesRef;
+
+class ToParentDocValues extends DocIdSetIterator {
+
+  interface Accumulator{
+    void reset() throws IOException;
+    void increment() throws IOException;
+  }
+
+  private static final class SortedDVs extends SortedDocValues implements Accumulator{
+    private final SortedDocValues values;
+    private final BlockJoinSelector.Type selection;
+    private int ord = -1;
+    private final ToParentDocValues iter;
+  
+    private SortedDVs(SortedDocValues values, BlockJoinSelector.Type selection, BitSet parents, DocIdSetIterator children) {
+      this.values = values;
+      this.selection = selection;
+      this.iter = new ToParentDocValues(values,parents, children, this);
+    }
+  
+    @Override
+    public int docID() {
+      return iter.docID();
+    }
+  
+    @Override
+    public
+    void reset() throws IOException {
+      ord = values.ordValue();
+    }
+  
+    @Override
+    public
+    void increment() throws IOException {
+      if (selection == BlockJoinSelector.Type.MIN) {
+        ord = Math.min(ord, values.ordValue());
+      } else if (selection == BlockJoinSelector.Type.MAX) {
+        ord = Math.max(ord, values.ordValue());
+      } else {
+        throw new AssertionError();
+      }
+    }
+  
+    @Override
+    public int nextDoc() throws IOException {
+      return iter.nextDoc();
+    }
+  
+    @Override
+    public int advance(int target) throws IOException {
+      return iter.advance(target);
+    }
+  
+    @Override
+    public boolean advanceExact(int targetParentDocID) throws IOException {
+      return iter.advanceExact(targetParentDocID);
+    }
+  
+    @Override
+    public int ordValue() {
+      return ord;
+    }
+  
+    @Override
+    public BytesRef lookupOrd(int ord) throws IOException {
+      return values.lookupOrd(ord);
+    }
+  
+    @Override
+    public int getValueCount() {
+      return values.getValueCount();
+    }
+  
+    @Override
+    public long cost() {
+      return values.cost();
+    }
+  }
+
+  static private final class NumDV extends NumericDocValues implements Accumulator{
+    private final NumericDocValues values;
+    private long value;
+    private final BlockJoinSelector.Type selection;
+  
+    private final ToParentDocValues iter;
+  
+    private NumDV(NumericDocValues values, BlockJoinSelector.Type selection, BitSet parents, DocIdSetIterator children) {
+      this.values = values;
+      this.selection = selection;
+      iter = new ToParentDocValues(values, parents, children, this);
+    }
+  
+    @Override
+    public void reset() throws IOException {
+      value = values.longValue();
+    }
+    
+    @Override
+    public void increment() throws IOException {
+      switch (selection) {
+        case MIN:
+          value = Math.min(value, values.longValue());
+          break;
+        case MAX:
+          value = Math.max(value, values.longValue());
+          break;
+        default:
+          throw new AssertionError();
+        }
+    }
+    
+    @Override
+    public int nextDoc() throws IOException {
+      return iter.nextDoc();
+    }
+  
+    @Override
+    public int advance(int targetParentDocID) throws IOException {
+      return iter.advance(targetParentDocID);
+    }
+  
+    @Override
+    public boolean advanceExact(int targetParentDocID) throws IOException {
+      return iter.advanceExact(targetParentDocID);
+    }
+  
+    @Override
+    public long longValue() {
+      return value;
+    }
+  
+    @Override
+    public int docID() {
+      return iter.docID();
+    }
+  
+    @Override
+    public long cost() {
+      return values.cost();
+    }
+  }
+
+  private ToParentDocValues(DocIdSetIterator values, BitSet parents, DocIdSetIterator children, Accumulator collect) {
+    this.parents = parents;
+    childWithValues = ConjunctionDISI.intersectIterators(Arrays.asList(children, values));
+    this.collector = collect;
+  }
+
+  
+  final private BitSet parents;
+  private int docID = -1;
+  final private Accumulator collector;
+  boolean seen=false;
+  private DocIdSetIterator childWithValues;
+
+  @Override
+  public int docID() {
+    return docID;
+  }
+
+  @Override
+  public int nextDoc() throws IOException {
+    assert docID != NO_MORE_DOCS;
+    
+    assert childWithValues.docID()!=docID || docID==-1;
+    if (childWithValues.docID()<docID || docID==-1) {
+      childWithValues.nextDoc();
+    }
+    if (childWithValues.docID() == NO_MORE_DOCS) {
+      docID = NO_MORE_DOCS;
+      return docID;
+    }
+    
+
+    assert parents.get(childWithValues.docID()) == false;
+    
+    int nextParentDocID = parents.nextSetBit(childWithValues.docID());
+    collector.reset();
+    seen=true;
+    
+    while (true) {
+      int childDocID = childWithValues.nextDoc();
+      assert childDocID != nextParentDocID;
+      if (childDocID > nextParentDocID) {
+        break;
+      }
+      collector.increment();
+    }
+
+    docID = nextParentDocID;
+
+    return docID;
+  }
+
+  @Override
+  public int advance(int target) throws IOException {
+    if (target >= parents.length()) {
+      docID = NO_MORE_DOCS;
+      return docID;
+    }
+    if (target == 0) {
+      assert docID() == -1;
+      return nextDoc();
+    }
+    int prevParentDocID = parents.prevSetBit(target-1);
+    if (childWithValues.docID() <= prevParentDocID) {
+      childWithValues.advance(prevParentDocID+1);
+    }
+    return nextDoc();
+  }
+
+  //@Override
+  public boolean advanceExact(int targetParentDocID) throws IOException {
+    if (targetParentDocID < docID) {
+      throw new IllegalArgumentException("target must be after the current document: current=" + docID + " target=" + targetParentDocID);
+    }
+    int previousDocId = docID;
+    docID = targetParentDocID;
+    if (targetParentDocID == previousDocId) {
+      return seen;//ord != -1; rlly???
+    }
+    docID = targetParentDocID;
+    seen =false;
+    //ord = -1;
+    if (parents.get(targetParentDocID) == false) {
+      return false;
+    }
+    int prevParentDocId = docID == 0 ? -1 : parents.prevSetBit(docID - 1);
+    int childDoc = childWithValues.docID();
+    if (childDoc <= prevParentDocId) {
+      childDoc = childWithValues.advance(prevParentDocId + 1);
+    }
+    if (childDoc >= docID) {
+      return false;
+    }
+    
+    if (childWithValues.docID() < docID) {
+      collector.reset();
+      seen=true;
+      childWithValues.nextDoc();
+    }
+    
+    if (seen == false) {
+      return false;
+    }
+
+    for (int doc = childWithValues.docID(); doc < docID; doc = childWithValues.nextDoc()) {
+        collector.increment();
+    }
+    return true;
+  }
+
+  @Override
+  public long cost() {
+    return 0;
+  }
+
+  static NumericDocValues wrap(NumericDocValues values, Type selection, BitSet parents2,
+      DocIdSetIterator children) {
+    return new ToParentDocValues.NumDV(values,selection, parents2, children);
+  }
+
+  static SortedDocValues wrap(SortedDocValues values, Type selection, BitSet parents2,
+      DocIdSetIterator children) {
+    return new ToParentDocValues.SortedDVs(values, selection, parents2, children);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/706d2018/lucene/join/src/test/org/apache/lucene/search/join/TestBlockJoinSelector.java
----------------------------------------------------------------------
diff --git a/lucene/join/src/test/org/apache/lucene/search/join/TestBlockJoinSelector.java b/lucene/join/src/test/org/apache/lucene/search/join/TestBlockJoinSelector.java
index 04cb771..838826f 100644
--- a/lucene/join/src/test/org/apache/lucene/search/join/TestBlockJoinSelector.java
+++ b/lucene/join/src/test/org/apache/lucene/search/join/TestBlockJoinSelector.java
@@ -16,20 +16,22 @@
  */
 package org.apache.lucene.search.join;
 
+import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
+
 import java.io.IOException;
 import java.util.Arrays;
+import java.util.Random;
 
 import org.apache.lucene.index.DocValues;
 import org.apache.lucene.index.NumericDocValues;
 import org.apache.lucene.index.SortedDocValues;
+import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.util.BitSet;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.FixedBitSet;
 import org.apache.lucene.util.LuceneTestCase;
 
-import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
-
 public class TestBlockJoinSelector extends LuceneTestCase {
 
   public void testDocsWithValue() {
@@ -64,6 +66,48 @@ public class TestBlockJoinSelector extends LuceneTestCase {
     assertFalse(docsWithValue.get(19));
   }
 
+  static void assertNoMoreDoc(DocIdSetIterator sdv, int maxDoc) throws IOException{
+    Random r = random();
+    if(r.nextBoolean()){
+      assertEquals(NO_MORE_DOCS, sdv.nextDoc());
+    } else {
+      if (r.nextBoolean()) {
+        assertEquals(NO_MORE_DOCS,  sdv.advance(sdv.docID()+random().nextInt(maxDoc-sdv.docID())));
+      } else {
+        final int noMatchDoc = sdv.docID()+random().nextInt(maxDoc-sdv.docID()-1)+1;
+        assertFalse(advanceExact(sdv,noMatchDoc));
+        assertEquals(noMatchDoc, sdv.docID());
+        if (r.nextBoolean()){
+          assertEquals(NO_MORE_DOCS, sdv.nextDoc());
+        }
+      }
+    }
+  }
+  
+  static int nextDoc(DocIdSetIterator sdv, int docId) throws IOException{
+    Random r = random();
+    if(r.nextBoolean()){
+      return sdv.nextDoc();
+    } else {
+      if (r.nextBoolean()) {
+        return sdv.advance(sdv.docID()+random().nextInt(docId-sdv.docID()-1)+1);
+      } else {
+        if (r.nextBoolean()){
+          final int noMatchDoc = sdv.docID()+random().nextInt(docId-sdv.docID()-1)+1;
+          assertFalse(advanceExact(sdv,noMatchDoc));
+          assertEquals(noMatchDoc, sdv.docID());
+        }
+        assertTrue(advanceExact(sdv,docId));
+        return sdv.docID();
+      }
+    }
+  }
+  
+  private static boolean advanceExact(DocIdSetIterator sdv, int target) throws IOException {
+    return sdv instanceof SortedDocValues ? ((SortedDocValues) sdv).advanceExact(target)
+        : ((NumericDocValues) sdv).advanceExact(target);
+  }
+
   public void testSortedSelector() throws IOException {
     final BitSet parents = new FixedBitSet(20);
     parents.set(0);
@@ -89,22 +133,18 @@ public class TestBlockJoinSelector extends LuceneTestCase {
     ords[18] = 10;
 
     final SortedDocValues mins = BlockJoinSelector.wrap(DocValues.singleton(new CannedSortedDocValues(ords)), BlockJoinSelector.Type.MIN, parents, children);
-    assertEquals(5, mins.nextDoc());
+    assertEquals(5, nextDoc(mins,5));
     assertEquals(3, mins.ordValue());
-    assertEquals(15, mins.nextDoc());
+    assertEquals(15, nextDoc(mins,15));
     assertEquals(10, mins.ordValue());
-    assertEquals(19, mins.nextDoc());
-    assertEquals(10, mins.ordValue());
-    assertEquals(NO_MORE_DOCS, mins.nextDoc());
+    assertNoMoreDoc(mins, 20);
 
     final SortedDocValues maxs = BlockJoinSelector.wrap(DocValues.singleton(new CannedSortedDocValues(ords)), BlockJoinSelector.Type.MAX, parents, children);
-    assertEquals(5, maxs.nextDoc());
+    assertEquals(5, nextDoc(maxs,5));
     assertEquals(7, maxs.ordValue());
-    assertEquals(15, maxs.nextDoc());
-    assertEquals(10, maxs.ordValue());
-    assertEquals(19, maxs.nextDoc());
+    assertEquals(15, nextDoc(maxs,15));
     assertEquals(10, maxs.ordValue());
-    assertEquals(NO_MORE_DOCS, maxs.nextDoc());
+    assertNoMoreDoc( maxs,20);
   }
 
   private static class CannedSortedDocValues extends SortedDocValues {
@@ -207,18 +247,18 @@ public class TestBlockJoinSelector extends LuceneTestCase {
     longs[18] = 10;
 
     final NumericDocValues mins = BlockJoinSelector.wrap(DocValues.singleton(new CannedNumericDocValues(longs, docsWithValue)), BlockJoinSelector.Type.MIN, parents, children);
-    assertEquals(5, mins.nextDoc());
+    assertEquals(5, nextDoc(mins,5));
     assertEquals(3, mins.longValue());
-    assertEquals(15, mins.nextDoc());
+    assertEquals(15, nextDoc(mins,15));
     assertEquals(10, mins.longValue());
-    assertEquals(NO_MORE_DOCS, mins.nextDoc());
+    assertNoMoreDoc(mins, 20);
 
     final NumericDocValues maxs = BlockJoinSelector.wrap(DocValues.singleton(new CannedNumericDocValues(longs, docsWithValue)), BlockJoinSelector.Type.MAX, parents, children);
-    assertEquals(5, maxs.nextDoc());
+    assertEquals(5, nextDoc(maxs, 5));
     assertEquals(7, maxs.longValue());
-    assertEquals(15, maxs.nextDoc());
+    assertEquals(15, nextDoc(maxs, 15));
     assertEquals(10, maxs.longValue());
-    assertEquals(NO_MORE_DOCS, maxs.nextDoc());
+    assertNoMoreDoc(maxs, 20);
   }
 
   private static class CannedNumericDocValues extends NumericDocValues {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/706d2018/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index aadc6db..8c06107 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -501,6 +501,9 @@ when using one of Exact*StatsCache (Mikhail Khludnev)
 * SOLR-10910: Clean up a few details left over from pluggable transient core and untangling
   CoreDescriptor/CoreContainer references (Erick Erickson)
 
+* LUCENE-7871: fix false positive match in BlockJoinSelector when children have no value, introducing 
+  wrap methods accepting children as DISI. Extracting ToParentDocValues (Mikhail Khludnev)
+
 Optimizations
 ----------------------
 * SOLR-10634: JSON Facet API: When a field/terms facet will retrieve all buckets (i.e. limit:-1)


[19/23] lucene-solr:jira/solr-10879: SOLR-10456: Deprecate timeout related setters from SolrClients, and replace with Builder based implementation

Posted by ab...@apache.org.
SOLR-10456: Deprecate timeout related setters from SolrClients, and replace with Builder based implementation


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

Branch: refs/heads/jira/solr-10879
Commit: 6c163658bbca15b1e4ff81d16b25e07df78468e8
Parents: 8218a5b
Author: Anshum Gupta <an...@apple.com>
Authored: Mon Jul 3 21:44:18 2017 -0700
Committer: Anshum Gupta <an...@apple.com>
Committed: Mon Jul 3 21:44:18 2017 -0700

----------------------------------------------------------------------
 solr/CHANGES.txt                                |   3 +
 .../TestSolrEntityProcessorEndToEnd.java        |   4 +-
 .../client/solrj/embedded/JettySolrRunner.java  |   7 +
 .../cloud/LeaderInitiatedRecoveryThread.java    |   7 +-
 .../OverseerAutoReplicaFailoverThread.java      |   7 +-
 .../cloud/OverseerCollectionMessageHandler.java |   7 +-
 .../org/apache/solr/cloud/RecoveryStrategy.java |   5 +-
 .../org/apache/solr/cloud/SyncStrategy.java     |   8 +-
 .../org/apache/solr/cloud/ZkController.java     |   7 +-
 .../apache/solr/handler/CdcrRequestHandler.java |  12 +-
 .../solr/handler/CdcrUpdateLogSynchronizer.java |   7 +-
 .../org/apache/solr/handler/IndexFetcher.java   |  28 ++--
 .../solr/handler/admin/CollectionsHandler.java  |   7 +-
 .../component/HttpShardHandlerFactory.java      |   4 +-
 .../client/solrj/impl/ConnectionReuseTest.java  |   4 +-
 .../solr/cloud/BaseCdcrDistributedZkTest.java   |   3 +-
 .../solr/cloud/BasicDistributedZkTest.java      |  24 ++--
 .../cloud/ChaosMonkeyNothingIsSafeTest.java     |  19 ++-
 ...MonkeyNothingIsSafeWithPullReplicasTest.java |  20 ++-
 .../cloud/FullSolrCloudDistribCmdsTest.java     |   3 +-
 .../FullThrottleStoppableIndexingThread.java    |   4 +-
 .../apache/solr/cloud/HttpPartitionTest.java    |   4 +-
 .../apache/solr/cloud/MigrateRouteKeyTest.java  |   8 +-
 .../org/apache/solr/cloud/ShardSplitTest.java   |  10 +-
 .../org/apache/solr/cloud/SyncSliceTest.java    |   5 +-
 .../cloud/TestRandomRequestDistribution.java    |   8 +-
 .../cloud/TestRequestStatusCollectionAPI.java   |   3 +-
 .../solr/cloud/UnloadDistributedZkTest.java     |  31 ++---
 .../apache/solr/cloud/hdfs/StressHdfsTest.java  |   3 +-
 .../solr/core/OpenCloseCoreStressTest.java      |   8 +-
 .../solr/handler/TestReplicationHandler.java    |   4 +-
 .../handler/TestReplicationHandlerBackup.java   |   4 +-
 .../apache/solr/handler/TestRestoreCore.java    |   4 +-
 .../handler/admin/CoreAdminHandlerTest.java     |  36 ++---
 .../solr/client/solrj/impl/CloudSolrClient.java |  67 +++++++++-
 .../solrj/impl/ConcurrentUpdateSolrClient.java  |  36 +++++
 .../solr/client/solrj/impl/HttpSolrClient.java  |  38 +++++-
 .../client/solrj/impl/LBHttpSolrClient.java     |  60 ++++++++-
 .../client/solrj/SolrExampleBinaryTest.java     |   3 +-
 .../solr/client/solrj/SolrExampleXMLTest.java   |   3 +-
 .../solr/client/solrj/SolrExceptionTest.java    |   3 +-
 .../client/solrj/SolrSchemalessExampleTest.java |   3 +-
 .../solr/client/solrj/TestLBHttpSolrClient.java |   4 +-
 .../solrj/impl/BasicHttpSolrClientTest.java     |   3 +-
 .../client/solrj/impl/CloudSolrClientTest.java  |   4 +-
 .../solrj/impl/HttpSolrClientConPoolTest.java   |   3 +-
 .../java/org/apache/solr/SolrJettyTestBase.java |   3 +-
 .../java/org/apache/solr/SolrTestCaseJ4.java    | 132 +++++++++++++++++++
 .../cloud/AbstractFullDistribZkTestBase.java    |  31 +++--
 49 files changed, 502 insertions(+), 209 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c163658/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 5e1481f..61f72c4 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -397,6 +397,9 @@ Other Changes
   (added leading '/' in request handlers). Switch all tests referring to "standard" request handler to
   instead refer to "/select" with SearchHandler. Deprecated the old StandardRequestHandler. (David Smiley)
 
+* SOLR-10456: Deprecate timeout related setters from SolrClients, and replace with Builder based implementation
+  (Jason Gerlowski, Anshum Gupta)
+
 ==================  6.7.0 ==================
 
 Consult the LUCENE_CHANGES.txt file for additional, low level, changes in this release.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c163658/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestSolrEntityProcessorEndToEnd.java
----------------------------------------------------------------------
diff --git a/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestSolrEntityProcessorEndToEnd.java b/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestSolrEntityProcessorEndToEnd.java
index 9e104ee..a66b21e 100644
--- a/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestSolrEntityProcessorEndToEnd.java
+++ b/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestSolrEntityProcessorEndToEnd.java
@@ -305,9 +305,7 @@ public class TestSolrEntityProcessorEndToEnd extends AbstractDataImportHandlerTe
       sidl.add(sd);
     }
 
-    try (HttpSolrClient solrServer = getHttpSolrClient(getSourceUrl())) {
-      solrServer.setConnectionTimeout(15000);
-      solrServer.setSoTimeout(30000);
+    try (HttpSolrClient solrServer = getHttpSolrClient(getSourceUrl(), 15000, 30000)) {
       solrServer.add(sidl);
       solrServer.commit(true, true);
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c163658/solr/core/src/java/org/apache/solr/client/solrj/embedded/JettySolrRunner.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/client/solrj/embedded/JettySolrRunner.java b/solr/core/src/java/org/apache/solr/client/solrj/embedded/JettySolrRunner.java
index 155f52e..608b234 100644
--- a/solr/core/src/java/org/apache/solr/client/solrj/embedded/JettySolrRunner.java
+++ b/solr/core/src/java/org/apache/solr/client/solrj/embedded/JettySolrRunner.java
@@ -508,6 +508,13 @@ public class JettySolrRunner {
   public SolrClient newClient() {
     return new HttpSolrClient.Builder(getBaseUrl().toString()).build();
   }
+  
+  public SolrClient newClient(int connectionTimeoutMillis, int socketTimeoutMillis) {
+    return new HttpSolrClient.Builder(getBaseUrl().toString())
+        .withConnectionTimeout(connectionTimeoutMillis)
+        .withSocketTimeout(socketTimeoutMillis)
+        .build();
+  }
 
   public DebugFilter getDebugFilter() {
     return (DebugFilter)debugFilter.getFilter();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c163658/solr/core/src/java/org/apache/solr/cloud/LeaderInitiatedRecoveryThread.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/LeaderInitiatedRecoveryThread.java b/solr/core/src/java/org/apache/solr/cloud/LeaderInitiatedRecoveryThread.java
index 4389d9e..8c892ce 100644
--- a/solr/core/src/java/org/apache/solr/cloud/LeaderInitiatedRecoveryThread.java
+++ b/solr/core/src/java/org/apache/solr/cloud/LeaderInitiatedRecoveryThread.java
@@ -201,9 +201,10 @@ public class LeaderInitiatedRecoveryThread extends Thread {
         log.info("Asking core={} coreNodeName={} on " + recoveryUrl + " to recover", coreNeedingRecovery, replicaCoreNodeName);
       }
 
-      try (HttpSolrClient client = new HttpSolrClient.Builder(recoveryUrl).build()) {
-        client.setSoTimeout(60000);
-        client.setConnectionTimeout(15000);
+      try (HttpSolrClient client = new HttpSolrClient.Builder(recoveryUrl)
+          .withConnectionTimeout(15000)
+          .withSocketTimeout(60000)
+          .build()) {
         try {
           client.request(recoverRequestCmd);
           

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c163658/solr/core/src/java/org/apache/solr/cloud/OverseerAutoReplicaFailoverThread.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/OverseerAutoReplicaFailoverThread.java b/solr/core/src/java/org/apache/solr/cloud/OverseerAutoReplicaFailoverThread.java
index ea09eef..2eccef8 100644
--- a/solr/core/src/java/org/apache/solr/cloud/OverseerAutoReplicaFailoverThread.java
+++ b/solr/core/src/java/org/apache/solr/cloud/OverseerAutoReplicaFailoverThread.java
@@ -443,10 +443,11 @@ public class OverseerAutoReplicaFailoverThread implements Runnable, Closeable {
       final String createUrl, final String dataDir, final String ulogDir,
       final String coreNodeName, final String coreName, final String shardId) {
 
-    try (HttpSolrClient client = new HttpSolrClient.Builder(createUrl).build()) {
+    try (HttpSolrClient client = new HttpSolrClient.Builder(createUrl)
+        .withConnectionTimeout(30000)
+        .withSocketTimeout(60000)
+        .build()) {
       log.debug("create url={}", createUrl);
-      client.setConnectionTimeout(30000);
-      client.setSoTimeout(60000);
       Create createCmd = new Create();
       createCmd.setCollection(collection);
       createCmd.setCoreNodeName(coreNodeName);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c163658/solr/core/src/java/org/apache/solr/cloud/OverseerCollectionMessageHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/OverseerCollectionMessageHandler.java b/solr/core/src/java/org/apache/solr/cloud/OverseerCollectionMessageHandler.java
index 600e6da..a8d74e8 100644
--- a/solr/core/src/java/org/apache/solr/cloud/OverseerCollectionMessageHandler.java
+++ b/solr/core/src/java/org/apache/solr/cloud/OverseerCollectionMessageHandler.java
@@ -503,9 +503,10 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler
 
   static UpdateResponse softCommit(String url) throws SolrServerException, IOException {
 
-    try (HttpSolrClient client = new HttpSolrClient.Builder(url).build()) {
-      client.setConnectionTimeout(30000);
-      client.setSoTimeout(120000);
+    try (HttpSolrClient client = new HttpSolrClient.Builder(url)
+        .withConnectionTimeout(30000)
+        .withSocketTimeout(120000)
+        .build()) {
       UpdateRequest ureq = new UpdateRequest();
       ureq.setParams(new ModifiableSolrParams());
       ureq.setAction(AbstractUpdateRequest.ACTION.COMMIT, false, true, true);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c163658/solr/core/src/java/org/apache/solr/cloud/RecoveryStrategy.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/RecoveryStrategy.java b/solr/core/src/java/org/apache/solr/cloud/RecoveryStrategy.java
index 063f794..563cccf 100644
--- a/solr/core/src/java/org/apache/solr/cloud/RecoveryStrategy.java
+++ b/solr/core/src/java/org/apache/solr/cloud/RecoveryStrategy.java
@@ -255,8 +255,9 @@ public class RecoveryStrategy implements Runnable, Closeable {
 
   final private void commitOnLeader(String leaderUrl) throws SolrServerException,
       IOException {
-    try (HttpSolrClient client = new HttpSolrClient.Builder(leaderUrl).build()) {
-      client.setConnectionTimeout(30000);
+    try (HttpSolrClient client = new HttpSolrClient.Builder(leaderUrl)
+        .withConnectionTimeout(30000)
+        .build()) {
       UpdateRequest ureq = new UpdateRequest();
       ureq.setParams(new ModifiableSolrParams());
       ureq.getParams().set(DistributedUpdateProcessor.COMMIT_END_POINT, true);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c163658/solr/core/src/java/org/apache/solr/cloud/SyncStrategy.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/SyncStrategy.java b/solr/core/src/java/org/apache/solr/cloud/SyncStrategy.java
index 386fef9..a0f00bb2 100644
--- a/solr/core/src/java/org/apache/solr/cloud/SyncStrategy.java
+++ b/solr/core/src/java/org/apache/solr/cloud/SyncStrategy.java
@@ -292,9 +292,11 @@ public class SyncStrategy {
         recoverRequestCmd.setAction(CoreAdminAction.REQUESTRECOVERY);
         recoverRequestCmd.setCoreName(coreName);
         
-        try (HttpSolrClient client = new HttpSolrClient.Builder(baseUrl).withHttpClient(SyncStrategy.this.client).build()) {
-          client.setConnectionTimeout(30000);
-          client.setSoTimeout(120000);
+        try (HttpSolrClient client = new HttpSolrClient.Builder(baseUrl)
+            .withHttpClient(SyncStrategy.this.client)
+            .withConnectionTimeout(30000)
+            .withSocketTimeout(120000)
+            .build()) {
           client.request(recoverRequestCmd);
         } catch (Throwable t) {
           SolrException.log(log, ZkCoreNodeProps.getCoreUrl(leaderProps) + ": Could not tell a replica to recover", t);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c163658/solr/core/src/java/org/apache/solr/cloud/ZkController.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/ZkController.java b/solr/core/src/java/org/apache/solr/cloud/ZkController.java
index 01e1693..c8d763f 100644
--- a/solr/core/src/java/org/apache/solr/cloud/ZkController.java
+++ b/solr/core/src/java/org/apache/solr/cloud/ZkController.java
@@ -1605,9 +1605,10 @@ public class ZkController {
         log.info("Replica " + myCoreNodeName +
             " NOT in leader-initiated recovery, need to wait for leader to see down state.");
 
-        try (HttpSolrClient client = new Builder(leaderBaseUrl).build()) {
-          client.setConnectionTimeout(15000);
-          client.setSoTimeout(120000);
+        try (HttpSolrClient client = new Builder(leaderBaseUrl)
+            .withConnectionTimeout(15000)
+            .withSocketTimeout(120000)
+            .build()) {
           WaitForState prepCmd = new WaitForState();
           prepCmd.setCoreName(leaderCoreName);
           prepCmd.setNodeName(getNodeName());

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c163658/solr/core/src/java/org/apache/solr/handler/CdcrRequestHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/CdcrRequestHandler.java b/solr/core/src/java/org/apache/solr/handler/CdcrRequestHandler.java
index 12272cf..de86164 100644
--- a/solr/core/src/java/org/apache/solr/handler/CdcrRequestHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/CdcrRequestHandler.java
@@ -789,8 +789,9 @@ public class CdcrRequestHandler extends RequestHandlerBase implements SolrCoreAw
 
     private void commitOnLeader(String leaderUrl) throws SolrServerException,
         IOException {
-      try (HttpSolrClient client = new HttpSolrClient.Builder(leaderUrl).build()) {
-        client.setConnectionTimeout(30000);
+      try (HttpSolrClient client = new HttpSolrClient.Builder(leaderUrl)
+          .withConnectionTimeout(30000)
+          .build()) {
         UpdateRequest ureq = new UpdateRequest();
         ureq.setParams(new ModifiableSolrParams());
         ureq.getParams().set(DistributedUpdateProcessor.COMMIT_END_POINT, true);
@@ -827,9 +828,10 @@ public class CdcrRequestHandler extends RequestHandlerBase implements SolrCoreAw
 
     @Override
     public Long call() throws Exception {
-      try (HttpSolrClient server = new HttpSolrClient.Builder(baseUrl).build()) {
-        server.setConnectionTimeout(15000);
-        server.setSoTimeout(60000);
+      try (HttpSolrClient server = new HttpSolrClient.Builder(baseUrl)
+          .withConnectionTimeout(15000)
+          .withSocketTimeout(60000)
+          .build()) {
 
         ModifiableSolrParams params = new ModifiableSolrParams();
         params.set(CommonParams.ACTION, CdcrParams.CdcrAction.SHARDCHECKPOINT.toString());

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c163658/solr/core/src/java/org/apache/solr/handler/CdcrUpdateLogSynchronizer.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/CdcrUpdateLogSynchronizer.java b/solr/core/src/java/org/apache/solr/handler/CdcrUpdateLogSynchronizer.java
index 7ce060c..80f27ce 100644
--- a/solr/core/src/java/org/apache/solr/handler/CdcrUpdateLogSynchronizer.java
+++ b/solr/core/src/java/org/apache/solr/handler/CdcrUpdateLogSynchronizer.java
@@ -131,9 +131,10 @@ class CdcrUpdateLogSynchronizer implements CdcrStateManager.CdcrStateObserver {
           return;
         }
 
-        HttpSolrClient server = new HttpSolrClient.Builder(leaderUrl).build();
-        server.setConnectionTimeout(15000);
-        server.setSoTimeout(60000);
+        HttpSolrClient server = new HttpSolrClient.Builder(leaderUrl)
+            .withConnectionTimeout(15000)
+            .withSocketTimeout(60000)
+            .build();
 
         ModifiableSolrParams params = new ModifiableSolrParams();
         params.set(CommonParams.ACTION, CdcrParams.CdcrAction.LASTPROCESSEDVERSION.toString());

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c163658/solr/core/src/java/org/apache/solr/handler/IndexFetcher.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/IndexFetcher.java b/solr/core/src/java/org/apache/solr/handler/IndexFetcher.java
index 9c9918e..1bf452f 100644
--- a/solr/core/src/java/org/apache/solr/handler/IndexFetcher.java
+++ b/solr/core/src/java/org/apache/solr/handler/IndexFetcher.java
@@ -276,9 +276,11 @@ public class IndexFetcher {
     QueryRequest req = new QueryRequest(params);
 
     // TODO modify to use shardhandler
-    try (HttpSolrClient client = new Builder(masterUrl).withHttpClient(myHttpClient).build()) {
-      client.setSoTimeout(soTimeout);
-      client.setConnectionTimeout(connTimeout);
+    try (HttpSolrClient client = new Builder(masterUrl)
+        .withHttpClient(myHttpClient)
+        .withConnectionTimeout(connTimeout)
+        .withSocketTimeout(soTimeout)
+        .build()) {
 
       return client.request(req);
     } catch (SolrServerException e) {
@@ -298,9 +300,11 @@ public class IndexFetcher {
     QueryRequest req = new QueryRequest(params);
 
     // TODO modify to use shardhandler
-    try (HttpSolrClient client = new HttpSolrClient.Builder(masterUrl).withHttpClient(myHttpClient).build()) {
-      client.setSoTimeout(soTimeout);
-      client.setConnectionTimeout(connTimeout);
+    try (HttpSolrClient client = new HttpSolrClient.Builder(masterUrl)
+        .withHttpClient(myHttpClient)
+        .withConnectionTimeout(connTimeout)
+        .withSocketTimeout(soTimeout)
+        .build()) {
       NamedList response = client.request(req);
 
       List<Map<String, Object>> files = (List<Map<String,Object>>) response.get(CMD_GET_FILE_LIST);
@@ -1691,9 +1695,9 @@ public class IndexFetcher {
       try (HttpSolrClient client = new Builder(masterUrl)
           .withHttpClient(myHttpClient)
           .withResponseParser(null)
+          .withConnectionTimeout(connTimeout)
+          .withSocketTimeout(soTimeout)
           .build()) {
-        client.setSoTimeout(soTimeout);
-        client.setConnectionTimeout(connTimeout);
         QueryRequest req = new QueryRequest(params);
         response = client.request(req);
         is = (InputStream) response.get("stream");
@@ -1800,9 +1804,11 @@ public class IndexFetcher {
     params.set(CommonParams.QT, ReplicationHandler.PATH);
 
     // TODO use shardhandler
-    try (HttpSolrClient client = new HttpSolrClient.Builder(masterUrl).withHttpClient(myHttpClient).build()) {
-      client.setSoTimeout(soTimeout);
-      client.setConnectionTimeout(connTimeout);
+    try (HttpSolrClient client = new HttpSolrClient.Builder(masterUrl)
+        .withHttpClient(myHttpClient)
+        .withConnectionTimeout(connTimeout)
+        .withSocketTimeout(soTimeout)
+        .build()) {
       QueryRequest request = new QueryRequest(params);
       return client.request(request);
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c163658/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java b/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java
index 122aa4a..256400e 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java
@@ -442,9 +442,10 @@ public class CollectionsHandler extends RequestHandlerBase implements Permission
       ZkNodeProps leaderProps = docCollection.getLeader(shard);
       ZkCoreNodeProps nodeProps = new ZkCoreNodeProps(leaderProps);
 
-      try (HttpSolrClient client = new Builder(nodeProps.getBaseUrl()).build()) {
-        client.setConnectionTimeout(15000);
-        client.setSoTimeout(60000);
+      try (HttpSolrClient client = new Builder(nodeProps.getBaseUrl())
+          .withConnectionTimeout(15000)
+          .withSocketTimeout(60000)
+          .build()) {
         RequestSyncShard reqSyncShard = new RequestSyncShard();
         reqSyncShard.setCollection(collection);
         reqSyncShard.setShard(shard);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c163658/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandlerFactory.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandlerFactory.java b/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandlerFactory.java
index 73d9707..6bfd36a 100644
--- a/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandlerFactory.java
+++ b/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandlerFactory.java
@@ -224,9 +224,9 @@ public class HttpShardHandlerFactory extends ShardHandlerFactory implements org.
   protected LBHttpSolrClient createLoadbalancer(HttpClient httpClient){
     LBHttpSolrClient client = new Builder()
         .withHttpClient(httpClient)
+        .withConnectionTimeout(connectionTimeout)
+        .withSocketTimeout(soTimeout)
         .build();
-    client.setConnectionTimeout(connectionTimeout);
-    client.setSoTimeout(soTimeout);
     return client;
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c163658/solr/core/src/test/org/apache/solr/client/solrj/impl/ConnectionReuseTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/client/solrj/impl/ConnectionReuseTest.java b/solr/core/src/test/org/apache/solr/client/solrj/impl/ConnectionReuseTest.java
index 56ec3f2..55915f9 100644
--- a/solr/core/src/test/org/apache/solr/client/solrj/impl/ConnectionReuseTest.java
+++ b/solr/core/src/test/org/apache/solr/client/solrj/impl/ConnectionReuseTest.java
@@ -76,11 +76,9 @@ public class ConnectionReuseTest extends SolrCloudTestCase {
       case 1:
         return getHttpSolrClient(url.toString() + "/" + COLLECTION, httpClient);
       case 2:
-        CloudSolrClient client = getCloudSolrClient(cluster.getZkServer().getZkAddress(), random().nextBoolean(), httpClient);
+        CloudSolrClient client = getCloudSolrClient(cluster.getZkServer().getZkAddress(), random().nextBoolean(), httpClient, 30000, 60000);
         client.setParallelUpdates(random().nextBoolean());
         client.setDefaultCollection(COLLECTION);
-        client.getLbClient().setConnectionTimeout(30000);
-        client.getLbClient().setSoTimeout(60000);
         return client;
     }
     throw new RuntimeException("impossible");

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c163658/solr/core/src/test/org/apache/solr/cloud/BaseCdcrDistributedZkTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/BaseCdcrDistributedZkTest.java b/solr/core/src/test/org/apache/solr/cloud/BaseCdcrDistributedZkTest.java
index 8ce346b..fc563aa 100644
--- a/solr/core/src/test/org/apache/solr/cloud/BaseCdcrDistributedZkTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/BaseCdcrDistributedZkTest.java
@@ -782,8 +782,7 @@ public class BaseCdcrDistributedZkTest extends AbstractDistribZkTestBase {
   protected static SolrClient createNewSolrServer(String baseUrl) {
     try {
       // setup the server...
-      HttpSolrClient s = getHttpSolrClient(baseUrl);
-      s.setConnectionTimeout(DEFAULT_CONNECTION_TIMEOUT);
+      HttpSolrClient s = getHttpSolrClient(baseUrl, DEFAULT_CONNECTION_TIMEOUT);
       return s;
     } catch (Exception ex) {
       throw new RuntimeException(ex);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c163658/solr/core/src/test/org/apache/solr/cloud/BasicDistributedZkTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/BasicDistributedZkTest.java b/solr/core/src/test/org/apache/solr/cloud/BasicDistributedZkTest.java
index 518142a..c095c25 100644
--- a/solr/core/src/test/org/apache/solr/cloud/BasicDistributedZkTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/BasicDistributedZkTest.java
@@ -536,9 +536,7 @@ public class BasicDistributedZkTest extends AbstractFullDistribZkTestBase {
 
   private void testStopAndStartCoresInOneInstance() throws Exception {
     JettySolrRunner jetty = jettys.get(0);
-    try (final HttpSolrClient httpSolrClient = (HttpSolrClient) jetty.newClient()) {
-      httpSolrClient.setConnectionTimeout(15000);
-      httpSolrClient.setSoTimeout(60000);
+    try (final HttpSolrClient httpSolrClient = (HttpSolrClient) jetty.newClient(15000, 60000)) {
       ThreadPoolExecutor executor = null;
       try {
         executor = new ExecutorUtil.MDCAwareThreadPoolExecutor(0, Integer.MAX_VALUE,
@@ -772,9 +770,7 @@ public class BasicDistributedZkTest extends AbstractFullDistribZkTestBase {
     String collection = elements[elements.length - 1];
     String urlString = url.toString();
     urlString = urlString.substring(0, urlString.length() - collection.length() - 1);
-    try (HttpSolrClient client = getHttpSolrClient(urlString)) {
-      client.setConnectionTimeout(15000);
-      client.setSoTimeout(60000);
+    try (HttpSolrClient client = getHttpSolrClient(urlString, 15000, 60000)) {
       ModifiableSolrParams params = new ModifiableSolrParams();
       //params.set("qt", "/admin/metrics?prefix=UPDATE.updateHandler&registry=solr.core." + collection);
       params.set("qt", "/admin/metrics");
@@ -860,9 +856,7 @@ public class BasicDistributedZkTest extends AbstractFullDistribZkTestBase {
     ZkCoreNodeProps props = new ZkCoreNodeProps(getCommonCloudSolrClient().getZkStateReader().getClusterState().getLeader(oneInstanceCollection2, "shard1"));
     
     // now test that unloading a core gets us a new leader
-    try (HttpSolrClient unloadClient = getHttpSolrClient(jettys.get(0).getBaseUrl().toString())) {
-      unloadClient.setConnectionTimeout(15000);
-      unloadClient.setSoTimeout(60000);
+    try (HttpSolrClient unloadClient = getHttpSolrClient(jettys.get(0).getBaseUrl().toString(), 15000, 60000)) {
       Unload unloadCmd = new Unload(true);
       unloadCmd.setCoreName(props.getCoreName());
 
@@ -1133,6 +1127,18 @@ public class BasicDistributedZkTest extends AbstractFullDistribZkTestBase {
       throw new RuntimeException(ex);
     }
   }
+  
+  protected SolrClient createNewSolrClient(String collection, String baseUrl, int connectionTimeoutMillis, int socketTimeoutMillis) {
+    try {
+      // setup the server...
+      HttpSolrClient client = getHttpSolrClient(baseUrl + "/" + collection, connectionTimeoutMillis, socketTimeoutMillis);
+
+      return client;
+    }
+    catch (Exception ex) {
+      throw new RuntimeException(ex);
+    }
+  }
 
   @Override
   protected QueryResponse queryServer(ModifiableSolrParams params) throws SolrServerException, IOException {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c163658/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeyNothingIsSafeTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeyNothingIsSafeTest.java b/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeyNothingIsSafeTest.java
index 2e31520..522f4dd 100644
--- a/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeyNothingIsSafeTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeyNothingIsSafeTest.java
@@ -59,7 +59,7 @@ public class ChaosMonkeyNothingIsSafeTest extends AbstractFullDistribZkTestBase
   protected static final String[] fieldNames = new String[]{"f_i", "f_f", "f_d", "f_l", "f_dt"};
   protected static final RandVal[] randVals = new RandVal[]{rint, rfloat, rdouble, rlong, rdate};
 
-  private int clientSoTimeout;
+  private int clientSoTimeout = 60000;
   
   public String[] getFieldNames() {
     return fieldNames;
@@ -92,9 +92,7 @@ public class ChaosMonkeyNothingIsSafeTest extends AbstractFullDistribZkTestBase
     }
     fixShardCount(numShards);
 
-    // None of the operations used here are particularly costly, so this should work.
-    // Using this low timeout will also help us catch index stalling.
-    clientSoTimeout = 5000;
+
   }
 
   @Override
@@ -102,9 +100,20 @@ public class ChaosMonkeyNothingIsSafeTest extends AbstractFullDistribZkTestBase
     return onlyLeaderIndexes;
   }
 
+  @Override
+  protected CloudSolrClient createCloudClient(String defaultCollection) {
+    CloudSolrClient client = getCloudSolrClient(zkServer.getZkAddress(), random().nextBoolean(), 30000, clientSoTimeout);
+    client.setParallelUpdates(random().nextBoolean());
+    if (defaultCollection != null) client.setDefaultCollection(defaultCollection);
+    return client;
+  }
+
   @Test
   public void test() throws Exception {
-    cloudClient.setSoTimeout(clientSoTimeout);
+    // None of the operations used here are particularly costly, so this should work.
+    // Using this low timeout will also help us catch index stalling.
+    clientSoTimeout = 5000;
+    cloudClient = createCloudClient(DEFAULT_COLLECTION);
     boolean testSuccessful = false;
     try {
       handle.clear();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c163658/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeyNothingIsSafeWithPullReplicasTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeyNothingIsSafeWithPullReplicasTest.java b/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeyNothingIsSafeWithPullReplicasTest.java
index 672cd6b..2e5aa7d 100644
--- a/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeyNothingIsSafeWithPullReplicasTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeyNothingIsSafeWithPullReplicasTest.java
@@ -84,7 +84,7 @@ public class ChaosMonkeyNothingIsSafeWithPullReplicasTest extends AbstractFullDi
   protected static final String[] fieldNames = new String[]{"f_i", "f_f", "f_d", "f_l", "f_dt"};
   protected static final RandVal[] randVals = new RandVal[]{rint, rfloat, rdouble, rlong, rdate};
 
-  private int clientSoTimeout;
+  private int clientSoTimeout = 60000;
   
   public String[] getFieldNames() {
     return fieldNames;
@@ -115,19 +115,29 @@ public class ChaosMonkeyNothingIsSafeWithPullReplicasTest extends AbstractFullDi
     fixShardCount(numNodes);
     log.info("Starting ChaosMonkey test with {} shards and {} nodes", sliceCount, numNodes);
 
-    // None of the operations used here are particularly costly, so this should work.
-    // Using this low timeout will also help us catch index stalling.
-    clientSoTimeout = 5000;
+
   }
 
   @Override
   protected boolean useTlogReplicas() {
     return useTlogReplicas;
   }
+  
+  @Override
+  protected CloudSolrClient createCloudClient(String defaultCollection) {
+    CloudSolrClient client = getCloudSolrClient(zkServer.getZkAddress(), random().nextBoolean(), 30000, clientSoTimeout);
+    client.setParallelUpdates(random().nextBoolean());
+    if (defaultCollection != null) client.setDefaultCollection(defaultCollection);
+    return client;
+  }
+
 
   @Test
   public void test() throws Exception {
-    cloudClient.setSoTimeout(clientSoTimeout);
+    // None of the operations used here are particularly costly, so this should work.
+    // Using this low timeout will also help us catch index stalling.
+    clientSoTimeout = 5000;
+    cloudClient = createCloudClient(DEFAULT_COLLECTION);
     DocCollection docCollection = cloudClient.getZkStateReader().getClusterState().getCollection(DEFAULT_COLLECTION);
     assertEquals(this.sliceCount, docCollection.getSlices().size());
     Slice s = docCollection.getSlice("shard1");

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c163658/solr/core/src/test/org/apache/solr/cloud/FullSolrCloudDistribCmdsTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/FullSolrCloudDistribCmdsTest.java b/solr/core/src/test/org/apache/solr/cloud/FullSolrCloudDistribCmdsTest.java
index 9355c43..b4c5a46 100644
--- a/solr/core/src/test/org/apache/solr/cloud/FullSolrCloudDistribCmdsTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/FullSolrCloudDistribCmdsTest.java
@@ -691,8 +691,7 @@ public class FullSolrCloudDistribCmdsTest extends AbstractFullDistribZkTestBase
     long beforeCount = results.getResults().getNumFound();
     int cnt = TEST_NIGHTLY ? 2933 : 313;
     try (ConcurrentUpdateSolrClient concurrentClient = getConcurrentUpdateSolrClient(
-        ((HttpSolrClient) clients.get(0)).getBaseURL(), 10, 2)) {
-      concurrentClient.setConnectionTimeout(120000);
+        ((HttpSolrClient) clients.get(0)).getBaseURL(), 10, 2, 120000)) {
       for (int i = 0; i < cnt; i++) {
         index_specific(concurrentClient, id, docId++, "text_t", "some text so that it not's negligent work to parse this doc, even though it's still a pretty short doc");
       }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c163658/solr/core/src/test/org/apache/solr/cloud/FullThrottleStoppableIndexingThread.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/FullThrottleStoppableIndexingThread.java b/solr/core/src/test/org/apache/solr/cloud/FullThrottleStoppableIndexingThread.java
index c33ad4f..a74854d 100644
--- a/solr/core/src/test/org/apache/solr/cloud/FullThrottleStoppableIndexingThread.java
+++ b/solr/core/src/test/org/apache/solr/cloud/FullThrottleStoppableIndexingThread.java
@@ -58,9 +58,9 @@ class FullThrottleStoppableIndexingThread extends StoppableIndexingThread {
         .withHttpClient(httpClient)
         .withQueueSize(8)
         .withThreadCount(2)
+        .withConnectionTimeout(10000)
+        .withSocketTimeout(clientSoTimeout)
         .build();
-    cusc.setConnectionTimeout(10000);
-    cusc.setSoTimeout(clientSoTimeout);
   }
   
   @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c163658/solr/core/src/test/org/apache/solr/cloud/HttpPartitionTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/HttpPartitionTest.java b/solr/core/src/test/org/apache/solr/cloud/HttpPartitionTest.java
index ea989c0..aeaa7e9 100644
--- a/solr/core/src/test/org/apache/solr/cloud/HttpPartitionTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/HttpPartitionTest.java
@@ -96,11 +96,11 @@ public class HttpPartitionTest extends AbstractFullDistribZkTestBase {
     CloudSolrClient client = new CloudSolrClient.Builder()
         .withZkHost(zkServer.getZkAddress())
         .sendDirectUpdatesToAnyShardReplica()
+        .withConnectionTimeout(30000)
+        .withSocketTimeout(60000)
         .build();
     client.setParallelUpdates(random().nextBoolean());
     if (defaultCollection != null) client.setDefaultCollection(defaultCollection);
-    client.getLbClient().setConnectionTimeout(30000);
-    client.getLbClient().setSoTimeout(60000);
     return client;
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c163658/solr/core/src/test/org/apache/solr/cloud/MigrateRouteKeyTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/MigrateRouteKeyTest.java b/solr/core/src/test/org/apache/solr/cloud/MigrateRouteKeyTest.java
index 8e3f63d..b57d06f 100644
--- a/solr/core/src/test/org/apache/solr/cloud/MigrateRouteKeyTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/MigrateRouteKeyTest.java
@@ -80,13 +80,7 @@ public class MigrateRouteKeyTest extends SolrCloudTestCase {
   }
 
   protected void invokeCollectionMigration(CollectionAdminRequest.AsyncCollectionAdminRequest request) throws IOException, SolrServerException, InterruptedException {
-    if (random().nextBoolean()) {
-      cluster.getSolrClient().setSoTimeout(60000);  // can take a while
-      request.process(cluster.getSolrClient());
-    }
-    else {
-      request.processAndWait(cluster.getSolrClient(), 60000);
-    }
+    request.processAndWait(cluster.getSolrClient(), 60000);
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c163658/solr/core/src/test/org/apache/solr/cloud/ShardSplitTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/ShardSplitTest.java b/solr/core/src/test/org/apache/solr/cloud/ShardSplitTest.java
index 644221b..21f5b3c 100644
--- a/solr/core/src/test/org/apache/solr/cloud/ShardSplitTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/ShardSplitTest.java
@@ -932,9 +932,7 @@ public class ShardSplitTest extends BasicDistributedZkTest {
         .getBaseURL();
     baseUrl = baseUrl.substring(0, baseUrl.length() - "collection1".length());
 
-    try (HttpSolrClient baseServer = getHttpSolrClient(baseUrl)) {
-      baseServer.setConnectionTimeout(30000);
-      baseServer.setSoTimeout(60000 * 5);
+    try (HttpSolrClient baseServer = getHttpSolrClient(baseUrl, 30000, 60000 * 5)) {
       baseServer.request(request);
     }
   }
@@ -1007,15 +1005,13 @@ public class ShardSplitTest extends BasicDistributedZkTest {
 
   @Override
   protected SolrClient createNewSolrClient(String collection, String baseUrl) {
-    HttpSolrClient client = (HttpSolrClient) super.createNewSolrClient(collection, baseUrl);
-    client.setSoTimeout(5 * 60 * 1000);
+    HttpSolrClient client = (HttpSolrClient) super.createNewSolrClient(collection, baseUrl, DEFAULT_CONNECTION_TIMEOUT, 5 * 60 * 1000);
     return client;
   }
 
   @Override
   protected SolrClient createNewSolrClient(int port) {
-    HttpSolrClient client = (HttpSolrClient) super.createNewSolrClient(port);
-    client.setSoTimeout(5 * 60 * 1000);
+    HttpSolrClient client = (HttpSolrClient) super.createNewSolrClient(port, DEFAULT_CONNECTION_TIMEOUT, 5 * 60 * 1000);
     return client;
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c163658/solr/core/src/test/org/apache/solr/cloud/SyncSliceTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/SyncSliceTest.java b/solr/core/src/test/org/apache/solr/cloud/SyncSliceTest.java
index e5ac96c..43dfe27 100644
--- a/solr/core/src/test/org/apache/solr/cloud/SyncSliceTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/SyncSliceTest.java
@@ -108,9 +108,8 @@ public class SyncSliceTest extends AbstractFullDistribZkTestBase {
         .getBaseURL();
     baseUrl = baseUrl.substring(0, baseUrl.length() - "collection1".length());
     
-    try (HttpSolrClient baseClient = getHttpSolrClient(baseUrl)) {
-      // we only set the connect timeout, not so timeout
-      baseClient.setConnectionTimeout(30000);
+    // we only set the connect timeout, not so timeout
+    try (HttpSolrClient baseClient = getHttpSolrClient(baseUrl, 30000)) {
       baseClient.request(request);
     }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c163658/solr/core/src/test/org/apache/solr/cloud/TestRandomRequestDistribution.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/TestRandomRequestDistribution.java b/solr/core/src/test/org/apache/solr/cloud/TestRandomRequestDistribution.java
index 55cdee8..31b8b9c 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestRandomRequestDistribution.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestRandomRequestDistribution.java
@@ -92,9 +92,7 @@ public class TestRandomRequestDistribution extends AbstractFullDistribZkTestBase
     assertEquals(1, replicas.size());
     String baseUrl = replicas.iterator().next().getStr(ZkStateReader.BASE_URL_PROP);
     if (!baseUrl.endsWith("/")) baseUrl += "/";
-    try (HttpSolrClient client = getHttpSolrClient(baseUrl + "a1x2")) {
-      client.setSoTimeout(5000);
-      client.setConnectionTimeout(2000);
+    try (HttpSolrClient client = getHttpSolrClient(baseUrl + "a1x2", 2000, 5000)) {
 
       log.info("Making requests to " + baseUrl + "a1x2");
       for (int i = 0; i < 10; i++) {
@@ -170,9 +168,7 @@ public class TestRandomRequestDistribution extends AbstractFullDistribZkTestBase
     if (!baseUrl.endsWith("/")) baseUrl += "/";
     String path = baseUrl + "football";
     log.info("Firing queries against path=" + path);
-    try (HttpSolrClient client = getHttpSolrClient(path)) {
-      client.setSoTimeout(5000);
-      client.setConnectionTimeout(2000);
+    try (HttpSolrClient client = getHttpSolrClient(path, 2000, 5000)) {
 
       SolrCore leaderCore = null;
       for (JettySolrRunner jetty : jettys) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c163658/solr/core/src/test/org/apache/solr/cloud/TestRequestStatusCollectionAPI.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/TestRequestStatusCollectionAPI.java b/solr/core/src/test/org/apache/solr/cloud/TestRequestStatusCollectionAPI.java
index d8b66e9..a560e75 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestRequestStatusCollectionAPI.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestRequestStatusCollectionAPI.java
@@ -189,8 +189,7 @@ public class TestRequestStatusCollectionAPI extends BasicDistributedZkTest {
     String baseUrl = ((HttpSolrClient) shardToJetty.get(SHARD1).get(0).client.solrClient).getBaseURL();
     baseUrl = baseUrl.substring(0, baseUrl.length() - "collection1".length());
 
-    try (HttpSolrClient baseServer = getHttpSolrClient(baseUrl)) {
-      baseServer.setConnectionTimeout(15000);
+    try (HttpSolrClient baseServer = getHttpSolrClient(baseUrl, 15000)) {
       return baseServer.request(request);
     }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c163658/solr/core/src/test/org/apache/solr/cloud/UnloadDistributedZkTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/UnloadDistributedZkTest.java b/solr/core/src/test/org/apache/solr/cloud/UnloadDistributedZkTest.java
index daa73cf..45f8b81 100644
--- a/solr/core/src/test/org/apache/solr/cloud/UnloadDistributedZkTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/UnloadDistributedZkTest.java
@@ -221,8 +221,7 @@ public class UnloadDistributedZkTest extends BasicDistributedZkTest {
     // so that we start with some versions when we reload...
     DirectUpdateHandler2.commitOnClose = false;
 
-    try (HttpSolrClient addClient = getHttpSolrClient(jettys.get(2).getBaseUrl() + "/unloadcollection_shard1_replica3")) {
-      addClient.setConnectionTimeout(30000);
+    try (HttpSolrClient addClient = getHttpSolrClient(jettys.get(2).getBaseUrl() + "/unloadcollection_shard1_replica3", 30000)) {
 
       // add a few docs
       for (int x = 20; x < 100; x++) {
@@ -235,9 +234,7 @@ public class UnloadDistributedZkTest extends BasicDistributedZkTest {
     //collectionClient.commit();
     
     // unload the leader
-    try (HttpSolrClient collectionClient = getHttpSolrClient(leaderProps.getBaseUrl())) {
-      collectionClient.setConnectionTimeout(15000);
-      collectionClient.setSoTimeout(30000);
+    try (HttpSolrClient collectionClient = getHttpSolrClient(leaderProps.getBaseUrl(), 15000, 30000)) {
 
       Unload unloadCmd = new Unload(false);
       unloadCmd.setCoreName(leaderProps.getCoreName());
@@ -259,9 +256,7 @@ public class UnloadDistributedZkTest extends BasicDistributedZkTest {
     // ensure there is a leader
     zkStateReader.getLeaderRetry("unloadcollection", "shard1", 15000);
 
-    try (HttpSolrClient addClient = getHttpSolrClient(jettys.get(1).getBaseUrl() + "/unloadcollection_shard1_replica2")) {
-      addClient.setConnectionTimeout(30000);
-      addClient.setSoTimeout(90000);
+    try (HttpSolrClient addClient = getHttpSolrClient(jettys.get(1).getBaseUrl() + "/unloadcollection_shard1_replica2", 30000, 90000)) {
 
       // add a few docs while the leader is down
       for (int x = 101; x < 200; x++) {
@@ -281,9 +276,7 @@ public class UnloadDistributedZkTest extends BasicDistributedZkTest {
 
     // unload the leader again
     leaderProps = getLeaderUrlFromZk("unloadcollection", "shard1");
-    try (HttpSolrClient collectionClient = getHttpSolrClient(leaderProps.getBaseUrl())) {
-      collectionClient.setConnectionTimeout(15000);
-      collectionClient.setSoTimeout(30000);
+    try (HttpSolrClient collectionClient = getHttpSolrClient(leaderProps.getBaseUrl(), 15000, 30000)) {
 
       Unload unloadCmd = new Unload(false);
       unloadCmd.setCoreName(leaderProps.getCoreName());
@@ -313,27 +306,21 @@ public class UnloadDistributedZkTest extends BasicDistributedZkTest {
 
     long found1, found3;
 
-    try (HttpSolrClient adminClient = getHttpSolrClient(jettys.get(1).getBaseUrl() + "/unloadcollection_shard1_replica2")) {
-      adminClient.setConnectionTimeout(15000);
-      adminClient.setSoTimeout(30000);
+    try (HttpSolrClient adminClient = getHttpSolrClient(jettys.get(1).getBaseUrl() + "/unloadcollection_shard1_replica2", 15000, 30000)) {
       adminClient.commit();
       SolrQuery q = new SolrQuery("*:*");
       q.set("distrib", false);
       found1 = adminClient.query(q).getResults().getNumFound();
     }
 
-    try (HttpSolrClient adminClient = getHttpSolrClient(jettys.get(2).getBaseUrl() + "/unloadcollection_shard1_replica3")) {
-      adminClient.setConnectionTimeout(15000);
-      adminClient.setSoTimeout(30000);
+    try (HttpSolrClient adminClient = getHttpSolrClient(jettys.get(2).getBaseUrl() + "/unloadcollection_shard1_replica3", 15000, 30000)) {
       adminClient.commit();
       SolrQuery q = new SolrQuery("*:*");
       q.set("distrib", false);
       found3 = adminClient.query(q).getResults().getNumFound();
     }
 
-    try (HttpSolrClient adminClient = getHttpSolrClient(jettys.get(3).getBaseUrl() + "/unloadcollection_shard1_replica4")) {
-      adminClient.setConnectionTimeout(15000);
-      adminClient.setSoTimeout(30000);
+    try (HttpSolrClient adminClient = getHttpSolrClient(jettys.get(3).getBaseUrl() + "/unloadcollection_shard1_replica4", 15000, 30000)) {
       adminClient.commit();
       SolrQuery q = new SolrQuery("*:*");
       q.set("distrib", false);
@@ -348,9 +335,7 @@ public class UnloadDistributedZkTest extends BasicDistributedZkTest {
   
   private void testUnloadLotsOfCores() throws Exception {
     JettySolrRunner jetty = jettys.get(0);
-    try (final HttpSolrClient adminClient = (HttpSolrClient) jetty.newClient()) {
-      adminClient.setConnectionTimeout(15000);
-      adminClient.setSoTimeout(60000);
+    try (final HttpSolrClient adminClient = (HttpSolrClient) jetty.newClient(15000, 60000)) {
       int numReplicas = atLeast(3);
       ThreadPoolExecutor executor = new ExecutorUtil.MDCAwareThreadPoolExecutor(0, Integer.MAX_VALUE,
           5, TimeUnit.SECONDS, new SynchronousQueue<Runnable>(),

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c163658/solr/core/src/test/org/apache/solr/cloud/hdfs/StressHdfsTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/hdfs/StressHdfsTest.java b/solr/core/src/test/org/apache/solr/cloud/hdfs/StressHdfsTest.java
index 0955493..62e3f5f 100644
--- a/solr/core/src/test/org/apache/solr/cloud/hdfs/StressHdfsTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/hdfs/StressHdfsTest.java
@@ -180,7 +180,7 @@ public class StressHdfsTest extends BasicDistributedZkTest {
     
     int i = 0;
     for (SolrClient client : clients) {
-      try (HttpSolrClient c = getHttpSolrClient(getBaseUrl(client) + "/" + DELETE_DATA_DIR_COLLECTION)) {
+      try (HttpSolrClient c = getHttpSolrClient(getBaseUrl(client) + "/" + DELETE_DATA_DIR_COLLECTION, 30000)) {
         int docCnt = random().nextInt(1000) + 1;
         for (int j = 0; j < docCnt; j++) {
           c.add(getDoc("id", i++, "txt_t", "just some random text for a doc"));
@@ -192,7 +192,6 @@ public class StressHdfsTest extends BasicDistributedZkTest {
           c.commit(true, true, true);
         }
         
-        c.setConnectionTimeout(30000);
         NamedList<Object> response = c.query(
             new SolrQuery().setRequestHandler("/admin/system")).getResponse();
         NamedList<Object> coreInfo = (NamedList<Object>) response.get("core");

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c163658/solr/core/src/test/org/apache/solr/core/OpenCloseCoreStressTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/core/OpenCloseCoreStressTest.java b/solr/core/src/test/org/apache/solr/core/OpenCloseCoreStressTest.java
index b4ec29f..0bdf90c 100644
--- a/solr/core/src/test/org/apache/solr/core/OpenCloseCoreStressTest.java
+++ b/solr/core/src/test/org/apache/solr/core/OpenCloseCoreStressTest.java
@@ -136,15 +136,11 @@ public class OpenCloseCoreStressTest extends SolrTestCaseJ4 {
     // Mostly to keep annoying logging messages from being sent out all the time.
 
     for (int idx = 0; idx < indexingThreads; ++idx) {
-      HttpSolrClient client = getHttpSolrClient(url);
-      client.setConnectionTimeout(30000);
-      client.setSoTimeout(60000);
+      HttpSolrClient client = getHttpSolrClient(url, 30000, 60000);
       indexingClients.add(client);
     }
     for (int idx = 0; idx < queryThreads; ++idx) {
-      HttpSolrClient client = getHttpSolrClient(url);
-      client.setConnectionTimeout(30000);
-      client.setSoTimeout(30000);
+      HttpSolrClient client = getHttpSolrClient(url, 30000, 30000);
       queryingClients.add(client);
     }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c163658/solr/core/src/test/org/apache/solr/handler/TestReplicationHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/handler/TestReplicationHandler.java b/solr/core/src/test/org/apache/solr/handler/TestReplicationHandler.java
index 8148b88..62c0aa2 100644
--- a/solr/core/src/test/org/apache/solr/handler/TestReplicationHandler.java
+++ b/solr/core/src/test/org/apache/solr/handler/TestReplicationHandler.java
@@ -164,9 +164,7 @@ public class TestReplicationHandler extends SolrTestCaseJ4 {
     try {
       // setup the client...
       final String baseUrl = buildUrl(port) + "/" + DEFAULT_TEST_CORENAME;
-      HttpSolrClient client = getHttpSolrClient(baseUrl);
-      client.setConnectionTimeout(15000);
-      client.setSoTimeout(90000);
+      HttpSolrClient client = getHttpSolrClient(baseUrl, 15000, 90000);
       return client;
     }
     catch (Exception ex) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c163658/solr/core/src/test/org/apache/solr/handler/TestReplicationHandlerBackup.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/handler/TestReplicationHandlerBackup.java b/solr/core/src/test/org/apache/solr/handler/TestReplicationHandlerBackup.java
index 1ea16a0..0439775 100644
--- a/solr/core/src/test/org/apache/solr/handler/TestReplicationHandlerBackup.java
+++ b/solr/core/src/test/org/apache/solr/handler/TestReplicationHandlerBackup.java
@@ -83,9 +83,7 @@ public class TestReplicationHandlerBackup extends SolrJettyTestBase {
     try {
       // setup the client...
       final String baseUrl = buildUrl(port, context);
-      HttpSolrClient client = getHttpSolrClient(baseUrl);
-      client.setConnectionTimeout(15000);
-      client.setSoTimeout(60000);
+      HttpSolrClient client = getHttpSolrClient(baseUrl, 15000, 60000);
       return client;
     }
     catch (Exception ex) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c163658/solr/core/src/test/org/apache/solr/handler/TestRestoreCore.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/handler/TestRestoreCore.java b/solr/core/src/test/org/apache/solr/handler/TestRestoreCore.java
index eaf773a..2d1ac7f 100644
--- a/solr/core/src/test/org/apache/solr/handler/TestRestoreCore.java
+++ b/solr/core/src/test/org/apache/solr/handler/TestRestoreCore.java
@@ -71,9 +71,7 @@ public class TestRestoreCore extends SolrJettyTestBase {
     try {
       // setup the client...
       final String baseUrl = buildUrl(port, context);
-      HttpSolrClient client = getHttpSolrClient(baseUrl);
-      client.setConnectionTimeout(15000);
-      client.setSoTimeout(60000);
+      HttpSolrClient client = getHttpSolrClient(baseUrl, 15000, 60000);
       return client;
     }
     catch (Exception ex) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c163658/solr/core/src/test/org/apache/solr/handler/admin/CoreAdminHandlerTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/handler/admin/CoreAdminHandlerTest.java b/solr/core/src/test/org/apache/solr/handler/admin/CoreAdminHandlerTest.java
index 06c30e6..b9a282a 100644
--- a/solr/core/src/test/org/apache/solr/handler/admin/CoreAdminHandlerTest.java
+++ b/solr/core/src/test/org/apache/solr/handler/admin/CoreAdminHandlerTest.java
@@ -259,18 +259,14 @@ public class CoreAdminHandlerTest extends SolrTestCaseJ4 {
     JettySolrRunner runner = new JettySolrRunner(solrHomeDirectory.getAbsolutePath(), buildJettyConfig("/solr"));
     runner.start();
 
-    try (HttpSolrClient client = getHttpSolrClient(runner.getBaseUrl() + "/corex")) {
-      client.setConnectionTimeout(SolrTestCaseJ4.DEFAULT_CONNECTION_TIMEOUT);
-      client.setSoTimeout(SolrTestCaseJ4.DEFAULT_CONNECTION_TIMEOUT);
+    try (HttpSolrClient client = getHttpSolrClient(runner.getBaseUrl() + "/corex", DEFAULT_CONNECTION_TIMEOUT, DEFAULT_CONNECTION_TIMEOUT)) {
       SolrInputDocument doc = new SolrInputDocument();
       doc.addField("id", "123");
       client.add(doc);
       client.commit();
     }
 
-    try (HttpSolrClient client = getHttpSolrClient(runner.getBaseUrl().toString())) {
-      client.setConnectionTimeout(SolrTestCaseJ4.DEFAULT_CONNECTION_TIMEOUT);
-      client.setSoTimeout(SolrTestCaseJ4.DEFAULT_CONNECTION_TIMEOUT);
+    try (HttpSolrClient client = getHttpSolrClient(runner.getBaseUrl().toString(), DEFAULT_CONNECTION_TIMEOUT, DEFAULT_CONNECTION_TIMEOUT)) {
       CoreAdminRequest.Unload req = new CoreAdminRequest.Unload(false);
       req.setDeleteInstanceDir(true);
       req.setCoreName("corex");
@@ -295,25 +291,22 @@ public class CoreAdminHandlerTest extends SolrTestCaseJ4 {
     JettySolrRunner runner = new JettySolrRunner(solrHomeDirectory.getAbsolutePath(), buildJettyConfig("/solr"));
     runner.start();
 
-    try (HttpSolrClient client = getHttpSolrClient(runner.getBaseUrl() + "/corex")) {
-      client.setConnectionTimeout(SolrTestCaseJ4.DEFAULT_CONNECTION_TIMEOUT);
-      client.setSoTimeout(SolrTestCaseJ4.DEFAULT_CONNECTION_TIMEOUT);
+    try (HttpSolrClient client = getHttpSolrClient(runner.getBaseUrl() + "/corex", DEFAULT_CONNECTION_TIMEOUT,
+        DEFAULT_CONNECTION_TIMEOUT)) {
       SolrInputDocument doc = new SolrInputDocument();
       doc.addField("id", "123");
       client.add(doc);
       client.commit();
     }
 
-    try (HttpSolrClient client = getHttpSolrClient(runner.getBaseUrl() + "/corex")) {
-      client.setConnectionTimeout(SolrTestCaseJ4.DEFAULT_CONNECTION_TIMEOUT);
-      client.setSoTimeout(SolrTestCaseJ4.DEFAULT_CONNECTION_TIMEOUT);
+    try (HttpSolrClient client = getHttpSolrClient(runner.getBaseUrl() + "/corex", DEFAULT_CONNECTION_TIMEOUT,
+        DEFAULT_CONNECTION_TIMEOUT)) {
       QueryResponse result = client.query(new SolrQuery("id:*"));
       assertEquals(1,result.getResults().getNumFound());
     }
     
-    try (HttpSolrClient client = getHttpSolrClient(runner.getBaseUrl().toString())) {
-      client.setConnectionTimeout(SolrTestCaseJ4.DEFAULT_CONNECTION_TIMEOUT);
-      client.setSoTimeout(SolrTestCaseJ4.DEFAULT_CONNECTION_TIMEOUT);
+    try (HttpSolrClient client = getHttpSolrClient(runner.getBaseUrl().toString(), DEFAULT_CONNECTION_TIMEOUT,
+        DEFAULT_CONNECTION_TIMEOUT)) {
       CoreAdminRequest.Unload req = new CoreAdminRequest.Unload(false);
       req.setDeleteInstanceDir(false);//random().nextBoolean());
       req.setCoreName("corex");
@@ -321,9 +314,8 @@ public class CoreAdminHandlerTest extends SolrTestCaseJ4 {
     }
 
     HttpSolrClient.RemoteSolrException rse = expectThrows(HttpSolrClient.RemoteSolrException.class, () -> {
-      try (HttpSolrClient client = getHttpSolrClient(runner.getBaseUrl() + "/corex")) {
-        client.setConnectionTimeout(SolrTestCaseJ4.DEFAULT_CONNECTION_TIMEOUT);
-        client.setSoTimeout(SolrTestCaseJ4.DEFAULT_CONNECTION_TIMEOUT * 1000);
+      try (HttpSolrClient client = getHttpSolrClient(runner.getBaseUrl() + "/corex", DEFAULT_CONNECTION_TIMEOUT,
+          DEFAULT_CONNECTION_TIMEOUT * 1000)) {
         client.query(new SolrQuery("id:*"));
       } finally {
         runner.stop();
@@ -344,9 +336,7 @@ public class CoreAdminHandlerTest extends SolrTestCaseJ4 {
     JettySolrRunner runner = new JettySolrRunner(solrHomeDirectory.getAbsolutePath(), buildJettyConfig("/solr"));
     runner.start();
 
-    try (HttpSolrClient client = getHttpSolrClient(runner.getBaseUrl() + "/corex")) {
-      client.setConnectionTimeout(SolrTestCaseJ4.DEFAULT_CONNECTION_TIMEOUT);
-      client.setSoTimeout(SolrTestCaseJ4.DEFAULT_CONNECTION_TIMEOUT);
+    try (HttpSolrClient client = getHttpSolrClient(runner.getBaseUrl() + "/corex", DEFAULT_CONNECTION_TIMEOUT, DEFAULT_CONNECTION_TIMEOUT)) {
       SolrInputDocument doc = new SolrInputDocument();
       doc.addField("id", "123");
       client.add(doc);
@@ -365,9 +355,7 @@ public class CoreAdminHandlerTest extends SolrTestCaseJ4 {
     String top = SolrTestCaseJ4.TEST_HOME() + "/collection1/conf";
     FileUtils.copyFile(new File(top, "bad-error-solrconfig.xml"), new File(subHome, "solrconfig.xml"));
 
-    try (HttpSolrClient client = getHttpSolrClient(runner.getBaseUrl().toString())) {
-      client.setConnectionTimeout(SolrTestCaseJ4.DEFAULT_CONNECTION_TIMEOUT);
-      client.setSoTimeout(SolrTestCaseJ4.DEFAULT_CONNECTION_TIMEOUT);
+    try (HttpSolrClient client = getHttpSolrClient(runner.getBaseUrl().toString(), DEFAULT_CONNECTION_TIMEOUT, DEFAULT_CONNECTION_TIMEOUT)) {
       try {
         CoreAdminRequest.reloadCore("corex", client);
       } catch (Exception e) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c163658/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrClient.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrClient.java b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrClient.java
index 6cd5caf..e660fd5 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrClient.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrClient.java
@@ -195,6 +195,10 @@ public class CloudSolrClient extends SolrClient {
     this.retryExpiryTime = TimeUnit.NANOSECONDS.convert(secs, TimeUnit.SECONDS);
   }
 
+  /**
+   * @deprecated since 7.0  Use {@link Builder} methods instead. 
+   */
+  @Deprecated
   public void setSoTimeout(int timeout) {
     lbClient.setSoTimeout(timeout);
   }
@@ -264,14 +268,29 @@ public class CloudSolrClient extends SolrClient {
     }
     this.clientIsInternal = builder.httpClient == null;
     this.shutdownLBHttpSolrServer = builder.loadBalancedSolrClient == null;
-    if(builder.lbClientBuilder != null) builder.loadBalancedSolrClient = builder.lbClientBuilder.build();
+    if(builder.lbClientBuilder != null) {
+      propagateLBClientConfigOptions(builder);
+      builder.loadBalancedSolrClient = builder.lbClientBuilder.build();
+    }
     if(builder.loadBalancedSolrClient != null) builder.httpClient = builder.loadBalancedSolrClient.getHttpClient();
     this.myClient = (builder.httpClient == null) ? HttpClientUtil.createClient(null) : builder.httpClient;
-    if (builder.loadBalancedSolrClient == null) builder.loadBalancedSolrClient = createLBHttpSolrClient(myClient);
+    if (builder.loadBalancedSolrClient == null) builder.loadBalancedSolrClient = createLBHttpSolrClient(builder, myClient);
     this.lbClient = builder.loadBalancedSolrClient;
     this.updatesToLeaders = builder.shardLeadersOnly;
     this.directUpdatesToLeadersOnly = builder.directUpdatesToLeadersOnly;
   }
+  
+  private void propagateLBClientConfigOptions(Builder builder) {
+    final LBHttpSolrClient.Builder lbBuilder = builder.lbClientBuilder;
+    
+    if (builder.connectionTimeoutMillis != null) {
+      lbBuilder.withConnectionTimeout(builder.connectionTimeoutMillis);
+    }
+    
+    if (builder.socketTimeoutMillis != null) {
+      lbBuilder.withSocketTimeout(builder.socketTimeoutMillis);
+    }
+  }
 
   /**Sets the cache ttl for DocCollection Objects cached  . This is only applicable for collections which are persisted outside of clusterstate.json
    * @param seconds ttl value in seconds
@@ -1292,6 +1311,10 @@ public class CloudSolrClient extends SolrClient {
     return results;
   }
   
+  /**
+   * @deprecated since 7.0  Use {@link Builder} methods instead. 
+   */
+  @Deprecated
   public void setConnectionTimeout(int timeout) {
     this.lbClient.setConnectionTimeout(timeout); 
   }
@@ -1325,10 +1348,16 @@ public class CloudSolrClient extends SolrClient {
     return true;
   }
 
-  private static LBHttpSolrClient createLBHttpSolrClient(HttpClient httpClient) {
-    final LBHttpSolrClient lbClient = new LBHttpSolrClient.Builder()
-        .withHttpClient(httpClient)
-        .build();
+  private static LBHttpSolrClient createLBHttpSolrClient(Builder cloudSolrClientBuilder, HttpClient httpClient) {
+    final LBHttpSolrClient.Builder lbBuilder = new LBHttpSolrClient.Builder();
+    lbBuilder.withHttpClient(httpClient);
+    if (cloudSolrClientBuilder.connectionTimeoutMillis != null) {
+      lbBuilder.withConnectionTimeout(cloudSolrClientBuilder.connectionTimeoutMillis);
+    }
+    if (cloudSolrClientBuilder.socketTimeoutMillis != null) {
+      lbBuilder.withSocketTimeout(cloudSolrClientBuilder.socketTimeoutMillis);
+    }
+    final LBHttpSolrClient lbClient = lbBuilder.build();
     lbClient.setRequestWriter(new BinaryRequestWriter());
     lbClient.setParser(new BinaryResponseParser());
     
@@ -1348,6 +1377,8 @@ public class CloudSolrClient extends SolrClient {
     protected boolean shardLeadersOnly;
     protected boolean directUpdatesToLeadersOnly;
     protected ClusterStateProvider stateProvider;
+    protected Integer connectionTimeoutMillis;
+    protected Integer socketTimeoutMillis;
 
 
     public Builder() {
@@ -1483,6 +1514,30 @@ public class CloudSolrClient extends SolrClient {
       this.stateProvider = stateProvider;
       return this;
     }
+    
+    /**
+     * Tells {@link Builder} that created clients should obey the following timeout when connecting to Solr servers.
+     */
+    public Builder withConnectionTimeout(int connectionTimeoutMillis) {
+      if (connectionTimeoutMillis <= 0) {
+        throw new IllegalArgumentException("connectionTimeoutMillis must be a positive integer.");
+      }
+      
+      this.connectionTimeoutMillis = connectionTimeoutMillis;
+      return this;
+    }
+    
+    /**
+     * Tells {@link Builder} that created clients should set the following read timeout on all sockets.
+     */
+    public Builder withSocketTimeout(int socketTimeoutMillis) {
+      if (socketTimeoutMillis <= 0) {
+        throw new IllegalArgumentException("socketTimeoutMillis must be a positive integer.");
+      }
+      
+      this.socketTimeoutMillis = socketTimeoutMillis;
+      return this;
+    }
 
     /**
      * Create a {@link CloudSolrClient} based on the provided configuration.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c163658/solr/solrj/src/java/org/apache/solr/client/solrj/impl/ConcurrentUpdateSolrClient.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/ConcurrentUpdateSolrClient.java b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/ConcurrentUpdateSolrClient.java
index fead54f..2d9bfb1 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/ConcurrentUpdateSolrClient.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/ConcurrentUpdateSolrClient.java
@@ -129,6 +129,8 @@ public class ConcurrentUpdateSolrClient extends SolrClient {
     this.threadCount = builder.threadCount;
     this.runners = new LinkedList<>();
     this.streamDeletes = builder.streamDeletes;
+    this.connectionTimeout = builder.connectionTimeoutMillis;
+    this.soTimeout = builder.socketTimeoutMillis;
     
     if (builder.executorService != null) {
       this.scheduler = builder.executorService;
@@ -703,6 +705,10 @@ public class ConcurrentUpdateSolrClient extends SolrClient {
     }
   }
   
+  /**
+   * @deprecated since 7.0  Use {@link Builder} methods instead. 
+   */
+  @Deprecated
   public void setConnectionTimeout(int timeout) {
     this.connectionTimeout = timeout;
   }
@@ -710,7 +716,10 @@ public class ConcurrentUpdateSolrClient extends SolrClient {
   /**
    * set soTimeout (read timeout) on the underlying HttpConnectionManager. This is desirable for queries, but probably
    * not for indexing.
+   * 
+   * @deprecated since 7.0  Use {@link Builder} methods instead. 
    */
+  @Deprecated
   public void setSoTimeout(int timeout) {
     this.soTimeout = timeout;
   }
@@ -768,6 +777,8 @@ public class ConcurrentUpdateSolrClient extends SolrClient {
     protected int threadCount;
     protected ExecutorService executorService;
     protected boolean streamDeletes;
+    protected Integer connectionTimeoutMillis;
+    protected Integer socketTimeoutMillis;
 
     /**
      * Create a Builder object, based on the provided Solr URL.
@@ -850,6 +861,31 @@ public class ConcurrentUpdateSolrClient extends SolrClient {
     }
     
     /**
+     * Tells {@link Builder} that created clients should obey the following timeout when connecting to Solr servers.
+     */
+    public Builder withConnectionTimeout(int connectionTimeoutMillis) {
+      if (connectionTimeoutMillis <= 0) {
+        throw new IllegalArgumentException("connectionTimeoutMillis must be a positive integer.");
+      }
+      
+      this.connectionTimeoutMillis = connectionTimeoutMillis;
+      return this;
+    }
+    
+    /**
+     * Tells {@link Builder} that created clients should set the following read timeout on all sockets.
+     */
+    public Builder withSocketTimeout(int socketTimeoutMillis) {
+      if (socketTimeoutMillis <= 0) {
+        throw new IllegalArgumentException("socketTimeoutMillis must be a positive integer.");
+      }
+      
+      this.socketTimeoutMillis = socketTimeoutMillis;
+      return this;
+    }
+
+    
+    /**
      * Create a {@link ConcurrentUpdateSolrClient} based on the provided configuration options.
      */
     public ConcurrentUpdateSolrClient build() {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c163658/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpSolrClient.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpSolrClient.java b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpSolrClient.java
index ac85584..1b0354f 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpSolrClient.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpSolrClient.java
@@ -199,6 +199,8 @@ public class HttpSolrClient extends SolrClient {
     
     this.parser = builder.responseParser;
     this.invariantParams = builder.invariantParams;
+    this.connectionTimeout = builder.connectionTimeoutMillis;
+    this.soTimeout = builder.socketTimeoutMillis;
   }
 
   public Set<String> getQueryParams() {
@@ -713,7 +715,10 @@ public class HttpSolrClient extends SolrClient {
    * 
    * @param timeout
    *          Timeout in milliseconds
-   **/
+   *          
+   * @deprecated since 7.0  Use {@link Builder} methods instead. 
+   */
+  @Deprecated
   public void setConnectionTimeout(int timeout) {
     this.connectionTimeout = timeout;
   }
@@ -724,7 +729,10 @@ public class HttpSolrClient extends SolrClient {
    * 
    * @param timeout
    *          Timeout in milliseconds
-   **/
+   *          
+s   * @deprecated since 7.0  Use {@link Builder} methods instead. 
+   */
+  @Deprecated
   public void setSoTimeout(int timeout) {
     this.soTimeout = timeout;
   }
@@ -825,6 +833,8 @@ public class HttpSolrClient extends SolrClient {
     protected ResponseParser responseParser;
     protected boolean compression;
     protected ModifiableSolrParams invariantParams = new ModifiableSolrParams();
+    protected Integer connectionTimeoutMillis;
+    protected Integer socketTimeoutMillis;
 
     public Builder() {
       this.responseParser = new BinaryResponseParser();
@@ -931,6 +941,30 @@ public class HttpSolrClient extends SolrClient {
       this.invariantParams.add(params);
       return this;
     }
+    
+    /**
+     * Tells {@link Builder} that created clients should obey the following timeout when connecting to Solr servers.
+     */
+    public Builder withConnectionTimeout(int connectionTimeoutMillis) {
+      if (connectionTimeoutMillis <= 0) {
+        throw new IllegalArgumentException("connectionTimeoutMillis must be a positive integer.");
+      }
+      
+      this.connectionTimeoutMillis = connectionTimeoutMillis;
+      return this;
+    }
+    
+    /**
+     * Tells {@link Builder} that created clients should set the following read timeout on all sockets.
+     */
+    public Builder withSocketTimeout(int socketTimeoutMillis) {
+      if (socketTimeoutMillis <= 0) {
+        throw new IllegalArgumentException("socketTimeoutMillis must be a positive integer.");
+      }
+      
+      this.socketTimeoutMillis = socketTimeoutMillis;
+      return this;
+    }
 
     /**
      * Create a {@link HttpSolrClient} based on provided configuration.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c163658/solr/solrj/src/java/org/apache/solr/client/solrj/impl/LBHttpSolrClient.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/LBHttpSolrClient.java b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/LBHttpSolrClient.java
index b96c935..2dfd4b4 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/LBHttpSolrClient.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/LBHttpSolrClient.java
@@ -268,7 +268,8 @@ public class LBHttpSolrClient extends SolrClient {
     this.clientIsInternal = builder.httpClient == null;
     this.httpSolrClientBuilder = builder.httpSolrClientBuilder;
     this.httpClient = builder.httpClient == null ? constructClient(builder.baseSolrUrls.toArray(new String[builder.baseSolrUrls.size()])) : builder.httpClient;
-    
+    this.connectionTimeout = builder.connectionTimeoutMillis;
+    this.soTimeout = builder.socketTimeoutMillis;    
     this.parser = builder.responseParser;
 
     if (! builder.baseSolrUrls.isEmpty()) {
@@ -316,16 +317,28 @@ public class LBHttpSolrClient extends SolrClient {
     HttpSolrClient client;
     if (httpSolrClientBuilder != null) {
       synchronized (this) {
-        client = httpSolrClientBuilder
+        httpSolrClientBuilder
             .withBaseSolrUrl(server)
-            .withHttpClient(httpClient)
-            .build();
+            .withHttpClient(httpClient);
+        if (connectionTimeout != null) {
+          httpSolrClientBuilder.withConnectionTimeout(connectionTimeout);
+        }
+        if (soTimeout != null) {
+          httpSolrClientBuilder.withSocketTimeout(soTimeout);
+        }
+        client = httpSolrClientBuilder.build();
       }
     } else {
-      client = new HttpSolrClient.Builder(server)
+      final HttpSolrClient.Builder clientBuilder = new HttpSolrClient.Builder(server)
           .withHttpClient(httpClient)
-          .withResponseParser(parser)
-          .build();
+          .withResponseParser(parser);
+      if (connectionTimeout != null) {
+        clientBuilder.withConnectionTimeout(connectionTimeout);
+      }
+      if (soTimeout != null) {
+        clientBuilder.withSocketTimeout(soTimeout);
+      }
+      client = clientBuilder.build();
     }
     if (requestWriter != null) {
       client.setRequestWriter(requestWriter);
@@ -558,6 +571,10 @@ public class LBHttpSolrClient extends SolrClient {
     return null;
   }
 
+  /**
+   * @deprecated since 7.0  Use {@link Builder} methods instead. 
+   */
+  @Deprecated
   public void setConnectionTimeout(int timeout) {
     this.connectionTimeout = timeout;
     synchronized (aliveServers) {
@@ -575,7 +592,10 @@ public class LBHttpSolrClient extends SolrClient {
   /**
    * set soTimeout (read timeout) on the underlying HttpConnectionManager. This is desirable for queries, but probably
    * not for indexing.
+   *
+   * @deprecated since 7.0  Use {@link Builder} methods instead. 
    */
+  @Deprecated
   public void setSoTimeout(int timeout) {
     this.soTimeout = timeout;
     synchronized (aliveServers) {
@@ -866,6 +886,8 @@ public class LBHttpSolrClient extends SolrClient {
     protected HttpClient httpClient;
     protected ResponseParser responseParser;
     protected HttpSolrClient.Builder httpSolrClientBuilder;
+    protected Integer connectionTimeoutMillis;
+    protected Integer socketTimeoutMillis;
 
     public Builder() {
       this.baseSolrUrls = new ArrayList<>();
@@ -958,6 +980,30 @@ public class LBHttpSolrClient extends SolrClient {
       this.httpSolrClientBuilder = builder;
       return this;
     }
+    
+    /**
+     * Tells {@link Builder} that created clients should obey the following timeout when connecting to Solr servers.
+     */
+    public Builder withConnectionTimeout(int connectionTimeoutMillis) {
+      if (connectionTimeoutMillis <= 0) {
+        throw new IllegalArgumentException("connectionTimeoutMillis must be a positive integer.");
+      }
+      
+      this.connectionTimeoutMillis = connectionTimeoutMillis;
+      return this;
+    }
+    
+    /**
+     * Tells {@link Builder} that created clients should set the following read timeout on all sockets.
+     */
+    public Builder withSocketTimeout(int socketTimeoutMillis) {
+      if (socketTimeoutMillis <= 0) {
+        throw new IllegalArgumentException("socketTimeoutMillis must be a positive integer.");
+      }
+      
+      this.socketTimeoutMillis = socketTimeoutMillis;
+      return this;
+    }
 
     /**
      * Create a {@link HttpSolrClient} based on provided configuration.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c163658/solr/solrj/src/test/org/apache/solr/client/solrj/SolrExampleBinaryTest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/SolrExampleBinaryTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/SolrExampleBinaryTest.java
index dc53517..b1f1ee9 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/SolrExampleBinaryTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/SolrExampleBinaryTest.java
@@ -40,8 +40,7 @@ public class SolrExampleBinaryTest extends SolrExampleTests {
     try {
       // setup the server...
       String url = jetty.getBaseUrl().toString() + "/collection1";
-      HttpSolrClient client = getHttpSolrClient(url);
-      client.setConnectionTimeout(DEFAULT_CONNECTION_TIMEOUT);
+      HttpSolrClient client = getHttpSolrClient(url, DEFAULT_CONNECTION_TIMEOUT);
       client.setUseMultiPartPost(random().nextBoolean());
 
       // where the magic happens

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c163658/solr/solrj/src/test/org/apache/solr/client/solrj/SolrExampleXMLTest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/SolrExampleXMLTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/SolrExampleXMLTest.java
index 92ab1ec..5290347 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/SolrExampleXMLTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/SolrExampleXMLTest.java
@@ -37,9 +37,8 @@ public class SolrExampleXMLTest extends SolrExampleTests {
   public SolrClient createNewSolrClient() {
     try {
       String url = jetty.getBaseUrl().toString() + "/collection1";
-      HttpSolrClient client = getHttpSolrClient(url);
+      HttpSolrClient client = getHttpSolrClient(url, DEFAULT_CONNECTION_TIMEOUT);
       client.setUseMultiPartPost(random().nextBoolean());
-      client.setConnectionTimeout(DEFAULT_CONNECTION_TIMEOUT);
       client.setParser(new XMLResponseParser());
       client.setRequestWriter(new RequestWriter());
       return client;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c163658/solr/solrj/src/test/org/apache/solr/client/solrj/SolrExceptionTest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/SolrExceptionTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/SolrExceptionTest.java
index 6b92213..358dcc7 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/SolrExceptionTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/SolrExceptionTest.java
@@ -40,8 +40,7 @@ public class SolrExceptionTest extends LuceneTestCase {
       // switched to a local address to avoid going out on the net, ns lookup issues, etc.
       // set a 1ms timeout to let the connection fail faster.
       httpClient = HttpClientUtil.createClient(null);
-      try (HttpSolrClient client = getHttpSolrClient("http://[ff01::114]:11235/solr/", httpClient)) {
-        client.setConnectionTimeout(1);
+      try (HttpSolrClient client = getHttpSolrClient("http://[ff01::114]:11235/solr/", httpClient, 1)) {
         SolrQuery query = new SolrQuery("test123");
         client.query(query);
       }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c163658/solr/solrj/src/test/org/apache/solr/client/solrj/SolrSchemalessExampleTest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/SolrSchemalessExampleTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/SolrSchemalessExampleTest.java
index a7b4619..47faf78 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/SolrSchemalessExampleTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/SolrSchemalessExampleTest.java
@@ -133,8 +133,7 @@ public class SolrSchemalessExampleTest extends SolrExampleTestsBase {
     try {
       // setup the server...
       String url = jetty.getBaseUrl().toString() + "/collection1";
-      HttpSolrClient client = getHttpSolrClient(url);
-      client.setConnectionTimeout(DEFAULT_CONNECTION_TIMEOUT);
+      HttpSolrClient client = getHttpSolrClient(url, DEFAULT_CONNECTION_TIMEOUT);
       client.setUseMultiPartPost(random().nextBoolean());
       
       if (random().nextBoolean()) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c163658/solr/solrj/src/test/org/apache/solr/client/solrj/TestLBHttpSolrClient.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/TestLBHttpSolrClient.java b/solr/solrj/src/test/org/apache/solr/client/solrj/TestLBHttpSolrClient.java
index 3c1c45f..4aa5951 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/TestLBHttpSolrClient.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/TestLBHttpSolrClient.java
@@ -206,9 +206,7 @@ public class TestLBHttpSolrClient extends SolrTestCaseJ4 {
 
     CloseableHttpClient myHttpClient = HttpClientUtil.createClient(null);
     try {
-      LBHttpSolrClient client = getLBHttpSolrClient(myHttpClient, s);
-      client.setConnectionTimeout(500);
-      client.setSoTimeout(500);
+      LBHttpSolrClient client = getLBHttpSolrClient(myHttpClient, 500, 500, s);
       client.setAliveCheckInterval(500);
   
       // Kill a server and test again

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c163658/solr/solrj/src/test/org/apache/solr/client/solrj/impl/BasicHttpSolrClientTest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/impl/BasicHttpSolrClientTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/impl/BasicHttpSolrClientTest.java
index a5b45a2..42966c0 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/impl/BasicHttpSolrClientTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/impl/BasicHttpSolrClientTest.java
@@ -209,8 +209,7 @@ public class BasicHttpSolrClientTest extends SolrJettyTestBase {
   public void testTimeout() throws Exception {
 
     SolrQuery q = new SolrQuery("*:*");
-    try(HttpSolrClient client = getHttpSolrClient(jetty.getBaseUrl().toString() + "/slow/foo")) {
-      client.setSoTimeout(2000);
+    try(HttpSolrClient client = getHttpSolrClient(jetty.getBaseUrl().toString() + "/slow/foo", DEFAULT_CONNECTION_TIMEOUT, 2000)) {
       client.query(q, METHOD.GET);
       fail("No exception thrown.");
     } catch (SolrServerException e) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c163658/solr/solrj/src/test/org/apache/solr/client/solrj/impl/CloudSolrClientTest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/impl/CloudSolrClientTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/impl/CloudSolrClientTest.java
index 49cec5d..aadf0e2 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/impl/CloudSolrClientTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/impl/CloudSolrClientTest.java
@@ -463,9 +463,7 @@ public class CloudSolrClientTest extends SolrCloudTestCase {
       SolrServerException, IOException {
 
     NamedList<Object> resp;
-    try (HttpSolrClient client = getHttpSolrClient(baseUrl + "/"+ collectionName)) {
-      client.setConnectionTimeout(15000);
-      client.setSoTimeout(60000);
+    try (HttpSolrClient client = getHttpSolrClient(baseUrl + "/"+ collectionName, 15000, 60000)) {
       ModifiableSolrParams params = new ModifiableSolrParams();
       params.set("qt", "/admin/mbeans");
       params.set("stats", "true");

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c163658/solr/solrj/src/test/org/apache/solr/client/solrj/impl/HttpSolrClientConPoolTest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/impl/HttpSolrClientConPoolTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/impl/HttpSolrClientConPoolTest.java
index e38ec12..5c4aab5 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/impl/HttpSolrClientConPoolTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/impl/HttpSolrClientConPoolTest.java
@@ -70,8 +70,7 @@ public class HttpSolrClientConPoolTest extends SolrJettyTestBase {
       fooUrl = jetty.getBaseUrl().toString() + "/" + "collection1";
       CloseableHttpClient httpClient = HttpClientUtil.createClient(new ModifiableSolrParams(), pool,
             false /* let client shutdown it*/);
-      client1 = getHttpSolrClient(fooUrl, httpClient);
-      client1.setConnectionTimeout(DEFAULT_CONNECTION_TIMEOUT);
+      client1 = getHttpSolrClient(fooUrl, httpClient, DEFAULT_CONNECTION_TIMEOUT);
     }
     final String barUrl = yetty.getBaseUrl().toString() + "/" + "collection1";
     

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c163658/solr/test-framework/src/java/org/apache/solr/SolrJettyTestBase.java
----------------------------------------------------------------------
diff --git a/solr/test-framework/src/java/org/apache/solr/SolrJettyTestBase.java b/solr/test-framework/src/java/org/apache/solr/SolrJettyTestBase.java
index a936ee5..7703ecb 100644
--- a/solr/test-framework/src/java/org/apache/solr/SolrJettyTestBase.java
+++ b/solr/test-framework/src/java/org/apache/solr/SolrJettyTestBase.java
@@ -152,8 +152,7 @@ abstract public class SolrJettyTestBase extends SolrTestCaseJ4
       try {
         // setup the client...
         String url = jetty.getBaseUrl().toString() + "/" + "collection1";
-        HttpSolrClient client = getHttpSolrClient(url);
-        client.setConnectionTimeout(DEFAULT_CONNECTION_TIMEOUT);
+        HttpSolrClient client = getHttpSolrClient(url, DEFAULT_CONNECTION_TIMEOUT);
         return client;
       }
       catch( Exception ex ) {


[16/23] lucene-solr:jira/solr-10879: Add "Versions of Major Components" section

Posted by ab...@apache.org.
Add "Versions of Major Components" section


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

Branch: refs/heads/jira/solr-10879
Commit: 68a46acaf03dab785524e48b63dda8e90087d8ef
Parents: 157ff9a
Author: Jan Høydahl <ja...@apache.org>
Authored: Mon Jul 3 19:11:57 2017 +0200
Committer: Jan Høydahl <ja...@apache.org>
Committed: Mon Jul 3 19:14:18 2017 +0200

----------------------------------------------------------------------
 solr/CHANGES.txt | 9 +++++++++
 1 file changed, 9 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/68a46aca/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 1bc960d..fac639b 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -18,6 +18,15 @@ See the Quick Start guide at http://lucene.apache.org/solr/quickstart.html
 
 ==================  7.0.0 ==================
 
+Versions of Major Components
+---------------------
+Apache Tika 1.13
+Carrot2 3.15.0
+Velocity 1.7 and Velocity Tools 2.0
+Apache UIMA 2.3.1
+Apache ZooKeeper 3.4.10
+Jetty 9.3.14.v20161028
+
 Upgrading from Solr 6.x
 ----------------------
 


[14/23] lucene-solr:jira/solr-10879: Formatting fix for ref-guide

Posted by ab...@apache.org.
Formatting fix for ref-guide


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

Branch: refs/heads/jira/solr-10879
Commit: b978f37e658877f6ffb0241df70cd8255ac74433
Parents: bb2d6c1
Author: Jan Høydahl <ja...@apache.org>
Authored: Mon Jul 3 14:55:53 2017 +0200
Committer: Jan Høydahl <ja...@apache.org>
Committed: Mon Jul 3 14:56:06 2017 +0200

----------------------------------------------------------------------
 solr/solr-ref-guide/src/uploading-data-with-index-handlers.adoc | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b978f37e/solr/solr-ref-guide/src/uploading-data-with-index-handlers.adoc
----------------------------------------------------------------------
diff --git a/solr/solr-ref-guide/src/uploading-data-with-index-handlers.adoc b/solr/solr-ref-guide/src/uploading-data-with-index-handlers.adoc
index 6a9d350..a8c56bb 100644
--- a/solr/solr-ref-guide/src/uploading-data-with-index-handlers.adoc
+++ b/solr/solr-ref-guide/src/uploading-data-with-index-handlers.adoc
@@ -132,7 +132,7 @@ Documents can be deleted from the index in two ways. "Delete by ID" deletes the
 [IMPORTANT]
 ====
 
-When using the Join query parser in a Delete By Query, you should use the `score` parameter with a value of " `none`" to avoid a `ClassCastException`. See the section on the <<other-parsers.adoc#other-parsers,Join Query Parser>> for more details on the `score` parameter.
+When using the Join query parser in a Delete By Query, you should use the `score` parameter with a value of "none" to avoid a `ClassCastException`. See the section on the <<other-parsers.adoc#other-parsers,Join Query Parser>> for more details on the `score` parameter.
 
 ====
 


[10/23] lucene-solr:jira/solr-10879: Remove old unsupported codecs.

Posted by ab...@apache.org.
Remove old unsupported codecs.


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

Branch: refs/heads/jira/solr-10879
Commit: 9ca3dd26e423b396047326cd32b57d3cc6279f01
Parents: 869a48c
Author: Adrien Grand <jp...@gmail.com>
Authored: Mon Jul 3 14:16:18 2017 +0200
Committer: Adrien Grand <jp...@gmail.com>
Committed: Mon Jul 3 14:16:18 2017 +0200

----------------------------------------------------------------------
 .../lucene/codecs/lucene60/Lucene60Codec.java   | 177 -----------------
 .../apache/lucene/codecs/lucene60/package.html  |  25 ---
 .../lucene/codecs/lucene62/Lucene62Codec.java   | 176 -----------------
 .../apache/lucene/codecs/lucene62/package.html  |  25 ---
 .../java/org/apache/lucene/index/package.html   |  27 ---
 .../services/org.apache.lucene.codecs.Codec     |   2 -
 .../org.apache.lucene.codecs.DocValuesFormat    |   1 -
 .../lucene/codecs/lucene60/Lucene60RWCodec.java |  38 ----
 .../lucene/codecs/lucene62/Lucene62RWCodec.java |  44 -----
 .../lucene62/Lucene62RWSegmentInfoFormat.java   | 193 -------------------
 .../lucene62/TestLucene62SegmentInfoFormat.java |  48 -----
 .../index/TestBackwardsCompatibility.java       |  12 +-
 .../lucene/index/TestIndexWriterOnOldIndex.java |   7 +-
 .../lucene/index/TestManyPointsInOldIndex.java  |   1 +
 .../org/apache/lucene/index/dvupdates.6.0.0.zip | Bin 3420 -> 0 bytes
 .../org/apache/lucene/index/empty.6.0.0.zip     | Bin 225 -> 0 bytes
 .../org/apache/lucene/index/manypointsindex.zip | Bin 3739 -> 0 bytes
 .../org/apache/lucene/index/maxposindex.zip     | Bin 1702 -> 0 bytes
 .../org/apache/lucene/index/moreterms.6.0.0.zip | Bin 157215 -> 0 bytes
 .../org/apache/lucene/index/sorted.6.2.0.zip    | Bin 157007 -> 0 bytes
 .../org/apache/lucene/index/sorted.6.2.1.zip    | Bin 100974 -> 0 bytes
 .../org/apache/lucene/index/sorted.6.3.0.zip    | Bin 71090 -> 0 bytes
 .../index/unsupported.630.brokenoffsets.zip     | Bin 3203 -> 0 bytes
 .../index/unsupported.single-empty-doc.630.zip  | Bin 1363 -> 0 bytes
 24 files changed, 13 insertions(+), 763 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9ca3dd26/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
deleted file mode 100644
index f9b8671..0000000
--- a/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene60/Lucene60Codec.java
+++ /dev/null
@@ -1,177 +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;
-
-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
- * @deprecated Only for 6.0 back compat
- */
-@Deprecated
-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 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 NormsFormat normsFormat() {
-    return normsFormat;
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9ca3dd26/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene60/package.html
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene60/package.html b/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene60/package.html
deleted file mode 100644
index 6b4e234..0000000
--- a/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene60/package.html
+++ /dev/null
@@ -1,25 +0,0 @@
-<!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 6.0 file format.
-</body>
-</html>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9ca3dd26/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene62/Lucene62Codec.java
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene62/Lucene62Codec.java b/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene62/Lucene62Codec.java
deleted file mode 100644
index 3dd7daa..0000000
--- a/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene62/Lucene62Codec.java
+++ /dev/null
@@ -1,176 +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.lucene62;
-
-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.Lucene50StoredFieldsFormat;
-import org.apache.lucene.codecs.lucene50.Lucene50TermVectorsFormat;
-import org.apache.lucene.codecs.lucene50.Lucene50StoredFieldsFormat.Mode;
-import org.apache.lucene.codecs.lucene53.Lucene53NormsFormat;
-import org.apache.lucene.codecs.lucene60.Lucene60FieldInfosFormat;
-import org.apache.lucene.codecs.lucene60.Lucene60PointsFormat;
-import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat;
-import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
-
-/**
- * Implements the Lucene 6.2 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 Lucene62Codec extends Codec {
-  private final TermVectorsFormat vectorsFormat = new Lucene50TermVectorsFormat();
-  private final FieldInfosFormat fieldInfosFormat = new Lucene60FieldInfosFormat();
-  private final SegmentInfoFormat segmentInfosFormat = new Lucene62SegmentInfoFormat();
-  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 Lucene62Codec.this.getPostingsFormatForField(field);
-    }
-  };
-  
-  private final DocValuesFormat docValuesFormat = new PerFieldDocValuesFormat() {
-    @Override
-    public DocValuesFormat getDocValuesFormatForField(String field) {
-      return Lucene62Codec.this.getDocValuesFormatForField(field);
-    }
-  };
-  
-  private final StoredFieldsFormat storedFieldsFormat;
-
-  /** 
-   * Instantiates a new codec.
-   */
-  public Lucene62Codec() {
-    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 Lucene62Codec(Mode mode) {
-    super("Lucene62");
-    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 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 NormsFormat normsFormat() {
-    return normsFormat;
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9ca3dd26/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene62/package.html
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene62/package.html b/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene62/package.html
deleted file mode 100644
index 74e66d9..0000000
--- a/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene62/package.html
+++ /dev/null
@@ -1,25 +0,0 @@
-<!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 6.2 file format.
-</body>
-</html>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9ca3dd26/lucene/backward-codecs/src/java/org/apache/lucene/index/package.html
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/java/org/apache/lucene/index/package.html b/lucene/backward-codecs/src/java/org/apache/lucene/index/package.html
deleted file mode 100644
index 42ff91a..0000000
--- a/lucene/backward-codecs/src/java/org/apache/lucene/index/package.html
+++ /dev/null
@@ -1,27 +0,0 @@
-<!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.
--->
-<!-- not a package-info.java, because we already defined this package in core/ -->
-<html>
-<head>
-  <meta http-equiv="Content-Type" content="text/html; charset=iso-8859-1">
-  <title>Tools for handling backwards compatibility issues with indices.</title>
-</head>
-<body>
-Tools for handling backwards compatibility issues with indices.
-</body>
-</html>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9ca3dd26/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 6954d7a..4a812de 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
@@ -13,5 +13,3 @@
 #  See the License for the specific language governing permissions and
 #  limitations under the License.
 
-org.apache.lucene.codecs.lucene60.Lucene60Codec
-org.apache.lucene.codecs.lucene62.Lucene62Codec

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9ca3dd26/lucene/backward-codecs/src/resources/META-INF/services/org.apache.lucene.codecs.DocValuesFormat
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/resources/META-INF/services/org.apache.lucene.codecs.DocValuesFormat b/lucene/backward-codecs/src/resources/META-INF/services/org.apache.lucene.codecs.DocValuesFormat
index 26984ef..4a812de 100644
--- a/lucene/backward-codecs/src/resources/META-INF/services/org.apache.lucene.codecs.DocValuesFormat
+++ b/lucene/backward-codecs/src/resources/META-INF/services/org.apache.lucene.codecs.DocValuesFormat
@@ -13,4 +13,3 @@
 #  See the License for the specific language governing permissions and
 #  limitations under the License.
 
-org.apache.lucene.codecs.lucene54.Lucene54DocValuesFormat

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9ca3dd26/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene60/Lucene60RWCodec.java
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene60/Lucene60RWCodec.java b/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene60/Lucene60RWCodec.java
deleted file mode 100644
index eecfe09..0000000
--- a/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene60/Lucene60RWCodec.java
+++ /dev/null
@@ -1,38 +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;
-
-import org.apache.lucene.codecs.NormsFormat;
-import org.apache.lucene.codecs.SegmentInfoFormat;
-import org.apache.lucene.codecs.lucene50.Lucene50RWSegmentInfoFormat;
-import org.apache.lucene.codecs.lucene53.Lucene53RWNormsFormat;
-
-public class Lucene60RWCodec extends Lucene60Codec {
-
-  private final SegmentInfoFormat segmentInfoFormat = new Lucene50RWSegmentInfoFormat();
-  private final NormsFormat normsFormat = new Lucene53RWNormsFormat();
-
-  @Override
-  public SegmentInfoFormat segmentInfoFormat() {
-    return segmentInfoFormat;
-  }
-
-  @Override
-  public NormsFormat normsFormat() {
-    return normsFormat;
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9ca3dd26/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene62/Lucene62RWCodec.java
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene62/Lucene62RWCodec.java b/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene62/Lucene62RWCodec.java
deleted file mode 100644
index 34d3a7f..0000000
--- a/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene62/Lucene62RWCodec.java
+++ /dev/null
@@ -1,44 +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.lucene62;
-
-import org.apache.lucene.codecs.NormsFormat;
-import org.apache.lucene.codecs.SegmentInfoFormat;
-import org.apache.lucene.codecs.lucene53.Lucene53RWNormsFormat;
-import org.apache.lucene.codecs.lucene62.Lucene62Codec;
-
-/**
- * Read-write version of 6.2 codec for testing
- * @deprecated for test purposes only
- */
-@Deprecated
-public class Lucene62RWCodec extends Lucene62Codec {
-
-  private final SegmentInfoFormat segmentInfoFormat = new Lucene62RWSegmentInfoFormat();
-  private final NormsFormat normsFormat = new Lucene53RWNormsFormat();
-
-  @Override
-  public SegmentInfoFormat segmentInfoFormat() {
-    return segmentInfoFormat;
-  }
-  
-  @Override
-  public NormsFormat normsFormat() {
-    return normsFormat;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9ca3dd26/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene62/Lucene62RWSegmentInfoFormat.java
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene62/Lucene62RWSegmentInfoFormat.java b/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene62/Lucene62RWSegmentInfoFormat.java
deleted file mode 100644
index f2fbe9d..0000000
--- a/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene62/Lucene62RWSegmentInfoFormat.java
+++ /dev/null
@@ -1,193 +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.lucene62;
-
-import java.io.IOException;
-import java.util.Set;
-
-import org.apache.lucene.codecs.CodecUtil;
-import org.apache.lucene.index.IndexFileNames;
-import org.apache.lucene.index.SegmentInfo;
-import org.apache.lucene.search.Sort;
-import org.apache.lucene.search.SortField;
-import org.apache.lucene.search.SortedNumericSelector;
-import org.apache.lucene.search.SortedNumericSortField;
-import org.apache.lucene.search.SortedSetSelector;
-import org.apache.lucene.search.SortedSetSortField;
-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 6.2 SegmentInfoFormat for testing
- * @deprecated for test purposes only
- */
-@Deprecated
-public class Lucene62RWSegmentInfoFormat extends Lucene62SegmentInfoFormat {
-
-  @Override
-  public void write(Directory dir, SegmentInfo si, IOContext ioContext) throws IOException {
-    final String fileName = IndexFileNames.segmentFileName(si.name, "", Lucene62SegmentInfoFormat.SI_EXTENSION);
-
-    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,
-                                   Lucene62SegmentInfoFormat.CODEC_NAME,
-                                   Lucene62SegmentInfoFormat.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());
-
-      Sort indexSort = si.getIndexSort();
-      int numSortFields = indexSort == null ? 0 : indexSort.getSort().length;
-      output.writeVInt(numSortFields);
-      for (int i = 0; i < numSortFields; ++i) {
-        SortField sortField = indexSort.getSort()[i];
-        SortField.Type sortType = sortField.getType();
-        output.writeString(sortField.getField());
-        int sortTypeID;
-        switch (sortField.getType()) {
-          case STRING:
-            sortTypeID = 0;
-            break;
-          case LONG:
-            sortTypeID = 1;
-            break;
-          case INT:
-            sortTypeID = 2;
-            break;
-          case DOUBLE:
-            sortTypeID = 3;
-            break;
-          case FLOAT:
-            sortTypeID = 4;
-            break;
-          case CUSTOM:
-            if (sortField instanceof SortedSetSortField) {
-              sortTypeID = 5;
-              sortType = SortField.Type.STRING;
-            } else if (sortField instanceof SortedNumericSortField) {
-              sortTypeID = 6;
-              sortType = ((SortedNumericSortField) sortField).getNumericType();
-            } else {
-              throw new IllegalStateException("Unexpected SortedNumericSortField " + sortField);
-            }
-            break;
-          default:
-            throw new IllegalStateException("Unexpected sort type: " + sortField.getType());
-        }
-        output.writeVInt(sortTypeID);
-        if (sortTypeID == 5) {
-          SortedSetSortField ssf = (SortedSetSortField) sortField;
-          if (ssf.getSelector() == SortedSetSelector.Type.MIN) {
-            output.writeByte((byte) 0);
-          } else if (ssf.getSelector() == SortedSetSelector.Type.MAX) {
-            output.writeByte((byte) 1);
-          } else if (ssf.getSelector() == SortedSetSelector.Type.MIDDLE_MIN) {
-            output.writeByte((byte) 2);
-          } else if (ssf.getSelector() == SortedSetSelector.Type.MIDDLE_MAX) {
-            output.writeByte((byte) 3);
-          } else {
-            throw new IllegalStateException("Unexpected SortedSetSelector type: " + ssf.getSelector());
-          }
-        } else if (sortTypeID == 6) {
-          SortedNumericSortField snsf = (SortedNumericSortField) sortField;
-          if (snsf.getNumericType() == SortField.Type.LONG) {
-            output.writeByte((byte) 0);
-          } else if (snsf.getNumericType() == SortField.Type.INT) {
-            output.writeByte((byte) 1);
-          } else if (snsf.getNumericType() == SortField.Type.DOUBLE) {
-            output.writeByte((byte) 2);
-          } else if (snsf.getNumericType() == SortField.Type.FLOAT) {
-            output.writeByte((byte) 3);
-          } else {
-            throw new IllegalStateException("Unexpected SortedNumericSelector type: " + snsf.getNumericType());
-          }
-          if (snsf.getSelector() == SortedNumericSelector.Type.MIN) {
-            output.writeByte((byte) 0);
-          } else if (snsf.getSelector() == SortedNumericSelector.Type.MAX) {
-            output.writeByte((byte) 1);
-          } else {
-            throw new IllegalStateException("Unexpected sorted numeric selector type: " + snsf.getSelector());
-          }
-        }
-        output.writeByte((byte) (sortField.getReverse() ? 0 : 1));
-
-        // write missing value 
-        Object missingValue = sortField.getMissingValue();
-        if (missingValue == null) {
-          output.writeByte((byte) 0);
-        } else {
-          switch(sortType) {
-          case STRING:
-            if (missingValue == SortField.STRING_LAST) {
-              output.writeByte((byte) 1);
-            } else if (missingValue == SortField.STRING_FIRST) {
-              output.writeByte((byte) 2);
-            } else {
-              throw new AssertionError("unrecognized missing value for STRING field \"" + sortField.getField() + "\": " + missingValue);
-            }
-            break;
-          case LONG:
-            output.writeByte((byte) 1);
-            output.writeLong(((Long) missingValue).longValue());
-            break;
-          case INT:
-            output.writeByte((byte) 1);
-            output.writeInt(((Integer) missingValue).intValue());
-            break;
-          case DOUBLE:
-            output.writeByte((byte) 1);
-            output.writeLong(Double.doubleToLongBits(((Double) missingValue).doubleValue()));
-            break;
-          case FLOAT:
-            output.writeByte((byte) 1);
-            output.writeInt(Float.floatToIntBits(((Float) missingValue).floatValue()));
-            break;
-          default:
-            throw new IllegalStateException("Unexpected sort type: " + sortField.getType());
-          }
-        }
-      }
-
-      CodecUtil.writeFooter(output);
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9ca3dd26/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene62/TestLucene62SegmentInfoFormat.java
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene62/TestLucene62SegmentInfoFormat.java b/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene62/TestLucene62SegmentInfoFormat.java
deleted file mode 100644
index e0efa95..0000000
--- a/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene62/TestLucene62SegmentInfoFormat.java
+++ /dev/null
@@ -1,48 +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.lucene62;
-
-import org.apache.lucene.codecs.Codec;
-import org.apache.lucene.index.BaseSegmentInfoFormatTestCase;
-import org.apache.lucene.util.Version;
-
-/**
- * Tests Lucene62SegmentInfoFormat
- */
-public class TestLucene62SegmentInfoFormat extends BaseSegmentInfoFormatTestCase {
-
-  @Override
-  protected int getCreatedVersionMajor() {
-    return Version.LUCENE_6_2_0.major;
-  }
-
-  @Override
-  protected Version[] getVersions() {
-    return new Version[] { Version.LUCENE_6_2_0 };
-  }
-
-  @Override
-  protected Codec getCodec() {
-    return new Lucene62RWCodec();
-  }
-
-  @Override
-  protected boolean supportsMinVersion() {
-    return false;
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9ca3dd26/lucene/backward-codecs/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java b/lucene/backward-codecs/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java
index ad50382..22b0ab6 100644
--- a/lucene/backward-codecs/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java
+++ b/lucene/backward-codecs/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java
@@ -1436,9 +1436,10 @@ public class TestBackwardsCompatibility extends LuceneTestCase {
     }
   }
 
-  public static final String emptyIndex = "empty.6.0.0.zip";
+  public static final String emptyIndex = "empty.7.0.0.zip";
 
   public void testUpgradeEmptyOldIndex() throws Exception {
+    assumeTrue("Reenable when 7.0 is released", false);
     Path oldIndexDir = createTempDir("emptyIndex");
     TestUtil.unzip(getDataInputStream(emptyIndex), oldIndexDir);
     Directory dir = newFSDirectory(oldIndexDir);
@@ -1450,9 +1451,10 @@ public class TestBackwardsCompatibility extends LuceneTestCase {
     dir.close();
   }
 
-  public static final String moreTermsIndex = "moreterms.6.0.0.zip";
+  public static final String moreTermsIndex = "moreterms.7.0.0.zip";
 
   public void testMoreTerms() throws Exception {
+    assumeTrue("Reenable when 7.0 is released", false);
     Path oldIndexDir = createTempDir("moreterms");
     TestUtil.unzip(getDataInputStream(moreTermsIndex), oldIndexDir);
     Directory dir = newFSDirectory(oldIndexDir);
@@ -1462,7 +1464,7 @@ public class TestBackwardsCompatibility extends LuceneTestCase {
     dir.close();
   }
 
-  public static final String dvUpdatesIndex = "dvupdates.6.0.0.zip";
+  public static final String dvUpdatesIndex = "dvupdates.7.0.0.zip";
 
   private void assertNumericDocValues(LeafReader r, String f, String cf) throws IOException {
     NumericDocValues ndvf = r.getNumericDocValues(f);
@@ -1497,6 +1499,7 @@ public class TestBackwardsCompatibility extends LuceneTestCase {
   }
 
   public void testDocValuesUpdates() throws Exception {
+    assumeTrue("Reenable when 7.0 is released", false);
     Path oldIndexDir = createTempDir("dvupdates");
     TestUtil.unzip(getDataInputStream(dvUpdatesIndex), oldIndexDir);
     Directory dir = newFSDirectory(oldIndexDir);
@@ -1559,7 +1562,8 @@ public class TestBackwardsCompatibility extends LuceneTestCase {
   }
 
   public void testSortedIndex() throws Exception {
-    String[] versions = new String[] {"6.2.0", "6.2.1", "6.3.0"};
+    assumeTrue("Reenable when 7.0 is released", false);
+    String[] versions = new String[] {};
     for(String version : versions) {
       Path path = createTempDir("sorted");
       InputStream resource = TestBackwardsCompatibility.class.getResourceAsStream("sorted." + version + ".zip");

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9ca3dd26/lucene/backward-codecs/src/test/org/apache/lucene/index/TestIndexWriterOnOldIndex.java
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/index/TestIndexWriterOnOldIndex.java b/lucene/backward-codecs/src/test/org/apache/lucene/index/TestIndexWriterOnOldIndex.java
index b48b0fd..db33fb3 100644
--- a/lucene/backward-codecs/src/test/org/apache/lucene/index/TestIndexWriterOnOldIndex.java
+++ b/lucene/backward-codecs/src/test/org/apache/lucene/index/TestIndexWriterOnOldIndex.java
@@ -29,14 +29,15 @@ import org.apache.lucene.util.Version;
 public class TestIndexWriterOnOldIndex extends LuceneTestCase {
 
   public void testOpenModeAndCreatedVersion() throws IOException {
-    InputStream resource = getClass().getResourceAsStream("unsupported.index.single-empty-doc.630.zip");
+    assumeTrue("Reenable when 7.0 is released", false);
+    InputStream resource = getClass().getResourceAsStream("unsupported.index.single-empty-doc.7.0.0.zip");
     assertNotNull(resource);
     Path path = createTempDir();
     TestUtil.unzip(resource, path);
     Directory dir = newFSDirectory(path);
     for (OpenMode openMode : OpenMode.values()) {
       Directory tmpDir = newDirectory(dir);
-      assertEquals(6 /** 6.3.0 */, SegmentInfos.readLatestCommit(tmpDir).getIndexCreatedVersionMajor());
+      assertEquals(7 /** 7.0.0 */, SegmentInfos.readLatestCommit(tmpDir).getIndexCreatedVersionMajor());
       IndexWriter w = new IndexWriter(tmpDir, newIndexWriterConfig().setOpenMode(openMode));
       w.commit();
       w.close();
@@ -45,7 +46,7 @@ public class TestIndexWriterOnOldIndex extends LuceneTestCase {
           assertEquals(Version.LATEST.major, SegmentInfos.readLatestCommit(tmpDir).getIndexCreatedVersionMajor());
           break;
         default:
-          assertEquals(6 /** 6.3.0 */, SegmentInfos.readLatestCommit(tmpDir).getIndexCreatedVersionMajor());
+          assertEquals(7 /** 7.0.0 */, SegmentInfos.readLatestCommit(tmpDir).getIndexCreatedVersionMajor());
       }
       tmpDir.close();
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9ca3dd26/lucene/backward-codecs/src/test/org/apache/lucene/index/TestManyPointsInOldIndex.java
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/index/TestManyPointsInOldIndex.java b/lucene/backward-codecs/src/test/org/apache/lucene/index/TestManyPointsInOldIndex.java
index 043979b..c567bec 100644
--- a/lucene/backward-codecs/src/test/org/apache/lucene/index/TestManyPointsInOldIndex.java
+++ b/lucene/backward-codecs/src/test/org/apache/lucene/index/TestManyPointsInOldIndex.java
@@ -59,6 +59,7 @@ public class TestManyPointsInOldIndex extends LuceneTestCase {
   }
 
   public void testCheckOldIndex() throws IOException {
+    assumeTrue("Reenable when 7.0 is released", false);
     Path path = createTempDir("manypointsindex");
     InputStream resource = getClass().getResourceAsStream("manypointsindex.zip");
     assertNotNull("manypointsindex not found", resource);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9ca3dd26/lucene/backward-codecs/src/test/org/apache/lucene/index/dvupdates.6.0.0.zip
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/index/dvupdates.6.0.0.zip b/lucene/backward-codecs/src/test/org/apache/lucene/index/dvupdates.6.0.0.zip
deleted file mode 100644
index 38ac8e1..0000000
Binary files a/lucene/backward-codecs/src/test/org/apache/lucene/index/dvupdates.6.0.0.zip and /dev/null differ

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9ca3dd26/lucene/backward-codecs/src/test/org/apache/lucene/index/empty.6.0.0.zip
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/index/empty.6.0.0.zip b/lucene/backward-codecs/src/test/org/apache/lucene/index/empty.6.0.0.zip
deleted file mode 100644
index 8ca64d3..0000000
Binary files a/lucene/backward-codecs/src/test/org/apache/lucene/index/empty.6.0.0.zip and /dev/null differ

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9ca3dd26/lucene/backward-codecs/src/test/org/apache/lucene/index/manypointsindex.zip
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/index/manypointsindex.zip b/lucene/backward-codecs/src/test/org/apache/lucene/index/manypointsindex.zip
deleted file mode 100644
index c7c0bf7..0000000
Binary files a/lucene/backward-codecs/src/test/org/apache/lucene/index/manypointsindex.zip and /dev/null differ

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9ca3dd26/lucene/backward-codecs/src/test/org/apache/lucene/index/maxposindex.zip
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/index/maxposindex.zip b/lucene/backward-codecs/src/test/org/apache/lucene/index/maxposindex.zip
deleted file mode 100644
index aaabb56..0000000
Binary files a/lucene/backward-codecs/src/test/org/apache/lucene/index/maxposindex.zip and /dev/null differ

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9ca3dd26/lucene/backward-codecs/src/test/org/apache/lucene/index/moreterms.6.0.0.zip
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/index/moreterms.6.0.0.zip b/lucene/backward-codecs/src/test/org/apache/lucene/index/moreterms.6.0.0.zip
deleted file mode 100644
index 77c448d..0000000
Binary files a/lucene/backward-codecs/src/test/org/apache/lucene/index/moreterms.6.0.0.zip and /dev/null differ

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9ca3dd26/lucene/backward-codecs/src/test/org/apache/lucene/index/sorted.6.2.0.zip
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/index/sorted.6.2.0.zip b/lucene/backward-codecs/src/test/org/apache/lucene/index/sorted.6.2.0.zip
deleted file mode 100644
index b595d9b..0000000
Binary files a/lucene/backward-codecs/src/test/org/apache/lucene/index/sorted.6.2.0.zip and /dev/null differ

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9ca3dd26/lucene/backward-codecs/src/test/org/apache/lucene/index/sorted.6.2.1.zip
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/index/sorted.6.2.1.zip b/lucene/backward-codecs/src/test/org/apache/lucene/index/sorted.6.2.1.zip
deleted file mode 100644
index 40d1ccc..0000000
Binary files a/lucene/backward-codecs/src/test/org/apache/lucene/index/sorted.6.2.1.zip and /dev/null differ

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9ca3dd26/lucene/backward-codecs/src/test/org/apache/lucene/index/sorted.6.3.0.zip
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/index/sorted.6.3.0.zip b/lucene/backward-codecs/src/test/org/apache/lucene/index/sorted.6.3.0.zip
deleted file mode 100644
index 6d89d781..0000000
Binary files a/lucene/backward-codecs/src/test/org/apache/lucene/index/sorted.6.3.0.zip and /dev/null differ

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9ca3dd26/lucene/backward-codecs/src/test/org/apache/lucene/index/unsupported.630.brokenoffsets.zip
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/index/unsupported.630.brokenoffsets.zip b/lucene/backward-codecs/src/test/org/apache/lucene/index/unsupported.630.brokenoffsets.zip
deleted file mode 100644
index 3cf476a..0000000
Binary files a/lucene/backward-codecs/src/test/org/apache/lucene/index/unsupported.630.brokenoffsets.zip and /dev/null differ

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9ca3dd26/lucene/backward-codecs/src/test/org/apache/lucene/index/unsupported.single-empty-doc.630.zip
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/index/unsupported.single-empty-doc.630.zip b/lucene/backward-codecs/src/test/org/apache/lucene/index/unsupported.single-empty-doc.630.zip
deleted file mode 100644
index 1bf1d08..0000000
Binary files a/lucene/backward-codecs/src/test/org/apache/lucene/index/unsupported.single-empty-doc.630.zip and /dev/null differ


[05/23] lucene-solr:jira/solr-10879: Changes for version upgrade to 8.0 on master for the 7.0 release

Posted by ab...@apache.org.
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/189f2b3f/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene50/TestLucene50SegmentInfoFormat.java
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene50/TestLucene50SegmentInfoFormat.java b/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene50/TestLucene50SegmentInfoFormat.java
deleted file mode 100644
index 0a9bf79..0000000
--- a/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene50/TestLucene50SegmentInfoFormat.java
+++ /dev/null
@@ -1,50 +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 org.apache.lucene.codecs.Codec;
-import org.apache.lucene.codecs.lucene60.Lucene60RWCodec;
-import org.apache.lucene.index.BaseSegmentInfoFormatTestCase;
-import org.apache.lucene.util.Version;
-
-public class TestLucene50SegmentInfoFormat extends BaseSegmentInfoFormatTestCase {
-
-  @Override
-  protected Codec getCodec() {
-    return new Lucene60RWCodec();
-  }
-
-  @Override
-  protected int getCreatedVersionMajor() {
-    return Version.LUCENE_6_0_0.major;
-  }
-
-  @Override
-  protected Version[] getVersions() {
-    return new Version[] { Version.LUCENE_6_0_0 };
-  }
-
-  @Override
-  protected boolean supportsIndexSort() {
-    return false;
-  }
-
-  @Override
-  protected boolean supportsMinVersion() {
-    return false;
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/189f2b3f/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene53/Lucene53NormsConsumer.java
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene53/Lucene53NormsConsumer.java b/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene53/Lucene53NormsConsumer.java
deleted file mode 100644
index 833500c..0000000
--- a/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene53/Lucene53NormsConsumer.java
+++ /dev/null
@@ -1,159 +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.lucene53;
-
-
-import java.io.IOException;
-
-import org.apache.lucene.codecs.CodecUtil;
-import org.apache.lucene.codecs.LegacyDocValuesIterables;
-import org.apache.lucene.codecs.NormsConsumer;
-import org.apache.lucene.codecs.NormsProducer;
-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.util.IOUtils;
-
-import static org.apache.lucene.codecs.lucene53.Lucene53NormsFormat.VERSION_CURRENT;
-
-/**
- * Writer for {@link Lucene53NormsFormat}
- */
-class Lucene53NormsConsumer extends NormsConsumer { 
-  IndexOutput data, meta;
-  final int maxDoc;
-
-  Lucene53NormsConsumer(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, 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, VERSION_CURRENT, state.segmentInfo.getId(), state.segmentSuffix);
-      maxDoc = state.segmentInfo.maxDoc();
-      success = true;
-    } finally {
-      if (!success) {
-        IOUtils.closeWhileHandlingException(this);
-      }
-    }
-  }
-
-  @Override
-  public void addNormsField(FieldInfo field, NormsProducer normsProducer) throws IOException {
-    addNormsField(field, LegacyDocValuesIterables.normsIterable(field, normsProducer, maxDoc));
-  }
-
-  private void addNormsField(FieldInfo field, Iterable<Number> values) throws IOException {
-    meta.writeVInt(field.number);
-    long minValue = Long.MAX_VALUE;
-    long maxValue = Long.MIN_VALUE;
-    int count = 0;
-
-    for (Number nv : values) {
-      if (nv == null) {
-        throw new IllegalStateException("illegal norms data for field " + field.name + ", got null for value: " + count);
-      }
-      final long v = nv.longValue();
-      minValue = Math.min(minValue, v);
-      maxValue = Math.max(maxValue, v);
-      count++;
-    }
-
-    if (count != maxDoc) {
-      throw new IllegalStateException("illegal norms data for field " + field.name + ", expected count=" + maxDoc + ", got=" + count);
-    }
-
-    if (minValue == maxValue) {
-      addConstant(minValue);
-    } else if (minValue >= Byte.MIN_VALUE && maxValue <= Byte.MAX_VALUE) {
-      addByte1(values);
-    } else if (minValue >= Short.MIN_VALUE && maxValue <= Short.MAX_VALUE) {
-      addByte2(values);
-    } else if (minValue >= Integer.MIN_VALUE && maxValue <= Integer.MAX_VALUE) {
-      addByte4(values);
-    } else {
-      addByte8(values);
-    }
-  }
-
-  private void addConstant(long constant) throws IOException {
-    meta.writeByte((byte) 0);
-    meta.writeLong(constant);
-  }
-
-  private void addByte1(Iterable<Number> values) throws IOException {
-    meta.writeByte((byte) 1);
-    meta.writeLong(data.getFilePointer());
-
-    for (Number value : values) {
-      data.writeByte(value.byteValue());
-    }
-  }
-
-  private void addByte2(Iterable<Number> values) throws IOException {
-    meta.writeByte((byte) 2);
-    meta.writeLong(data.getFilePointer());
-
-    for (Number value : values) {
-      data.writeShort(value.shortValue());
-    }
-  }
-
-  private void addByte4(Iterable<Number> values) throws IOException {
-    meta.writeByte((byte) 4);
-    meta.writeLong(data.getFilePointer());
-
-    for (Number value : values) {
-      data.writeInt(value.intValue());
-    }
-  }
-
-  private void addByte8(Iterable<Number> values) throws IOException {
-    meta.writeByte((byte) 8);
-    meta.writeLong(data.getFilePointer());
-
-    for (Number value : values) {
-      data.writeLong(value.longValue());
-    }
-  }
-
-  @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;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/189f2b3f/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene53/Lucene53RWNormsFormat.java
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene53/Lucene53RWNormsFormat.java b/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene53/Lucene53RWNormsFormat.java
deleted file mode 100644
index 86a2b6a..0000000
--- a/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene53/Lucene53RWNormsFormat.java
+++ /dev/null
@@ -1,31 +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.lucene53;
-
-import java.io.IOException;
-
-import org.apache.lucene.codecs.NormsConsumer;
-import org.apache.lucene.index.SegmentWriteState;
-
-public class Lucene53RWNormsFormat extends Lucene53NormsFormat {
-
-  @Override
-  public NormsConsumer normsConsumer(SegmentWriteState state) throws IOException {
-    return new Lucene53NormsConsumer(state, DATA_CODEC, DATA_EXTENSION, METADATA_CODEC, METADATA_EXTENSION);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/189f2b3f/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene53/TestLucene53NormsFormat.java
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene53/TestLucene53NormsFormat.java b/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene53/TestLucene53NormsFormat.java
deleted file mode 100644
index 7d37b45..0000000
--- a/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene53/TestLucene53NormsFormat.java
+++ /dev/null
@@ -1,44 +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.lucene53;
-
-import org.apache.lucene.codecs.Codec;
-import org.apache.lucene.codecs.lucene62.Lucene62RWCodec;
-import org.apache.lucene.index.BaseNormsFormatTestCase;
-import org.apache.lucene.util.Version;
-
-/**
- * Tests Lucene53NormsFormat
- */
-public class TestLucene53NormsFormat extends BaseNormsFormatTestCase {
-  private final Codec codec = new Lucene62RWCodec();
-
-  @Override
-  protected int getCreatedVersionMajor() {
-    return Version.LUCENE_6_2_0.major;
-  }
-
-  @Override
-  protected Codec getCodec() {
-    return codec;
-  }
-
-  @Override
-  protected boolean codecSupportsSparsity() {
-    return false;
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/189f2b3f/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene54/TestLucene54DocValuesFormat.java
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene54/TestLucene54DocValuesFormat.java b/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene54/TestLucene54DocValuesFormat.java
deleted file mode 100644
index a761dfc..0000000
--- a/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene54/TestLucene54DocValuesFormat.java
+++ /dev/null
@@ -1,640 +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.lucene54;
-
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-import java.util.TreeSet;
-
-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.codecs.lucene54.Lucene54DocValuesProducer.SparseNumericDocValues;
-import org.apache.lucene.codecs.lucene54.Lucene54DocValuesProducer.SparseNumericDocValuesRandomAccessWrapper;
-import org.apache.lucene.document.BinaryDocValuesField;
-import org.apache.lucene.document.Document;
-import org.apache.lucene.document.Field;
-import org.apache.lucene.document.NumericDocValuesField;
-import org.apache.lucene.document.SortedDocValuesField;
-import org.apache.lucene.document.SortedNumericDocValuesField;
-import org.apache.lucene.document.SortedSetDocValuesField;
-import org.apache.lucene.document.StoredField;
-import org.apache.lucene.document.StringField;
-import org.apache.lucene.index.BaseCompressingDocValuesFormatTestCase;
-import org.apache.lucene.index.BinaryDocValues;
-import org.apache.lucene.index.DirectoryReader;
-import org.apache.lucene.index.DocValues;
-import org.apache.lucene.index.IndexReader;
-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.LeafReaderContext;
-import org.apache.lucene.index.NumericDocValues;
-import org.apache.lucene.index.RandomIndexWriter;
-import org.apache.lucene.index.SerialMergeScheduler;
-import org.apache.lucene.index.SortedDocValues;
-import org.apache.lucene.index.SortedNumericDocValues;
-import org.apache.lucene.index.SortedSetDocValues;
-import org.apache.lucene.index.Term;
-import org.apache.lucene.index.Terms;
-import org.apache.lucene.index.TermsEnum.SeekStatus;
-import org.apache.lucene.search.DocIdSetIterator;
-import org.apache.lucene.index.TermsEnum;
-import org.apache.lucene.store.Directory;
-import org.apache.lucene.store.RAMFile;
-import org.apache.lucene.store.RAMInputStream;
-import org.apache.lucene.store.RAMOutputStream;
-import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.BytesRefBuilder;
-import org.apache.lucene.util.LongValues;
-import org.apache.lucene.util.TestUtil;
-
-/**
- * Tests Lucene54DocValuesFormat
- */
-public class TestLucene54DocValuesFormat extends BaseCompressingDocValuesFormatTestCase {
-  private final Codec codec = TestUtil.alwaysDocValuesFormat(new Lucene54DocValuesFormat());
-
-  @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), 1d, 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), 1d, 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);
-    }
-  }
-
-  @Slow
-  public void testSparseDocValuesVsStoredFields() throws Exception {
-    int numIterations = atLeast(1);
-    for (int i = 0; i < numIterations; i++) {
-      doTestSparseDocValuesVsStoredFields();
-    }
-  }
-
-  private void doTestSparseDocValuesVsStoredFields() throws Exception {
-    final long[] values = new long[TestUtil.nextInt(random(), 1, 500)];
-    for (int i = 0; i < values.length; ++i) {
-      values[i] = random().nextLong();
-    }
-
-    Directory dir = newFSDirectory(createTempDir());
-    IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random()));
-    conf.setMergeScheduler(new SerialMergeScheduler());
-    RandomIndexWriter writer = new RandomIndexWriter(random(), dir, conf);
-
-    // sparse compression is only enabled if less than 1% of docs have a value
-    final int avgGap = 100;
-
-    final int numDocs = atLeast(200);
-    for (int i = random().nextInt(avgGap * 2); i >= 0; --i) {
-      writer.addDocument(new Document());
-    }
-    final int maxNumValuesPerDoc = random().nextBoolean() ? 1 : TestUtil.nextInt(random(), 2, 5);
-    for (int i = 0; i < numDocs; ++i) {
-      Document doc = new Document();
-
-      // single-valued
-      long docValue = values[random().nextInt(values.length)];
-      doc.add(new NumericDocValuesField("numeric", docValue));
-      doc.add(new SortedDocValuesField("sorted", new BytesRef(Long.toString(docValue))));
-      doc.add(new BinaryDocValuesField("binary", new BytesRef(Long.toString(docValue))));
-      doc.add(new StoredField("value", docValue));
-
-      // multi-valued
-      final int numValues = TestUtil.nextInt(random(), 1, maxNumValuesPerDoc);
-      for (int j = 0; j < numValues; ++j) {
-        docValue = values[random().nextInt(values.length)];
-        doc.add(new SortedNumericDocValuesField("sorted_numeric", docValue));
-        doc.add(new SortedSetDocValuesField("sorted_set", new BytesRef(Long.toString(docValue))));
-        doc.add(new StoredField("values", docValue));
-      }
-
-      writer.addDocument(doc);
-
-      // add a gap
-      for (int j = TestUtil.nextInt(random(), 0, avgGap * 2); j >= 0; --j) {
-        writer.addDocument(new Document());
-      }
-    }
-
-    if (random().nextBoolean()) {
-      writer.forceMerge(1);
-    }
-
-    final IndexReader indexReader = writer.getReader();
-    TestUtil.checkReader(indexReader);
-    writer.close();
-
-    for (LeafReaderContext context : indexReader.leaves()) {
-      final LeafReader reader = context.reader();
-      final NumericDocValues numeric = DocValues.getNumeric(reader, "numeric");
-
-      final SortedDocValues sorted = DocValues.getSorted(reader, "sorted");
-
-      final BinaryDocValues binary = DocValues.getBinary(reader, "binary");
-
-      final SortedNumericDocValues sortedNumeric = DocValues.getSortedNumeric(reader, "sorted_numeric");
-
-      final SortedSetDocValues sortedSet = DocValues.getSortedSet(reader, "sorted_set");
-
-      for (int i = 0; i < reader.maxDoc(); ++i) {
-        final Document doc = reader.document(i);
-        final IndexableField valueField = doc.getField("value");
-        final Long value = valueField == null ? null : valueField.numericValue().longValue();
-
-        if (value == null) {
-          assertTrue(numeric.docID() + " vs " + i, numeric.docID() < i);
-        } else {
-          assertEquals(i, numeric.nextDoc());
-          assertEquals(i, binary.nextDoc());
-          assertEquals(i, sorted.nextDoc());
-          assertEquals(value.longValue(), numeric.longValue());
-          assertTrue(sorted.ordValue() >= 0);
-          assertEquals(new BytesRef(Long.toString(value)), sorted.lookupOrd(sorted.ordValue()));
-          assertEquals(new BytesRef(Long.toString(value)), binary.binaryValue());
-        }
-
-        final IndexableField[] valuesFields = doc.getFields("values");
-        if (valuesFields.length == 0) {
-          assertTrue(sortedNumeric.docID() + " vs " + i, sortedNumeric.docID() < i);
-        } else {
-          final Set<Long> valueSet = new HashSet<>();
-          for (IndexableField sf : valuesFields) {
-            valueSet.add(sf.numericValue().longValue());
-          }
-
-          assertEquals(i, sortedNumeric.nextDoc());
-          assertEquals(valuesFields.length, sortedNumeric.docValueCount());
-          for (int j = 0; j < sortedNumeric.docValueCount(); ++j) {
-            assertTrue(valueSet.contains(sortedNumeric.nextValue()));
-          }
-          assertEquals(i, sortedSet.nextDoc());
-          int sortedSetCount = 0;
-          while (true) {
-            long ord = sortedSet.nextOrd();
-            if (ord == SortedSetDocValues.NO_MORE_ORDS) {
-              break;
-            }
-            assertTrue(valueSet.contains(Long.parseLong(sortedSet.lookupOrd(ord).utf8ToString())));
-            sortedSetCount++;
-          }
-          assertEquals(valueSet.size(), sortedSetCount);
-        }
-      }
-    }
-
-    indexReader.close();
-    dir.close();
-  }
-
-  // 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 Lucene54DocValuesFormat();
-    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 = getOnlyLeafReader(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());
-      }
-    }
-  }
-
-  public void testSparseLongValues() throws IOException {
-    final int iters = atLeast(5);
-    for (int iter = 0; iter < iters; ++iter) {
-      final int numDocs = TestUtil.nextInt(random(), 0, 100);
-      final int[] docIds = new int[numDocs];
-      final long[] values = new long[numDocs];
-      final int maxDoc;
-      if (numDocs == 0) {
-        maxDoc = 1 + random().nextInt(10);
-      } else {
-        docIds[0] = random().nextInt(10);
-        for (int i = 1; i < docIds.length; ++i) {
-          docIds[i] = docIds[i - 1] + 1 + random().nextInt(100);
-        }
-        maxDoc = docIds[numDocs - 1] + 1 + random().nextInt(10);
-      }
-      for (int i = 0; i < values.length; ++i) {
-        values[i] = random().nextLong();
-      }
-      final long missingValue = random().nextLong();
-      final LongValues docIdsValues = new LongValues() {
-        @Override
-        public long get(long index) {
-          return docIds[Math.toIntExact(index)];
-        }
-      };
-      final LongValues valuesValues = new LongValues() {
-        @Override
-        public long get(long index) {
-          return values[Math.toIntExact(index)];
-        }
-      };
-      final SparseNumericDocValues sparseValues = new SparseNumericDocValues(numDocs, docIdsValues, valuesValues);
-
-      // sequential access
-      assertEquals(-1, sparseValues.docID());
-      for (int i = 0; i < docIds.length; ++i) {
-        assertEquals(docIds[i], sparseValues.nextDoc());
-      }
-      assertEquals(DocIdSetIterator.NO_MORE_DOCS, sparseValues.nextDoc());
-
-      // advance
-      for (int i = 0; i < 2000; ++i) {
-        final int target = TestUtil.nextInt(random(), 0, maxDoc);
-        int index = Arrays.binarySearch(docIds, target);
-        if (index < 0) {
-          index = -1 - index;
-        }
-        sparseValues.reset();
-        if (index > 0) {
-          assertEquals(docIds[index - 1], sparseValues.advance(Math.toIntExact(docIds[index - 1])));
-        }
-        if (index == docIds.length) {
-          assertEquals(DocIdSetIterator.NO_MORE_DOCS, sparseValues.advance(target));
-        } else {
-          assertEquals(docIds[index], sparseValues.advance(target));
-        }
-      }
-
-      // advanceExact
-      for (int i = 0; i < 2000; ++i) {
-        sparseValues.reset();
-        if (random().nextBoolean() && docIds.length > 0) {
-          sparseValues.advance(docIds[TestUtil.nextInt(random(), 0, docIds.length - 1)]);
-        }
-
-        final int target = TestUtil.nextInt(random(), Math.max(0, sparseValues.docID()), maxDoc - 1);
-        final boolean exists = sparseValues.advanceExact(target);
-        
-        final int index = Arrays.binarySearch(docIds, target);
-        assertEquals(index >= 0, exists);
-        assertEquals(target, sparseValues.docID());
-
-        final boolean exists2 = sparseValues.advanceExact(target);
-        assertEquals(index >= 0, exists2);
-        assertEquals(target, sparseValues.docID());
-
-        final int nextIndex = index >= 0 ? index + 1 : -1 - index;
-        if (nextIndex >= docIds.length) {
-          assertEquals(DocIdSetIterator.NO_MORE_DOCS, sparseValues.nextDoc());
-        } else {
-          assertEquals(docIds[nextIndex], sparseValues.nextDoc());
-        }
-      }
-      
-
-      final SparseNumericDocValuesRandomAccessWrapper raWrapper = new SparseNumericDocValuesRandomAccessWrapper(sparseValues, missingValue);
-
-      // random-access
-      for (int i = 0; i < 2000; ++i) {
-        final int docId = TestUtil.nextInt(random(), 0, maxDoc - 1);
-        final int idx = Arrays.binarySearch(docIds, docId);
-        final long value = raWrapper.get(docId);
-        if (idx >= 0) {
-          assertEquals(values[idx], value);
-        } else {
-          assertEquals(missingValue, value);
-        }
-      }
-
-      // sequential access
-      for (int docId = 0; docId < maxDoc; docId += random().nextInt(3)) {
-        final int idx = Arrays.binarySearch(docIds, docId);
-        final long value = raWrapper.get(docId);
-        if (idx >= 0) {
-          assertEquals(values[idx], value);
-        } else {
-          assertEquals(missingValue, value);
-        }
-      }
-    }
-  }
-
-  @Slow
-  public void testSortedSetAroundBlockSize() throws IOException {
-    final int frontier = 1 << Lucene54DocValuesFormat.DIRECT_MONOTONIC_BLOCK_SHIFT;
-    for (int maxDoc = frontier - 1; maxDoc <= frontier + 1; ++maxDoc) {
-      final Directory dir = newDirectory();
-      IndexWriter w = new IndexWriter(dir, newIndexWriterConfig().setMergePolicy(newLogMergePolicy()));
-      RAMFile buffer = new RAMFile();
-      RAMOutputStream out = new RAMOutputStream(buffer, false);
-      Document doc = new Document();
-      SortedSetDocValuesField field1 = new SortedSetDocValuesField("sset", new BytesRef());
-      doc.add(field1);
-      SortedSetDocValuesField field2 = new SortedSetDocValuesField("sset", new BytesRef());
-      doc.add(field2);
-      for (int i = 0; i < maxDoc; ++i) {
-        BytesRef s1 = new BytesRef(TestUtil.randomSimpleString(random(), 2));
-        BytesRef s2 = new BytesRef(TestUtil.randomSimpleString(random(), 2));
-        field1.setBytesValue(s1);
-        field2.setBytesValue(s2);
-        w.addDocument(doc);
-        Set<BytesRef> set = new TreeSet<>(Arrays.asList(s1, s2));
-        out.writeVInt(set.size());
-        for (BytesRef ref : set) {
-          out.writeVInt(ref.length);
-          out.writeBytes(ref.bytes, ref.offset, ref.length);
-        }
-      }
-      out.close();
-      w.forceMerge(1);
-      DirectoryReader r = DirectoryReader.open(w);
-      w.close();
-      LeafReader sr = getOnlyLeafReader(r);
-      assertEquals(maxDoc, sr.maxDoc());
-      SortedSetDocValues values = sr.getSortedSetDocValues("sset");
-      assertNotNull(values);
-      RAMInputStream in = new RAMInputStream("", buffer);
-      BytesRefBuilder b = new BytesRefBuilder();
-      for (int i = 0; i < maxDoc; ++i) {
-        assertEquals(i, values.nextDoc());
-        final int numValues = in.readVInt();
-
-        for (int j = 0; j < numValues; ++j) {
-          b.setLength(in.readVInt());
-          b.grow(b.length());
-          in.readBytes(b.bytes(), 0, b.length());
-          assertEquals(b.get(), values.lookupOrd(values.nextOrd()));
-        }
-
-        assertEquals(SortedSetDocValues.NO_MORE_ORDS, values.nextOrd());
-      }
-      r.close();
-      dir.close();
-    }
-  }
-
-  @Slow
-  public void testSortedNumericAroundBlockSize() throws IOException {
-    final int frontier = 1 << Lucene54DocValuesFormat.DIRECT_MONOTONIC_BLOCK_SHIFT;
-    for (int maxDoc = frontier - 1; maxDoc <= frontier + 1; ++maxDoc) {
-      final Directory dir = newDirectory();
-      IndexWriter w = new IndexWriter(dir, newIndexWriterConfig().setMergePolicy(newLogMergePolicy()));
-      RAMFile buffer = new RAMFile();
-      RAMOutputStream out = new RAMOutputStream(buffer, false);
-      Document doc = new Document();
-      SortedNumericDocValuesField field1 = new SortedNumericDocValuesField("snum", 0L);
-      doc.add(field1);
-      SortedNumericDocValuesField field2 = new SortedNumericDocValuesField("snum", 0L);
-      doc.add(field2);
-      for (int i = 0; i < maxDoc; ++i) {
-        long s1 = random().nextInt(100);
-        long s2 = random().nextInt(100);
-        field1.setLongValue(s1);
-        field2.setLongValue(s2);
-        w.addDocument(doc);
-        out.writeVLong(Math.min(s1, s2));
-        out.writeVLong(Math.max(s1, s2));
-      }
-      out.close();
-      w.forceMerge(1);
-      DirectoryReader r = DirectoryReader.open(w);
-      w.close();
-      LeafReader sr = getOnlyLeafReader(r);
-      assertEquals(maxDoc, sr.maxDoc());
-      SortedNumericDocValues values = sr.getSortedNumericDocValues("snum");
-      assertNotNull(values);
-      RAMInputStream in = new RAMInputStream("", buffer);
-      for (int i = 0; i < maxDoc; ++i) {
-        assertEquals(i, values.nextDoc());
-        assertEquals(2, values.docValueCount());
-        assertEquals(in.readVLong(), values.nextValue());
-        assertEquals(in.readVLong(), values.nextValue());
-      }
-      r.close();
-      dir.close();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/189f2b3f/lucene/core/src/java/org/apache/lucene/index/IndexFormatTooOldException.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/IndexFormatTooOldException.java b/lucene/core/src/java/org/apache/lucene/index/IndexFormatTooOldException.java
index b7c02f5..1fd7953 100644
--- a/lucene/core/src/java/org/apache/lucene/index/IndexFormatTooOldException.java
+++ b/lucene/core/src/java/org/apache/lucene/index/IndexFormatTooOldException.java
@@ -42,7 +42,7 @@ public class IndexFormatTooOldException extends IOException {
    * @lucene.internal */
   public IndexFormatTooOldException(String resourceDescription, String reason) {
     super("Format version is not supported (resource " + resourceDescription + "): " +
-        reason + ". This version of Lucene only supports indexes created with release 6.0 and later.");
+        reason + ". This version of Lucene only supports indexes created with release 7.0 and later.");
     this.resourceDescription = resourceDescription;
     this.reason = reason;
     this.version = null;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/189f2b3f/lucene/core/src/java/org/apache/lucene/util/Version.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/util/Version.java b/lucene/core/src/java/org/apache/lucene/util/Version.java
index 0b10d95..634caee 100644
--- a/lucene/core/src/java/org/apache/lucene/util/Version.java
+++ b/lucene/core/src/java/org/apache/lucene/util/Version.java
@@ -39,62 +39,6 @@ public final class Version {
   public static final Version LUCENE_6_0_0 = new Version(6, 0, 0);
 
   /**
-   * Match settings and bugs in Lucene's 6.0.1 release.
-   * @deprecated Use latest
-   */
-  @Deprecated
-  public static final Version LUCENE_6_0_1 = new Version(6, 0, 1);
-
-  /**
-   * Match settings and bugs in Lucene's 6.1.0 release.
-   * @deprecated Use latest
-   */
-  @Deprecated
-  public static final Version LUCENE_6_1_0 = new Version(6, 1, 0);
-
-  /**
-   * Match settings and bugs in Lucene's 6.2.0 release.
-   * @deprecated Use latest
-   */
-  @Deprecated
-  public static final Version LUCENE_6_2_0 = new Version(6, 2, 0);
-
-  /**
-   * Match settings and bugs in Lucene's 6.2.1 release.
-   * @deprecated Use latest
-   */
-  @Deprecated
-  public static final Version LUCENE_6_2_1 = new Version(6, 2, 1);
-
-  /**
-   * Match settings and bugs in Lucene's 6.3.0 release.
-   * @deprecated Use latest
-   */
-  @Deprecated
-  public static final Version LUCENE_6_3_0 = new Version(6, 3, 0);
-
-  /**
-   * Match settings and bugs in Lucene's 6.4.0 release.
-   * @deprecated Use latest
-   */
-  @Deprecated
-  public static final Version LUCENE_6_4_0 = new Version(6, 4, 0);
-
-  /**
-   * Match settings and bugs in Lucene's 6.4.1 release.
-   * @deprecated Use latest
-   */
-  @Deprecated
-  public static final Version LUCENE_6_4_1 = new Version(6, 4, 1);
-
-  /**
-   * Match settings and bugs in Lucene's 6.4.2 release.
-   * @deprecated Use latest
-   */
-  @Deprecated
-  public static final Version LUCENE_6_4_2 = new Version(6, 4, 2);
-
-  /**
    * Match settings and bugs in Lucene's 6.5.0 release.
    * @deprecated Use latest
    */
@@ -102,27 +46,6 @@ public final class Version {
   public static final Version LUCENE_6_5_0 = new Version(6, 5, 0);
 
   /**
-   * Match settings and bugs in Lucene's 6.5.1 release.
-   * @deprecated Use latest
-   */
-  @Deprecated
-  public static final Version LUCENE_6_5_1 = new Version(6, 5, 1);
-
-  /**
-   * Match settings and bugs in Lucene's 6.6.0 release.
-   * @deprecated Use latest
-   */
-  @Deprecated
-  public static final Version LUCENE_6_6_0 = new Version(6, 6, 0);
-
-  /**
-   * Match settings and bugs in Lucene's 6.7.0 release.
-   * @deprecated Use latest
-   */
-  @Deprecated
-  public static final Version LUCENE_6_7_0 = new Version(6, 7, 0);
-
-  /**
    * Match settings and bugs in Lucene's 7.0.0 release.
    * @deprecated (8.0.0) Use latest
    */

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/189f2b3f/lucene/core/src/test/org/apache/lucene/util/TestVersion.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/util/TestVersion.java b/lucene/core/src/test/org/apache/lucene/util/TestVersion.java
index 673aaae..cc30a5d 100644
--- a/lucene/core/src/test/org/apache/lucene/util/TestVersion.java
+++ b/lucene/core/src/test/org/apache/lucene/util/TestVersion.java
@@ -32,25 +32,26 @@ public class TestVersion extends LuceneTestCase {
         assertTrue("LATEST must be always onOrAfter("+v+")", Version.LATEST.onOrAfter(v));
       }
     }
-    assertTrue(Version.LUCENE_7_0_0.onOrAfter(Version.LUCENE_6_0_0));;
+    assertTrue(Version.LUCENE_8_0_0.onOrAfter(Version.LUCENE_7_0_0));;
   }
 
   public void testToString() {
-    assertEquals("6.0.0", Version.LUCENE_6_0_0.toString());
     assertEquals("7.0.0", Version.LUCENE_7_0_0.toString());
+    assertEquals("8.0.0", Version.LUCENE_8_0_0.toString());
   }
 
   public void testParseLeniently() throws Exception {
-    assertEquals(Version.LUCENE_6_0_0, Version.parseLeniently("6.0"));
-    assertEquals(Version.LUCENE_6_0_0, Version.parseLeniently("6.0.0"));
-    assertEquals(Version.LUCENE_6_0_0, Version.parseLeniently("LUCENE_60"));
-    assertEquals(Version.LUCENE_6_0_0, Version.parseLeniently("LUCENE_6_0"));
-    assertEquals(Version.LUCENE_6_0_0, Version.parseLeniently("LUCENE_6_0_0"));
     assertEquals(Version.LUCENE_7_0_0, Version.parseLeniently("7.0"));
     assertEquals(Version.LUCENE_7_0_0, Version.parseLeniently("7.0.0"));
     assertEquals(Version.LUCENE_7_0_0, Version.parseLeniently("LUCENE_70"));
     assertEquals(Version.LUCENE_7_0_0, Version.parseLeniently("LUCENE_7_0"));
     assertEquals(Version.LUCENE_7_0_0, Version.parseLeniently("LUCENE_7_0_0"));
+    assertEquals(Version.LUCENE_8_0_0, Version.parseLeniently("8.0"));
+    assertEquals(Version.LUCENE_8_0_0, Version.parseLeniently("8.0.0"));
+    assertEquals(Version.LUCENE_8_0_0, Version.parseLeniently("LUCENE_80"));
+    assertEquals(Version.LUCENE_8_0_0, Version.parseLeniently("LUCENE_8_0"));
+    assertEquals(Version.LUCENE_8_0_0, Version.parseLeniently("LUCENE_8_0_0"));
+
     assertEquals(Version.LATEST, Version.parseLeniently("LATEST"));
     assertEquals(Version.LATEST, Version.parseLeniently("latest"));
     assertEquals(Version.LATEST, Version.parseLeniently("LUCENE_CURRENT"));
@@ -74,9 +75,9 @@ public class TestVersion extends LuceneTestCase {
     assertTrue(expected.getMessage().contains("LUCENE61"));
 
     expected = expectThrows(ParseException.class, () -> {
-      Version.parseLeniently("LUCENE_6.0.0");
+      Version.parseLeniently("LUCENE_7.0.0");
     });
-    assertTrue(expected.getMessage().contains("LUCENE_6.0.0"));
+    assertTrue(expected.getMessage().contains("LUCENE_7.0.0"));
   }
 
   public void testParseLenientlyOnAllConstants() throws Exception {
@@ -94,8 +95,8 @@ public class TestVersion extends LuceneTestCase {
   }
 
   public void testParse() throws Exception {
-    assertEquals(Version.LUCENE_6_0_0, Version.parse("6.0.0"));
     assertEquals(Version.LUCENE_7_0_0, Version.parse("7.0.0"));
+    assertEquals(Version.LUCENE_8_0_0, Version.parse("8.0.0"));
     
     // Version does not pass judgement on the major version:
     assertEquals(1, Version.parse("1.0").major);
@@ -103,69 +104,69 @@ public class TestVersion extends LuceneTestCase {
   }
 
   public void testForwardsCompatibility() throws Exception {
-    assertTrue(Version.parse("6.10.20").onOrAfter(Version.LUCENE_6_0_0));
+    assertTrue(Version.parse("7.10.20").onOrAfter(Version.LUCENE_7_0_0));
   }
 
   public void testParseExceptions() {
     ParseException expected = expectThrows(ParseException.class, () -> {
-      Version.parse("LUCENE_6_0_0");
+      Version.parse("LUCENE_7_0_0");
     });
-    assertTrue(expected.getMessage().contains("LUCENE_6_0_0"));
+    assertTrue(expected.getMessage().contains("LUCENE_7_0_0"));
 
     expected = expectThrows(ParseException.class, () -> {
-      Version.parse("6.256");
+      Version.parse("7.256");
     });
-    assertTrue(expected.getMessage().contains("6.256"));
+    assertTrue(expected.getMessage().contains("7.256"));
 
     expected = expectThrows(ParseException.class, () -> {
-      Version.parse("6.-1");
+      Version.parse("7.-1");
     });
-    assertTrue(expected.getMessage().contains("6.-1"));
+    assertTrue(expected.getMessage().contains("7.-1"));
 
     expected = expectThrows(ParseException.class, () -> {
-      Version.parse("6.1.256");
+      Version.parse("7.1.256");
     });
-    assertTrue(expected.getMessage().contains("6.1.256"));
+    assertTrue(expected.getMessage().contains("7.1.256"));
 
     expected = expectThrows(ParseException.class, () -> {
-      Version.parse("6.1.-1");
+      Version.parse("7.1.-1");
     });
-    assertTrue(expected.getMessage().contains("6.1.-1"));
+    assertTrue(expected.getMessage().contains("7.1.-1"));
 
     expected = expectThrows(ParseException.class, () -> {
-      Version.parse("6.1.1.3");
+      Version.parse("7.1.1.3");
     });
-    assertTrue(expected.getMessage().contains("6.1.1.3"));
+    assertTrue(expected.getMessage().contains("7.1.1.3"));
 
     expected = expectThrows(ParseException.class, () -> {
-      Version.parse("6.1.1.-1");
+      Version.parse("7.1.1.-1");
     });
-    assertTrue(expected.getMessage().contains("6.1.1.-1"));
+    assertTrue(expected.getMessage().contains("7.1.1.-1"));
 
     expected = expectThrows(ParseException.class, () -> {
-      Version.parse("6.1.1.1");
+      Version.parse("7.1.1.1");
     });
-    assertTrue(expected.getMessage().contains("6.1.1.1"));
+    assertTrue(expected.getMessage().contains("7.1.1.1"));
 
     expected = expectThrows(ParseException.class, () -> {
-      Version.parse("6.1.1.2");
+      Version.parse("7.1.1.2");
     });
-    assertTrue(expected.getMessage().contains("6.1.1.2"));
+    assertTrue(expected.getMessage().contains("7.1.1.2"));
 
     expected = expectThrows(ParseException.class, () -> {
-      Version.parse("6.0.0.0");
+      Version.parse("7.0.0.0");
     });
-    assertTrue(expected.getMessage().contains("6.0.0.0"));
+    assertTrue(expected.getMessage().contains("7.0.0.0"));
 
     expected = expectThrows(ParseException.class, () -> {
-      Version.parse("6.0.0.1.42");
+      Version.parse("7.0.0.1.42");
     });
-    assertTrue(expected.getMessage().contains("6.0.0.1.42"));
+    assertTrue(expected.getMessage().contains("7.0.0.1.42"));
 
     expected = expectThrows(ParseException.class, () -> {
-      Version.parse("6..0.1");
+      Version.parse("7..0.1");
     });
-    assertTrue(expected.getMessage().contains("6..0.1"));
+    assertTrue(expected.getMessage().contains("7..0.1"));
   }
   
   public void testDeprecations() throws Exception {


[03/23] lucene-solr:jira/solr-10879: Move backcompat oldIndexes to unsupportedIndexes in TestBackwardsCompatibilty, and rename the index files to unsupported.

Posted by ab...@apache.org.
Move backcompat oldIndexes to unsupportedIndexes in TestBackwardsCompatibilty, and rename the index files to unsupported.


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

Branch: refs/heads/jira/solr-10879
Commit: 774e3d8f29e658da64e85bde07ecfb5fc30b2173
Parents: 3b6d741
Author: Anshum Gupta <an...@apple.com>
Authored: Fri Jun 30 10:45:50 2017 -0700
Committer: Anshum Gupta <an...@apple.com>
Committed: Fri Jun 30 10:45:50 2017 -0700

----------------------------------------------------------------------
 .../index/TestBackwardsCompatibility.java       |  52 +++++++++----------
 .../lucene/index/TestIndexWriterOnOldIndex.java |   2 +-
 .../org/apache/lucene/index/index.6.0.0-cfs.zip | Bin 15807 -> 0 bytes
 .../apache/lucene/index/index.6.0.0-nocfs.zip   | Bin 15806 -> 0 bytes
 .../org/apache/lucene/index/index.6.0.1-cfs.zip | Bin 15820 -> 0 bytes
 .../apache/lucene/index/index.6.0.1-nocfs.zip   | Bin 15823 -> 0 bytes
 .../org/apache/lucene/index/index.6.1.0-cfs.zip | Bin 15803 -> 0 bytes
 .../apache/lucene/index/index.6.1.0-nocfs.zip   | Bin 15829 -> 0 bytes
 .../org/apache/lucene/index/index.6.2.0-cfs.zip | Bin 15880 -> 0 bytes
 .../apache/lucene/index/index.6.2.0-nocfs.zip   | Bin 15867 -> 0 bytes
 .../org/apache/lucene/index/index.6.2.1-cfs.zip | Bin 15851 -> 0 bytes
 .../apache/lucene/index/index.6.2.1-nocfs.zip   | Bin 15845 -> 0 bytes
 .../org/apache/lucene/index/index.6.3.0-cfs.zip | Bin 15869 -> 0 bytes
 .../apache/lucene/index/index.6.3.0-nocfs.zip   | Bin 15876 -> 0 bytes
 .../org/apache/lucene/index/index.6.4.0-cfs.zip | Bin 15829 -> 0 bytes
 .../apache/lucene/index/index.6.4.0-nocfs.zip   | Bin 15831 -> 0 bytes
 .../org/apache/lucene/index/index.6.4.1-cfs.zip | Bin 15848 -> 0 bytes
 .../apache/lucene/index/index.6.4.1-nocfs.zip   | Bin 15838 -> 0 bytes
 .../org/apache/lucene/index/index.6.4.2-cfs.zip | Bin 15856 -> 0 bytes
 .../apache/lucene/index/index.6.4.2-nocfs.zip   | Bin 15886 -> 0 bytes
 .../org/apache/lucene/index/index.6.5.0-cfs.zip | Bin 15832 -> 0 bytes
 .../apache/lucene/index/index.6.5.0-nocfs.zip   | Bin 15836 -> 0 bytes
 .../org/apache/lucene/index/index.6.5.1-cfs.zip | Bin 15842 -> 0 bytes
 .../apache/lucene/index/index.6.5.1-nocfs.zip   | Bin 15827 -> 0 bytes
 .../org/apache/lucene/index/index.6.6.0-cfs.zip | Bin 15875 -> 0 bytes
 .../apache/lucene/index/index.6.6.0-nocfs.zip   | Bin 15883 -> 0 bytes
 .../lucene/index/index.630.brokenoffsets.zip    | Bin 3203 -> 0 bytes
 .../lucene/index/index.single-empty-doc.630.zip | Bin 1363 -> 0 bytes
 .../lucene/index/unsupported.6.0.0-cfs.zip      | Bin 0 -> 15807 bytes
 .../lucene/index/unsupported.6.0.0-nocfs.zip    | Bin 0 -> 15806 bytes
 .../lucene/index/unsupported.6.0.1-cfs.zip      | Bin 0 -> 15820 bytes
 .../lucene/index/unsupported.6.0.1-nocfs.zip    | Bin 0 -> 15823 bytes
 .../lucene/index/unsupported.6.1.0-cfs.zip      | Bin 0 -> 15803 bytes
 .../lucene/index/unsupported.6.1.0-nocfs.zip    | Bin 0 -> 15829 bytes
 .../lucene/index/unsupported.6.2.0-cfs.zip      | Bin 0 -> 15880 bytes
 .../lucene/index/unsupported.6.2.0-nocfs.zip    | Bin 0 -> 15867 bytes
 .../lucene/index/unsupported.6.2.1-cfs.zip      | Bin 0 -> 15851 bytes
 .../lucene/index/unsupported.6.2.1-nocfs.zip    | Bin 0 -> 15845 bytes
 .../lucene/index/unsupported.6.3.0-cfs.zip      | Bin 0 -> 15869 bytes
 .../lucene/index/unsupported.6.3.0-nocfs.zip    | Bin 0 -> 15876 bytes
 .../lucene/index/unsupported.6.4.0-cfs.zip      | Bin 0 -> 15829 bytes
 .../lucene/index/unsupported.6.4.0-nocfs.zip    | Bin 0 -> 15831 bytes
 .../lucene/index/unsupported.6.4.1-cfs.zip      | Bin 0 -> 15848 bytes
 .../lucene/index/unsupported.6.4.1-nocfs.zip    | Bin 0 -> 15838 bytes
 .../lucene/index/unsupported.6.4.2-cfs.zip      | Bin 0 -> 15856 bytes
 .../lucene/index/unsupported.6.4.2-nocfs.zip    | Bin 0 -> 15886 bytes
 .../lucene/index/unsupported.6.5.0-cfs.zip      | Bin 0 -> 15832 bytes
 .../lucene/index/unsupported.6.5.0-nocfs.zip    | Bin 0 -> 15836 bytes
 .../lucene/index/unsupported.6.5.1-cfs.zip      | Bin 0 -> 15842 bytes
 .../lucene/index/unsupported.6.5.1-nocfs.zip    | Bin 0 -> 15827 bytes
 .../lucene/index/unsupported.6.6.0-cfs.zip      | Bin 0 -> 15875 bytes
 .../lucene/index/unsupported.6.6.0-nocfs.zip    | Bin 0 -> 15883 bytes
 .../index/unsupported.630.brokenoffsets.zip     | Bin 0 -> 3203 bytes
 .../index/unsupported.single-empty-doc.630.zip  | Bin 0 -> 1363 bytes
 54 files changed, 27 insertions(+), 27 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/774e3d8f/lucene/backward-codecs/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java b/lucene/backward-codecs/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java
index 1250a30..ad50382 100644
--- a/lucene/backward-codecs/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java
+++ b/lucene/backward-codecs/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java
@@ -276,30 +276,6 @@ public class TestBackwardsCompatibility extends LuceneTestCase {
   }
 
   final static String[] oldNames = {
-    "6.0.0-cfs",
-    "6.0.0-nocfs",
-    "6.0.1-cfs",
-    "6.0.1-nocfs",
-    "6.1.0-cfs",
-    "6.1.0-nocfs",
-    "6.2.0-cfs",
-    "6.2.0-nocfs",
-    "6.2.1-cfs",
-    "6.2.1-nocfs",
-    "6.3.0-cfs",
-    "6.3.0-nocfs",
-    "6.4.0-cfs",
-    "6.4.0-nocfs",
-    "6.4.1-cfs",
-    "6.4.1-nocfs",
-    "6.4.2-cfs",
-    "6.4.2-nocfs",
-    "6.5.0-cfs",
-    "6.5.0-nocfs",
-    "6.5.1-cfs",
-    "6.5.1-nocfs",
-    "6.6.0-cfs",
-    "6.6.0-nocfs"
   };
   
   final String[] unsupportedNames = {
@@ -432,7 +408,31 @@ public class TestBackwardsCompatibility extends LuceneTestCase {
       "5.5.3-cfs",
       "5.5.3-nocfs",
       "5.5.4-cfs",
-      "5.5.4-nocfs"
+      "5.5.4-nocfs",
+      "6.0.0-cfs",
+      "6.0.0-nocfs",
+      "6.0.1-cfs",
+      "6.0.1-nocfs",
+      "6.1.0-cfs",
+      "6.1.0-nocfs",
+      "6.2.0-cfs",
+      "6.2.0-nocfs",
+      "6.2.1-cfs",
+      "6.2.1-nocfs",
+      "6.3.0-cfs",
+      "6.3.0-nocfs",
+      "6.4.0-cfs",
+      "6.4.0-nocfs",
+      "6.4.1-cfs",
+      "6.4.1-nocfs",
+      "6.4.2-cfs",
+      "6.4.2-nocfs",
+      "6.5.0-cfs",
+      "6.5.0-nocfs",
+      "6.5.1-cfs",
+      "6.5.1-nocfs",
+      "6.6.0-cfs",
+      "6.6.0-nocfs"
   };
 
   // TODO: on 6.0.0 release, gen the single segment indices and add here:
@@ -1495,7 +1495,7 @@ public class TestBackwardsCompatibility extends LuceneTestCase {
     }
     reader.close();
   }
-  
+
   public void testDocValuesUpdates() throws Exception {
     Path oldIndexDir = createTempDir("dvupdates");
     TestUtil.unzip(getDataInputStream(dvUpdatesIndex), oldIndexDir);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/774e3d8f/lucene/backward-codecs/src/test/org/apache/lucene/index/TestIndexWriterOnOldIndex.java
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/index/TestIndexWriterOnOldIndex.java b/lucene/backward-codecs/src/test/org/apache/lucene/index/TestIndexWriterOnOldIndex.java
index c77b926..b48b0fd 100644
--- a/lucene/backward-codecs/src/test/org/apache/lucene/index/TestIndexWriterOnOldIndex.java
+++ b/lucene/backward-codecs/src/test/org/apache/lucene/index/TestIndexWriterOnOldIndex.java
@@ -29,7 +29,7 @@ import org.apache.lucene.util.Version;
 public class TestIndexWriterOnOldIndex extends LuceneTestCase {
 
   public void testOpenModeAndCreatedVersion() throws IOException {
-    InputStream resource = getClass().getResourceAsStream("index.single-empty-doc.630.zip");
+    InputStream resource = getClass().getResourceAsStream("unsupported.index.single-empty-doc.630.zip");
     assertNotNull(resource);
     Path path = createTempDir();
     TestUtil.unzip(resource, path);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/774e3d8f/lucene/backward-codecs/src/test/org/apache/lucene/index/index.6.0.0-cfs.zip
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/index/index.6.0.0-cfs.zip b/lucene/backward-codecs/src/test/org/apache/lucene/index/index.6.0.0-cfs.zip
deleted file mode 100644
index c8622df..0000000
Binary files a/lucene/backward-codecs/src/test/org/apache/lucene/index/index.6.0.0-cfs.zip and /dev/null differ

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/774e3d8f/lucene/backward-codecs/src/test/org/apache/lucene/index/index.6.0.0-nocfs.zip
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/index/index.6.0.0-nocfs.zip b/lucene/backward-codecs/src/test/org/apache/lucene/index/index.6.0.0-nocfs.zip
deleted file mode 100644
index 3c245d1..0000000
Binary files a/lucene/backward-codecs/src/test/org/apache/lucene/index/index.6.0.0-nocfs.zip and /dev/null differ

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/774e3d8f/lucene/backward-codecs/src/test/org/apache/lucene/index/index.6.0.1-cfs.zip
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/index/index.6.0.1-cfs.zip b/lucene/backward-codecs/src/test/org/apache/lucene/index/index.6.0.1-cfs.zip
deleted file mode 100644
index f10f1a8..0000000
Binary files a/lucene/backward-codecs/src/test/org/apache/lucene/index/index.6.0.1-cfs.zip and /dev/null differ

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/774e3d8f/lucene/backward-codecs/src/test/org/apache/lucene/index/index.6.0.1-nocfs.zip
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/index/index.6.0.1-nocfs.zip b/lucene/backward-codecs/src/test/org/apache/lucene/index/index.6.0.1-nocfs.zip
deleted file mode 100644
index d45b7fd..0000000
Binary files a/lucene/backward-codecs/src/test/org/apache/lucene/index/index.6.0.1-nocfs.zip and /dev/null differ

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/774e3d8f/lucene/backward-codecs/src/test/org/apache/lucene/index/index.6.1.0-cfs.zip
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/index/index.6.1.0-cfs.zip b/lucene/backward-codecs/src/test/org/apache/lucene/index/index.6.1.0-cfs.zip
deleted file mode 100644
index ff1b952..0000000
Binary files a/lucene/backward-codecs/src/test/org/apache/lucene/index/index.6.1.0-cfs.zip and /dev/null differ

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/774e3d8f/lucene/backward-codecs/src/test/org/apache/lucene/index/index.6.1.0-nocfs.zip
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/index/index.6.1.0-nocfs.zip b/lucene/backward-codecs/src/test/org/apache/lucene/index/index.6.1.0-nocfs.zip
deleted file mode 100644
index 4e2d6a8..0000000
Binary files a/lucene/backward-codecs/src/test/org/apache/lucene/index/index.6.1.0-nocfs.zip and /dev/null differ

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/774e3d8f/lucene/backward-codecs/src/test/org/apache/lucene/index/index.6.2.0-cfs.zip
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/index/index.6.2.0-cfs.zip b/lucene/backward-codecs/src/test/org/apache/lucene/index/index.6.2.0-cfs.zip
deleted file mode 100644
index 36b6d83..0000000
Binary files a/lucene/backward-codecs/src/test/org/apache/lucene/index/index.6.2.0-cfs.zip and /dev/null differ

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/774e3d8f/lucene/backward-codecs/src/test/org/apache/lucene/index/index.6.2.0-nocfs.zip
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/index/index.6.2.0-nocfs.zip b/lucene/backward-codecs/src/test/org/apache/lucene/index/index.6.2.0-nocfs.zip
deleted file mode 100644
index 95ae26c..0000000
Binary files a/lucene/backward-codecs/src/test/org/apache/lucene/index/index.6.2.0-nocfs.zip and /dev/null differ

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/774e3d8f/lucene/backward-codecs/src/test/org/apache/lucene/index/index.6.2.1-cfs.zip
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/index/index.6.2.1-cfs.zip b/lucene/backward-codecs/src/test/org/apache/lucene/index/index.6.2.1-cfs.zip
deleted file mode 100644
index 36b2e1d..0000000
Binary files a/lucene/backward-codecs/src/test/org/apache/lucene/index/index.6.2.1-cfs.zip and /dev/null differ

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/774e3d8f/lucene/backward-codecs/src/test/org/apache/lucene/index/index.6.2.1-nocfs.zip
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/index/index.6.2.1-nocfs.zip b/lucene/backward-codecs/src/test/org/apache/lucene/index/index.6.2.1-nocfs.zip
deleted file mode 100644
index 411825e..0000000
Binary files a/lucene/backward-codecs/src/test/org/apache/lucene/index/index.6.2.1-nocfs.zip and /dev/null differ

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/774e3d8f/lucene/backward-codecs/src/test/org/apache/lucene/index/index.6.3.0-cfs.zip
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/index/index.6.3.0-cfs.zip b/lucene/backward-codecs/src/test/org/apache/lucene/index/index.6.3.0-cfs.zip
deleted file mode 100644
index 737054d..0000000
Binary files a/lucene/backward-codecs/src/test/org/apache/lucene/index/index.6.3.0-cfs.zip and /dev/null differ

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/774e3d8f/lucene/backward-codecs/src/test/org/apache/lucene/index/index.6.3.0-nocfs.zip
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/index/index.6.3.0-nocfs.zip b/lucene/backward-codecs/src/test/org/apache/lucene/index/index.6.3.0-nocfs.zip
deleted file mode 100644
index 14a82d7..0000000
Binary files a/lucene/backward-codecs/src/test/org/apache/lucene/index/index.6.3.0-nocfs.zip and /dev/null differ

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/774e3d8f/lucene/backward-codecs/src/test/org/apache/lucene/index/index.6.4.0-cfs.zip
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/index/index.6.4.0-cfs.zip b/lucene/backward-codecs/src/test/org/apache/lucene/index/index.6.4.0-cfs.zip
deleted file mode 100644
index e5bad02..0000000
Binary files a/lucene/backward-codecs/src/test/org/apache/lucene/index/index.6.4.0-cfs.zip and /dev/null differ

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/774e3d8f/lucene/backward-codecs/src/test/org/apache/lucene/index/index.6.4.0-nocfs.zip
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/index/index.6.4.0-nocfs.zip b/lucene/backward-codecs/src/test/org/apache/lucene/index/index.6.4.0-nocfs.zip
deleted file mode 100644
index 70fc6ee..0000000
Binary files a/lucene/backward-codecs/src/test/org/apache/lucene/index/index.6.4.0-nocfs.zip and /dev/null differ

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/774e3d8f/lucene/backward-codecs/src/test/org/apache/lucene/index/index.6.4.1-cfs.zip
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/index/index.6.4.1-cfs.zip b/lucene/backward-codecs/src/test/org/apache/lucene/index/index.6.4.1-cfs.zip
deleted file mode 100644
index 477859e..0000000
Binary files a/lucene/backward-codecs/src/test/org/apache/lucene/index/index.6.4.1-cfs.zip and /dev/null differ

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/774e3d8f/lucene/backward-codecs/src/test/org/apache/lucene/index/index.6.4.1-nocfs.zip
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/index/index.6.4.1-nocfs.zip b/lucene/backward-codecs/src/test/org/apache/lucene/index/index.6.4.1-nocfs.zip
deleted file mode 100644
index 7411118..0000000
Binary files a/lucene/backward-codecs/src/test/org/apache/lucene/index/index.6.4.1-nocfs.zip and /dev/null differ

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/774e3d8f/lucene/backward-codecs/src/test/org/apache/lucene/index/index.6.4.2-cfs.zip
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/index/index.6.4.2-cfs.zip b/lucene/backward-codecs/src/test/org/apache/lucene/index/index.6.4.2-cfs.zip
deleted file mode 100644
index eee89f4..0000000
Binary files a/lucene/backward-codecs/src/test/org/apache/lucene/index/index.6.4.2-cfs.zip and /dev/null differ

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/774e3d8f/lucene/backward-codecs/src/test/org/apache/lucene/index/index.6.4.2-nocfs.zip
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/index/index.6.4.2-nocfs.zip b/lucene/backward-codecs/src/test/org/apache/lucene/index/index.6.4.2-nocfs.zip
deleted file mode 100644
index d55a6f6..0000000
Binary files a/lucene/backward-codecs/src/test/org/apache/lucene/index/index.6.4.2-nocfs.zip and /dev/null differ

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/774e3d8f/lucene/backward-codecs/src/test/org/apache/lucene/index/index.6.5.0-cfs.zip
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/index/index.6.5.0-cfs.zip b/lucene/backward-codecs/src/test/org/apache/lucene/index/index.6.5.0-cfs.zip
deleted file mode 100644
index 5bff9b2..0000000
Binary files a/lucene/backward-codecs/src/test/org/apache/lucene/index/index.6.5.0-cfs.zip and /dev/null differ

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/774e3d8f/lucene/backward-codecs/src/test/org/apache/lucene/index/index.6.5.0-nocfs.zip
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/index/index.6.5.0-nocfs.zip b/lucene/backward-codecs/src/test/org/apache/lucene/index/index.6.5.0-nocfs.zip
deleted file mode 100644
index 72e7dc2..0000000
Binary files a/lucene/backward-codecs/src/test/org/apache/lucene/index/index.6.5.0-nocfs.zip and /dev/null differ

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/774e3d8f/lucene/backward-codecs/src/test/org/apache/lucene/index/index.6.5.1-cfs.zip
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/index/index.6.5.1-cfs.zip b/lucene/backward-codecs/src/test/org/apache/lucene/index/index.6.5.1-cfs.zip
deleted file mode 100644
index 21eb747..0000000
Binary files a/lucene/backward-codecs/src/test/org/apache/lucene/index/index.6.5.1-cfs.zip and /dev/null differ

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/774e3d8f/lucene/backward-codecs/src/test/org/apache/lucene/index/index.6.5.1-nocfs.zip
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/index/index.6.5.1-nocfs.zip b/lucene/backward-codecs/src/test/org/apache/lucene/index/index.6.5.1-nocfs.zip
deleted file mode 100644
index 832d85d..0000000
Binary files a/lucene/backward-codecs/src/test/org/apache/lucene/index/index.6.5.1-nocfs.zip and /dev/null differ

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/774e3d8f/lucene/backward-codecs/src/test/org/apache/lucene/index/index.6.6.0-cfs.zip
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/index/index.6.6.0-cfs.zip b/lucene/backward-codecs/src/test/org/apache/lucene/index/index.6.6.0-cfs.zip
deleted file mode 100644
index 36e0b3d..0000000
Binary files a/lucene/backward-codecs/src/test/org/apache/lucene/index/index.6.6.0-cfs.zip and /dev/null differ

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/774e3d8f/lucene/backward-codecs/src/test/org/apache/lucene/index/index.6.6.0-nocfs.zip
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/index/index.6.6.0-nocfs.zip b/lucene/backward-codecs/src/test/org/apache/lucene/index/index.6.6.0-nocfs.zip
deleted file mode 100644
index 90ced65..0000000
Binary files a/lucene/backward-codecs/src/test/org/apache/lucene/index/index.6.6.0-nocfs.zip and /dev/null differ

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/774e3d8f/lucene/backward-codecs/src/test/org/apache/lucene/index/index.630.brokenoffsets.zip
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/index/index.630.brokenoffsets.zip b/lucene/backward-codecs/src/test/org/apache/lucene/index/index.630.brokenoffsets.zip
deleted file mode 100644
index 3cf476a..0000000
Binary files a/lucene/backward-codecs/src/test/org/apache/lucene/index/index.630.brokenoffsets.zip and /dev/null differ

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/774e3d8f/lucene/backward-codecs/src/test/org/apache/lucene/index/index.single-empty-doc.630.zip
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/index/index.single-empty-doc.630.zip b/lucene/backward-codecs/src/test/org/apache/lucene/index/index.single-empty-doc.630.zip
deleted file mode 100644
index 1bf1d08..0000000
Binary files a/lucene/backward-codecs/src/test/org/apache/lucene/index/index.single-empty-doc.630.zip and /dev/null differ

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/774e3d8f/lucene/backward-codecs/src/test/org/apache/lucene/index/unsupported.6.0.0-cfs.zip
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/index/unsupported.6.0.0-cfs.zip b/lucene/backward-codecs/src/test/org/apache/lucene/index/unsupported.6.0.0-cfs.zip
new file mode 100644
index 0000000..c8622df
Binary files /dev/null and b/lucene/backward-codecs/src/test/org/apache/lucene/index/unsupported.6.0.0-cfs.zip differ

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/774e3d8f/lucene/backward-codecs/src/test/org/apache/lucene/index/unsupported.6.0.0-nocfs.zip
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/index/unsupported.6.0.0-nocfs.zip b/lucene/backward-codecs/src/test/org/apache/lucene/index/unsupported.6.0.0-nocfs.zip
new file mode 100644
index 0000000..3c245d1
Binary files /dev/null and b/lucene/backward-codecs/src/test/org/apache/lucene/index/unsupported.6.0.0-nocfs.zip differ

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/774e3d8f/lucene/backward-codecs/src/test/org/apache/lucene/index/unsupported.6.0.1-cfs.zip
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/index/unsupported.6.0.1-cfs.zip b/lucene/backward-codecs/src/test/org/apache/lucene/index/unsupported.6.0.1-cfs.zip
new file mode 100644
index 0000000..f10f1a8
Binary files /dev/null and b/lucene/backward-codecs/src/test/org/apache/lucene/index/unsupported.6.0.1-cfs.zip differ

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/774e3d8f/lucene/backward-codecs/src/test/org/apache/lucene/index/unsupported.6.0.1-nocfs.zip
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/index/unsupported.6.0.1-nocfs.zip b/lucene/backward-codecs/src/test/org/apache/lucene/index/unsupported.6.0.1-nocfs.zip
new file mode 100644
index 0000000..d45b7fd
Binary files /dev/null and b/lucene/backward-codecs/src/test/org/apache/lucene/index/unsupported.6.0.1-nocfs.zip differ

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/774e3d8f/lucene/backward-codecs/src/test/org/apache/lucene/index/unsupported.6.1.0-cfs.zip
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/index/unsupported.6.1.0-cfs.zip b/lucene/backward-codecs/src/test/org/apache/lucene/index/unsupported.6.1.0-cfs.zip
new file mode 100644
index 0000000..ff1b952
Binary files /dev/null and b/lucene/backward-codecs/src/test/org/apache/lucene/index/unsupported.6.1.0-cfs.zip differ

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/774e3d8f/lucene/backward-codecs/src/test/org/apache/lucene/index/unsupported.6.1.0-nocfs.zip
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/index/unsupported.6.1.0-nocfs.zip b/lucene/backward-codecs/src/test/org/apache/lucene/index/unsupported.6.1.0-nocfs.zip
new file mode 100644
index 0000000..4e2d6a8
Binary files /dev/null and b/lucene/backward-codecs/src/test/org/apache/lucene/index/unsupported.6.1.0-nocfs.zip differ

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/774e3d8f/lucene/backward-codecs/src/test/org/apache/lucene/index/unsupported.6.2.0-cfs.zip
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/index/unsupported.6.2.0-cfs.zip b/lucene/backward-codecs/src/test/org/apache/lucene/index/unsupported.6.2.0-cfs.zip
new file mode 100644
index 0000000..36b6d83
Binary files /dev/null and b/lucene/backward-codecs/src/test/org/apache/lucene/index/unsupported.6.2.0-cfs.zip differ

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/774e3d8f/lucene/backward-codecs/src/test/org/apache/lucene/index/unsupported.6.2.0-nocfs.zip
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/index/unsupported.6.2.0-nocfs.zip b/lucene/backward-codecs/src/test/org/apache/lucene/index/unsupported.6.2.0-nocfs.zip
new file mode 100644
index 0000000..95ae26c
Binary files /dev/null and b/lucene/backward-codecs/src/test/org/apache/lucene/index/unsupported.6.2.0-nocfs.zip differ

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/774e3d8f/lucene/backward-codecs/src/test/org/apache/lucene/index/unsupported.6.2.1-cfs.zip
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/index/unsupported.6.2.1-cfs.zip b/lucene/backward-codecs/src/test/org/apache/lucene/index/unsupported.6.2.1-cfs.zip
new file mode 100644
index 0000000..36b2e1d
Binary files /dev/null and b/lucene/backward-codecs/src/test/org/apache/lucene/index/unsupported.6.2.1-cfs.zip differ

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/774e3d8f/lucene/backward-codecs/src/test/org/apache/lucene/index/unsupported.6.2.1-nocfs.zip
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/index/unsupported.6.2.1-nocfs.zip b/lucene/backward-codecs/src/test/org/apache/lucene/index/unsupported.6.2.1-nocfs.zip
new file mode 100644
index 0000000..411825e
Binary files /dev/null and b/lucene/backward-codecs/src/test/org/apache/lucene/index/unsupported.6.2.1-nocfs.zip differ

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/774e3d8f/lucene/backward-codecs/src/test/org/apache/lucene/index/unsupported.6.3.0-cfs.zip
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/index/unsupported.6.3.0-cfs.zip b/lucene/backward-codecs/src/test/org/apache/lucene/index/unsupported.6.3.0-cfs.zip
new file mode 100644
index 0000000..737054d
Binary files /dev/null and b/lucene/backward-codecs/src/test/org/apache/lucene/index/unsupported.6.3.0-cfs.zip differ

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/774e3d8f/lucene/backward-codecs/src/test/org/apache/lucene/index/unsupported.6.3.0-nocfs.zip
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/index/unsupported.6.3.0-nocfs.zip b/lucene/backward-codecs/src/test/org/apache/lucene/index/unsupported.6.3.0-nocfs.zip
new file mode 100644
index 0000000..14a82d7
Binary files /dev/null and b/lucene/backward-codecs/src/test/org/apache/lucene/index/unsupported.6.3.0-nocfs.zip differ

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/774e3d8f/lucene/backward-codecs/src/test/org/apache/lucene/index/unsupported.6.4.0-cfs.zip
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/index/unsupported.6.4.0-cfs.zip b/lucene/backward-codecs/src/test/org/apache/lucene/index/unsupported.6.4.0-cfs.zip
new file mode 100644
index 0000000..e5bad02
Binary files /dev/null and b/lucene/backward-codecs/src/test/org/apache/lucene/index/unsupported.6.4.0-cfs.zip differ

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/774e3d8f/lucene/backward-codecs/src/test/org/apache/lucene/index/unsupported.6.4.0-nocfs.zip
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/index/unsupported.6.4.0-nocfs.zip b/lucene/backward-codecs/src/test/org/apache/lucene/index/unsupported.6.4.0-nocfs.zip
new file mode 100644
index 0000000..70fc6ee
Binary files /dev/null and b/lucene/backward-codecs/src/test/org/apache/lucene/index/unsupported.6.4.0-nocfs.zip differ

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/774e3d8f/lucene/backward-codecs/src/test/org/apache/lucene/index/unsupported.6.4.1-cfs.zip
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/index/unsupported.6.4.1-cfs.zip b/lucene/backward-codecs/src/test/org/apache/lucene/index/unsupported.6.4.1-cfs.zip
new file mode 100644
index 0000000..477859e
Binary files /dev/null and b/lucene/backward-codecs/src/test/org/apache/lucene/index/unsupported.6.4.1-cfs.zip differ

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/774e3d8f/lucene/backward-codecs/src/test/org/apache/lucene/index/unsupported.6.4.1-nocfs.zip
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/index/unsupported.6.4.1-nocfs.zip b/lucene/backward-codecs/src/test/org/apache/lucene/index/unsupported.6.4.1-nocfs.zip
new file mode 100644
index 0000000..7411118
Binary files /dev/null and b/lucene/backward-codecs/src/test/org/apache/lucene/index/unsupported.6.4.1-nocfs.zip differ

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/774e3d8f/lucene/backward-codecs/src/test/org/apache/lucene/index/unsupported.6.4.2-cfs.zip
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/index/unsupported.6.4.2-cfs.zip b/lucene/backward-codecs/src/test/org/apache/lucene/index/unsupported.6.4.2-cfs.zip
new file mode 100644
index 0000000..eee89f4
Binary files /dev/null and b/lucene/backward-codecs/src/test/org/apache/lucene/index/unsupported.6.4.2-cfs.zip differ

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/774e3d8f/lucene/backward-codecs/src/test/org/apache/lucene/index/unsupported.6.4.2-nocfs.zip
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/index/unsupported.6.4.2-nocfs.zip b/lucene/backward-codecs/src/test/org/apache/lucene/index/unsupported.6.4.2-nocfs.zip
new file mode 100644
index 0000000..d55a6f6
Binary files /dev/null and b/lucene/backward-codecs/src/test/org/apache/lucene/index/unsupported.6.4.2-nocfs.zip differ

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/774e3d8f/lucene/backward-codecs/src/test/org/apache/lucene/index/unsupported.6.5.0-cfs.zip
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/index/unsupported.6.5.0-cfs.zip b/lucene/backward-codecs/src/test/org/apache/lucene/index/unsupported.6.5.0-cfs.zip
new file mode 100644
index 0000000..5bff9b2
Binary files /dev/null and b/lucene/backward-codecs/src/test/org/apache/lucene/index/unsupported.6.5.0-cfs.zip differ

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/774e3d8f/lucene/backward-codecs/src/test/org/apache/lucene/index/unsupported.6.5.0-nocfs.zip
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/index/unsupported.6.5.0-nocfs.zip b/lucene/backward-codecs/src/test/org/apache/lucene/index/unsupported.6.5.0-nocfs.zip
new file mode 100644
index 0000000..72e7dc2
Binary files /dev/null and b/lucene/backward-codecs/src/test/org/apache/lucene/index/unsupported.6.5.0-nocfs.zip differ

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/774e3d8f/lucene/backward-codecs/src/test/org/apache/lucene/index/unsupported.6.5.1-cfs.zip
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/index/unsupported.6.5.1-cfs.zip b/lucene/backward-codecs/src/test/org/apache/lucene/index/unsupported.6.5.1-cfs.zip
new file mode 100644
index 0000000..21eb747
Binary files /dev/null and b/lucene/backward-codecs/src/test/org/apache/lucene/index/unsupported.6.5.1-cfs.zip differ

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/774e3d8f/lucene/backward-codecs/src/test/org/apache/lucene/index/unsupported.6.5.1-nocfs.zip
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/index/unsupported.6.5.1-nocfs.zip b/lucene/backward-codecs/src/test/org/apache/lucene/index/unsupported.6.5.1-nocfs.zip
new file mode 100644
index 0000000..832d85d
Binary files /dev/null and b/lucene/backward-codecs/src/test/org/apache/lucene/index/unsupported.6.5.1-nocfs.zip differ

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/774e3d8f/lucene/backward-codecs/src/test/org/apache/lucene/index/unsupported.6.6.0-cfs.zip
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/index/unsupported.6.6.0-cfs.zip b/lucene/backward-codecs/src/test/org/apache/lucene/index/unsupported.6.6.0-cfs.zip
new file mode 100644
index 0000000..36e0b3d
Binary files /dev/null and b/lucene/backward-codecs/src/test/org/apache/lucene/index/unsupported.6.6.0-cfs.zip differ

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/774e3d8f/lucene/backward-codecs/src/test/org/apache/lucene/index/unsupported.6.6.0-nocfs.zip
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/index/unsupported.6.6.0-nocfs.zip b/lucene/backward-codecs/src/test/org/apache/lucene/index/unsupported.6.6.0-nocfs.zip
new file mode 100644
index 0000000..90ced65
Binary files /dev/null and b/lucene/backward-codecs/src/test/org/apache/lucene/index/unsupported.6.6.0-nocfs.zip differ

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/774e3d8f/lucene/backward-codecs/src/test/org/apache/lucene/index/unsupported.630.brokenoffsets.zip
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/index/unsupported.630.brokenoffsets.zip b/lucene/backward-codecs/src/test/org/apache/lucene/index/unsupported.630.brokenoffsets.zip
new file mode 100644
index 0000000..3cf476a
Binary files /dev/null and b/lucene/backward-codecs/src/test/org/apache/lucene/index/unsupported.630.brokenoffsets.zip differ

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/774e3d8f/lucene/backward-codecs/src/test/org/apache/lucene/index/unsupported.single-empty-doc.630.zip
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/index/unsupported.single-empty-doc.630.zip b/lucene/backward-codecs/src/test/org/apache/lucene/index/unsupported.single-empty-doc.630.zip
new file mode 100644
index 0000000..1bf1d08
Binary files /dev/null and b/lucene/backward-codecs/src/test/org/apache/lucene/index/unsupported.single-empty-doc.630.zip differ


[22/23] lucene-solr:jira/solr-10879: Merge branch 'master' into jira/solr-10879

Posted by ab...@apache.org.
Merge branch 'master' into jira/solr-10879


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

Branch: refs/heads/jira/solr-10879
Commit: c751fc9ffc998adfbbaaf6787a7e4375224b6980
Parents: 97ebece bc37e8b
Author: Andrzej Bialecki <ab...@apache.org>
Authored: Tue Jul 4 13:26:35 2017 +0200
Committer: Andrzej Bialecki <ab...@apache.org>
Committed: Tue Jul 4 13:26:35 2017 +0200

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |    6 +
 .../analysis/custom/TestCustomAnalyzer.java     |    8 +-
 .../lucene50/Lucene50SegmentInfoFormat.java     |   90 -
 .../apache/lucene/codecs/lucene50/package.html  |   25 -
 .../codecs/lucene53/Lucene53NormsFormat.java    |   91 -
 .../codecs/lucene53/Lucene53NormsProducer.java  |  236 ---
 .../lucene/codecs/lucene53/package-info.java    |   23 -
 .../lucene54/Lucene54DocValuesConsumer.java     |  797 --------
 .../lucene54/Lucene54DocValuesFormat.java       |  186 --
 .../lucene54/Lucene54DocValuesProducer.java     | 1803 ------------------
 .../lucene/codecs/lucene54/package-info.java    |  403 ----
 .../lucene/codecs/lucene60/Lucene60Codec.java   |  177 --
 .../apache/lucene/codecs/lucene60/package.html  |   25 -
 .../lucene/codecs/lucene62/Lucene62Codec.java   |  176 --
 .../apache/lucene/codecs/lucene62/package.html  |   25 -
 .../apache/lucene/index/FixBrokenOffsets.java   |  138 --
 .../java/org/apache/lucene/index/package.html   |   27 -
 .../services/org.apache.lucene.codecs.Codec     |    2 -
 .../org.apache.lucene.codecs.DocValuesFormat    |    1 -
 .../lucene50/Lucene50RWSegmentInfoFormat.java   |  125 --
 .../lucene50/TestLucene50SegmentInfoFormat.java |   50 -
 .../codecs/lucene53/Lucene53NormsConsumer.java  |  159 --
 .../codecs/lucene53/Lucene53RWNormsFormat.java  |   31 -
 .../lucene53/TestLucene53NormsFormat.java       |   44 -
 .../lucene54/TestLucene54DocValuesFormat.java   |  640 -------
 .../lucene/codecs/lucene60/Lucene60RWCodec.java |   38 -
 .../lucene/codecs/lucene62/Lucene62RWCodec.java |   44 -
 .../lucene62/Lucene62RWSegmentInfoFormat.java   |  193 --
 .../lucene62/TestLucene62SegmentInfoFormat.java |   48 -
 .../index/TestBackwardsCompatibility.java       |   64 +-
 .../lucene/index/TestFixBrokenOffsets.java      |  112 --
 .../lucene/index/TestIndexWriterOnOldIndex.java |    7 +-
 .../lucene/index/TestManyPointsInOldIndex.java  |    1 +
 .../org/apache/lucene/index/dvupdates.6.0.0.zip |  Bin 3420 -> 0 bytes
 .../org/apache/lucene/index/empty.6.0.0.zip     |  Bin 225 -> 0 bytes
 .../org/apache/lucene/index/index.6.0.0-cfs.zip |  Bin 15807 -> 0 bytes
 .../apache/lucene/index/index.6.0.0-nocfs.zip   |  Bin 15806 -> 0 bytes
 .../org/apache/lucene/index/index.6.0.1-cfs.zip |  Bin 15820 -> 0 bytes
 .../apache/lucene/index/index.6.0.1-nocfs.zip   |  Bin 15823 -> 0 bytes
 .../org/apache/lucene/index/index.6.1.0-cfs.zip |  Bin 15803 -> 0 bytes
 .../apache/lucene/index/index.6.1.0-nocfs.zip   |  Bin 15829 -> 0 bytes
 .../org/apache/lucene/index/index.6.2.0-cfs.zip |  Bin 15880 -> 0 bytes
 .../apache/lucene/index/index.6.2.0-nocfs.zip   |  Bin 15867 -> 0 bytes
 .../org/apache/lucene/index/index.6.2.1-cfs.zip |  Bin 15851 -> 0 bytes
 .../apache/lucene/index/index.6.2.1-nocfs.zip   |  Bin 15845 -> 0 bytes
 .../org/apache/lucene/index/index.6.3.0-cfs.zip |  Bin 15869 -> 0 bytes
 .../apache/lucene/index/index.6.3.0-nocfs.zip   |  Bin 15876 -> 0 bytes
 .../org/apache/lucene/index/index.6.4.0-cfs.zip |  Bin 15829 -> 0 bytes
 .../apache/lucene/index/index.6.4.0-nocfs.zip   |  Bin 15831 -> 0 bytes
 .../org/apache/lucene/index/index.6.4.1-cfs.zip |  Bin 15848 -> 0 bytes
 .../apache/lucene/index/index.6.4.1-nocfs.zip   |  Bin 15838 -> 0 bytes
 .../org/apache/lucene/index/index.6.4.2-cfs.zip |  Bin 15856 -> 0 bytes
 .../apache/lucene/index/index.6.4.2-nocfs.zip   |  Bin 15886 -> 0 bytes
 .../org/apache/lucene/index/index.6.5.0-cfs.zip |  Bin 15832 -> 0 bytes
 .../apache/lucene/index/index.6.5.0-nocfs.zip   |  Bin 15836 -> 0 bytes
 .../org/apache/lucene/index/index.6.5.1-cfs.zip |  Bin 15842 -> 0 bytes
 .../apache/lucene/index/index.6.5.1-nocfs.zip   |  Bin 15827 -> 0 bytes
 .../org/apache/lucene/index/index.6.6.0-cfs.zip |  Bin 15875 -> 0 bytes
 .../apache/lucene/index/index.6.6.0-nocfs.zip   |  Bin 15883 -> 0 bytes
 .../lucene/index/index.630.brokenoffsets.zip    |  Bin 3203 -> 0 bytes
 .../lucene/index/index.single-empty-doc.630.zip |  Bin 1363 -> 0 bytes
 .../org/apache/lucene/index/manypointsindex.zip |  Bin 3739 -> 0 bytes
 .../org/apache/lucene/index/maxposindex.zip     |  Bin 1702 -> 0 bytes
 .../org/apache/lucene/index/moreterms.6.0.0.zip |  Bin 157215 -> 0 bytes
 .../org/apache/lucene/index/sorted.6.2.0.zip    |  Bin 157007 -> 0 bytes
 .../org/apache/lucene/index/sorted.6.2.1.zip    |  Bin 100974 -> 0 bytes
 .../org/apache/lucene/index/sorted.6.3.0.zip    |  Bin 71090 -> 0 bytes
 .../lucene/index/unsupported.6.0.0-cfs.zip      |  Bin 0 -> 15807 bytes
 .../lucene/index/unsupported.6.0.0-nocfs.zip    |  Bin 0 -> 15806 bytes
 .../lucene/index/unsupported.6.0.1-cfs.zip      |  Bin 0 -> 15820 bytes
 .../lucene/index/unsupported.6.0.1-nocfs.zip    |  Bin 0 -> 15823 bytes
 .../lucene/index/unsupported.6.1.0-cfs.zip      |  Bin 0 -> 15803 bytes
 .../lucene/index/unsupported.6.1.0-nocfs.zip    |  Bin 0 -> 15829 bytes
 .../lucene/index/unsupported.6.2.0-cfs.zip      |  Bin 0 -> 15880 bytes
 .../lucene/index/unsupported.6.2.0-nocfs.zip    |  Bin 0 -> 15867 bytes
 .../lucene/index/unsupported.6.2.1-cfs.zip      |  Bin 0 -> 15851 bytes
 .../lucene/index/unsupported.6.2.1-nocfs.zip    |  Bin 0 -> 15845 bytes
 .../lucene/index/unsupported.6.3.0-cfs.zip      |  Bin 0 -> 15869 bytes
 .../lucene/index/unsupported.6.3.0-nocfs.zip    |  Bin 0 -> 15876 bytes
 .../lucene/index/unsupported.6.4.0-cfs.zip      |  Bin 0 -> 15829 bytes
 .../lucene/index/unsupported.6.4.0-nocfs.zip    |  Bin 0 -> 15831 bytes
 .../lucene/index/unsupported.6.4.1-cfs.zip      |  Bin 0 -> 15848 bytes
 .../lucene/index/unsupported.6.4.1-nocfs.zip    |  Bin 0 -> 15838 bytes
 .../lucene/index/unsupported.6.4.2-cfs.zip      |  Bin 0 -> 15856 bytes
 .../lucene/index/unsupported.6.4.2-nocfs.zip    |  Bin 0 -> 15886 bytes
 .../lucene/index/unsupported.6.5.0-cfs.zip      |  Bin 0 -> 15832 bytes
 .../lucene/index/unsupported.6.5.0-nocfs.zip    |  Bin 0 -> 15836 bytes
 .../lucene/index/unsupported.6.5.1-cfs.zip      |  Bin 0 -> 15842 bytes
 .../lucene/index/unsupported.6.5.1-nocfs.zip    |  Bin 0 -> 15827 bytes
 .../lucene/index/unsupported.6.6.0-cfs.zip      |  Bin 0 -> 15875 bytes
 .../lucene/index/unsupported.6.6.0-nocfs.zip    |  Bin 0 -> 15883 bytes
 .../org/apache/lucene/index/CheckIndex.java     |   47 +-
 .../index/IndexFormatTooOldException.java       |    2 +-
 .../org/apache/lucene/index/IndexWriter.java    |    6 +-
 .../org/apache/lucene/index/SegmentInfos.java   |    2 +-
 .../search/similarities/BM25Similarity.java     |   38 +-
 .../search/similarities/SimilarityBase.java     |   25 +-
 .../search/similarities/TFIDFSimilarity.java    |   25 +-
 .../java/org/apache/lucene/util/Version.java    |  101 +-
 .../apache/lucene/index/TestSegmentInfos.java   |   14 +-
 .../search/similarities/TestBM25Similarity.java |   42 -
 .../similarities/TestClassicSimilarity.java     |   59 +-
 .../search/similarities/TestSimilarityBase.java |   39 -
 .../org/apache/lucene/util/TestVersion.java     |   71 +-
 .../lucene/search/join/BlockJoinSelector.java   |  328 +---
 .../lucene/search/join/ToParentDocValues.java   |  296 +++
 .../search/join/TestBlockJoinSelector.java      |   76 +-
 .../java/org/apache/lucene/util/TestUtil.java   |    4 +-
 lucene/version.properties                       |    2 +-
 solr/CHANGES.txt                                |   74 +
 .../TestSolrEntityProcessorEndToEnd.java        |    4 +-
 .../client/solrj/embedded/JettySolrRunner.java  |    7 +
 .../org/apache/solr/cloud/AddReplicaCmd.java    |    2 +-
 .../org/apache/solr/cloud/DeleteReplicaCmd.java |    2 +-
 .../cloud/LeaderInitiatedRecoveryThread.java    |    7 +-
 .../org/apache/solr/cloud/MoveReplicaCmd.java   |   43 +-
 .../OverseerAutoReplicaFailoverThread.java      |    7 +-
 .../cloud/OverseerCollectionMessageHandler.java |   30 +-
 .../org/apache/solr/cloud/RecoveryStrategy.java |   32 +-
 .../org/apache/solr/cloud/ReplaceNodeCmd.java   |   43 +-
 .../org/apache/solr/cloud/SyncStrategy.java     |    8 +-
 .../org/apache/solr/cloud/ZkController.java     |    7 +-
 .../apache/solr/handler/CdcrRequestHandler.java |   12 +-
 .../solr/handler/CdcrUpdateLogSynchronizer.java |    7 +-
 .../org/apache/solr/handler/IndexFetcher.java   |   28 +-
 .../solr/handler/admin/CollectionsHandler.java  |    7 +-
 .../component/HttpShardHandlerFactory.java      |    4 +-
 .../solr/schema/FieldTypePluginLoader.java      |    6 +-
 .../org/apache/solr/schema/IndexSchema.java     |    4 +-
 .../apache/solr/schema/IndexSchemaFactory.java  |    9 +-
 .../similarities/SchemaSimilarityFactory.java   |    4 +-
 .../org/apache/solr/util/TestInjection.java     |    2 +-
 .../configsets/_default/conf/solrconfig.xml     |    2 +-
 solr/core/src/test-files/solr/solr.xml          |    1 +
 .../client/solrj/impl/ConnectionReuseTest.java  |    4 +-
 .../solr/cloud/BaseCdcrDistributedZkTest.java   |    3 +-
 .../solr/cloud/BasicDistributedZkTest.java      |   24 +-
 .../cloud/ChaosMonkeyNothingIsSafeTest.java     |   19 +-
 ...MonkeyNothingIsSafeWithPullReplicasTest.java |   20 +-
 .../cloud/FullSolrCloudDistribCmdsTest.java     |    3 +-
 .../FullThrottleStoppableIndexingThread.java    |    4 +-
 .../apache/solr/cloud/HttpPartitionTest.java    |    4 +-
 .../apache/solr/cloud/MigrateRouteKeyTest.java  |    8 +-
 .../apache/solr/cloud/MoveReplicaHDFSTest.java  |   53 +
 .../org/apache/solr/cloud/MoveReplicaTest.java  |   84 +-
 .../org/apache/solr/cloud/ShardSplitTest.java   |   10 +-
 .../org/apache/solr/cloud/SyncSliceTest.java    |    5 +-
 .../apache/solr/cloud/TestCloudRecovery.java    |    8 -
 .../org/apache/solr/cloud/TestPrepRecovery.java |  109 ++
 .../cloud/TestRandomRequestDistribution.java    |    8 +-
 .../cloud/TestRequestStatusCollectionAPI.java   |    3 +-
 .../solr/cloud/UnloadDistributedZkTest.java     |   31 +-
 .../apache/solr/cloud/hdfs/StressHdfsTest.java  |    3 +-
 .../solr/core/OpenCloseCoreStressTest.java      |    8 +-
 .../solr/handler/TestReplicationHandler.java    |    4 +-
 .../handler/TestReplicationHandlerBackup.java   |    4 +-
 .../apache/solr/handler/TestRestoreCore.java    |    4 +-
 .../handler/admin/CoreAdminHandlerTest.java     |   36 +-
 .../metrics/reporters/SolrJmxReporterTest.java  |   18 +-
 .../TestNonDefinedSimilarityFactory.java        |   10 -
 .../example-DIH/solr/atom/conf/solrconfig.xml   |    2 +-
 .../example-DIH/solr/db/conf/solrconfig.xml     |    2 +-
 .../example-DIH/solr/mail/conf/solrconfig.xml   |    2 +-
 .../example-DIH/solr/solr/conf/solrconfig.xml   |    2 +-
 .../example-DIH/solr/tika/conf/solrconfig.xml   |    2 +-
 solr/example/files/conf/solrconfig.xml          |    2 +-
 .../configsets/_default/conf/solrconfig.xml     |    2 +-
 .../conf/solrconfig.xml                         |    2 +-
 .../src/uploading-data-with-index-handlers.adoc |    2 +-
 .../solr/client/solrj/impl/CloudSolrClient.java |   67 +-
 .../solrj/impl/ConcurrentUpdateSolrClient.java  |   36 +
 .../solr/client/solrj/impl/HttpSolrClient.java  |   38 +-
 .../client/solrj/impl/LBHttpSolrClient.java     |   60 +-
 .../client/solrj/SolrExampleBinaryTest.java     |    3 +-
 .../solr/client/solrj/SolrExampleXMLTest.java   |    3 +-
 .../solr/client/solrj/SolrExceptionTest.java    |    3 +-
 .../client/solrj/SolrSchemalessExampleTest.java |    3 +-
 .../solr/client/solrj/TestLBHttpSolrClient.java |    4 +-
 .../solrj/impl/BasicHttpSolrClientTest.java     |    3 +-
 .../client/solrj/impl/CloudSolrClientTest.java  |    4 +-
 .../solrj/impl/HttpSolrClientConPoolTest.java   |    3 +-
 .../java/org/apache/solr/SolrJettyTestBase.java |    3 +-
 .../java/org/apache/solr/SolrTestCaseJ4.java    |  132 ++
 .../cloud/AbstractFullDistribZkTestBase.java    |   31 +-
 184 files changed, 1480 insertions(+), 6742 deletions(-)
----------------------------------------------------------------------



[18/23] lucene-solr:jira/solr-10879: SOLR-10456: Deprecate timeout related setters from SolrClients, and replace with Builder based implementation

Posted by ab...@apache.org.
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c163658/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java
----------------------------------------------------------------------
diff --git a/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java b/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java
index f9a93f7..f034f32 100644
--- a/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java
+++ b/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java
@@ -2300,6 +2300,50 @@ public abstract class SolrTestCaseJ4 extends LuceneTestCase {
    * Tests that do not wish to have any randomized behavior should use the 
    * {@link org.apache.solr.client.solrj.impl.CloudSolrClient.Builder} class directly
    */ 
+  public static CloudSolrClient getCloudSolrClient(String zkHost, boolean shardLeadersOnly, int socketTimeoutMillis) {
+    if (shardLeadersOnly) {
+      return new CloudSolrClientBuilder()
+          .withZkHost(zkHost)
+          .sendUpdatesOnlyToShardLeaders()
+          .withSocketTimeout(socketTimeoutMillis)
+          .build();
+    }
+    return new CloudSolrClientBuilder()
+        .withZkHost(zkHost)
+        .sendUpdatesToAllReplicasInShard()
+        .withSocketTimeout(socketTimeoutMillis)
+        .build();
+  }
+  
+  /**
+   * This method <i>may</i> randomize unspecified aspects of the resulting SolrClient.
+   * Tests that do not wish to have any randomized behavior should use the 
+   * {@link org.apache.solr.client.solrj.impl.CloudSolrClient.Builder} class directly
+   */ 
+  public static CloudSolrClient getCloudSolrClient(String zkHost, boolean shardLeadersOnly, int connectionTimeoutMillis, int socketTimeoutMillis) {
+    if (shardLeadersOnly) {
+      return new CloudSolrClientBuilder()
+          .withZkHost(zkHost)
+          .sendUpdatesOnlyToShardLeaders()
+          .withConnectionTimeout(connectionTimeoutMillis)
+          .withSocketTimeout(socketTimeoutMillis)
+          .build();
+    }
+    return new CloudSolrClientBuilder()
+        .withZkHost(zkHost)
+        .sendUpdatesToAllReplicasInShard()
+        .withConnectionTimeout(connectionTimeoutMillis)
+        .withSocketTimeout(socketTimeoutMillis)
+        .build();
+  }
+  
+  
+  
+  /**
+   * This method <i>may</i> randomize unspecified aspects of the resulting SolrClient.
+   * Tests that do not wish to have any randomized behavior should use the 
+   * {@link org.apache.solr.client.solrj.impl.CloudSolrClient.Builder} class directly
+   */ 
   public static CloudSolrClient getCloudSolrClient(String zkHost, boolean shardLeadersOnly, HttpClient httpClient) {
     if (shardLeadersOnly) {
       return new CloudSolrClientBuilder()
@@ -2318,6 +2362,31 @@ public abstract class SolrTestCaseJ4 extends LuceneTestCase {
   /**
    * This method <i>may</i> randomize unspecified aspects of the resulting SolrClient.
    * Tests that do not wish to have any randomized behavior should use the 
+   * {@link org.apache.solr.client.solrj.impl.CloudSolrClient.Builder} class directly
+   */ 
+  public static CloudSolrClient getCloudSolrClient(String zkHost, boolean shardLeadersOnly, HttpClient httpClient,
+      int connectionTimeoutMillis, int socketTimeoutMillis) {
+    if (shardLeadersOnly) {
+      return new CloudSolrClientBuilder()
+          .withZkHost(zkHost)
+          .withHttpClient(httpClient)
+          .sendUpdatesOnlyToShardLeaders()
+          .withConnectionTimeout(connectionTimeoutMillis)
+          .withSocketTimeout(socketTimeoutMillis)
+          .build();
+    }
+    return new CloudSolrClientBuilder()
+        .withZkHost(zkHost)
+        .withHttpClient(httpClient)
+        .sendUpdatesToAllReplicasInShard()
+        .withConnectionTimeout(connectionTimeoutMillis)
+        .withSocketTimeout(socketTimeoutMillis)
+        .build();
+  }
+  
+  /**
+   * This method <i>may</i> randomize unspecified aspects of the resulting SolrClient.
+   * Tests that do not wish to have any randomized behavior should use the 
    * {@link org.apache.solr.client.solrj.impl.ConcurrentUpdateSolrClient.Builder} class directly
    */ 
   public static ConcurrentUpdateSolrClient getConcurrentUpdateSolrClient(String baseSolrUrl, int queueSize, int threadCount) {
@@ -2332,6 +2401,19 @@ public abstract class SolrTestCaseJ4 extends LuceneTestCase {
    * Tests that do not wish to have any randomized behavior should use the 
    * {@link org.apache.solr.client.solrj.impl.ConcurrentUpdateSolrClient.Builder} class directly
    */ 
+  public static ConcurrentUpdateSolrClient getConcurrentUpdateSolrClient(String baseSolrUrl, int queueSize, int threadCount, int connectionTimeoutMillis) {
+    return new ConcurrentUpdateSolrClient.Builder(baseSolrUrl)
+        .withQueueSize(queueSize)
+        .withThreadCount(threadCount)
+        .withConnectionTimeout(connectionTimeoutMillis)
+        .build();
+  }
+  
+  /**
+   * This method <i>may</i> randomize unspecified aspects of the resulting SolrClient.
+   * Tests that do not wish to have any randomized behavior should use the 
+   * {@link org.apache.solr.client.solrj.impl.ConcurrentUpdateSolrClient.Builder} class directly
+   */ 
   public static ConcurrentUpdateSolrClient getConcurrentUpdateSolrClient(String baseSolrUrl, HttpClient httpClient, int queueSize, int threadCount) {
     return new ConcurrentUpdateSolrClient.Builder(baseSolrUrl)
         .withHttpClient(httpClient)
@@ -2357,6 +2439,21 @@ public abstract class SolrTestCaseJ4 extends LuceneTestCase {
    * Tests that do not wish to have any randomized behavior should use the 
    * {@link org.apache.solr.client.solrj.impl.LBHttpSolrClient.Builder} class directly
    */ 
+  public static LBHttpSolrClient getLBHttpSolrClient(HttpClient client, int connectionTimeoutMillis,
+      int socketTimeoutMillis, String... solrUrls) {
+    return new LBHttpSolrClient.Builder()
+        .withHttpClient(client)
+        .withBaseSolrUrls(solrUrls)
+        .withConnectionTimeout(connectionTimeoutMillis)
+        .withSocketTimeout(socketTimeoutMillis)
+        .build();
+  }
+  
+  /**
+   * This method <i>may</i> randomize unspecified aspects of the resulting SolrClient.
+   * Tests that do not wish to have any randomized behavior should use the 
+   * {@link org.apache.solr.client.solrj.impl.LBHttpSolrClient.Builder} class directly
+   */ 
   public static LBHttpSolrClient getLBHttpSolrClient(String... solrUrls) throws MalformedURLException {
     return new LBHttpSolrClient.Builder()
         .withBaseSolrUrls(solrUrls)
@@ -2398,6 +2495,18 @@ public abstract class SolrTestCaseJ4 extends LuceneTestCase {
         .withHttpClient(httpClient)
         .build();
   }
+  
+  /**
+   * This method <i>may</i> randomize unspecified aspects of the resulting SolrClient.
+   * Tests that do not wish to have any randomized behavior should use the 
+   * {@link org.apache.solr.client.solrj.impl.HttpSolrClient.Builder} class directly
+   */ 
+  public static HttpSolrClient getHttpSolrClient(String url, HttpClient httpClient, int connectionTimeoutMillis) {
+    return new Builder(url)
+        .withHttpClient(httpClient)
+        .withConnectionTimeout(connectionTimeoutMillis)
+        .build();
+  }
 
   /**
    * This method <i>may</i> randomize unspecified aspects of the resulting SolrClient.
@@ -2408,6 +2517,29 @@ public abstract class SolrTestCaseJ4 extends LuceneTestCase {
     return new Builder(url)
         .build();
   }
+  
+  /**
+   * This method <i>may</i> randomize unspecified aspects of the resulting SolrClient.
+   * Tests that do not wish to have any randomized behavior should use the 
+   * {@link org.apache.solr.client.solrj.impl.HttpSolrClient.Builder} class directly
+   */ 
+  public static HttpSolrClient getHttpSolrClient(String url, int connectionTimeoutMillis) {
+    return new Builder(url)
+        .withConnectionTimeout(connectionTimeoutMillis)
+        .build();
+  }
+  
+  /**
+   * This method <i>may</i> randomize unspecified aspects of the resulting SolrClient.
+   * Tests that do not wish to have any randomized behavior should use the 
+   * {@link org.apache.solr.client.solrj.impl.HttpSolrClient.Builder} class directly
+   */ 
+  public static HttpSolrClient getHttpSolrClient(String url, int connectionTimeoutMillis, int socketTimeoutMillis) {
+    return new Builder(url)
+        .withConnectionTimeout(connectionTimeoutMillis)
+        .withSocketTimeout(socketTimeoutMillis)
+        .build();
+  }
 
   /** 
    * Returns a randomly generated Date in the appropriate Solr external (input) format 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c163658/solr/test-framework/src/java/org/apache/solr/cloud/AbstractFullDistribZkTestBase.java
----------------------------------------------------------------------
diff --git a/solr/test-framework/src/java/org/apache/solr/cloud/AbstractFullDistribZkTestBase.java b/solr/test-framework/src/java/org/apache/solr/cloud/AbstractFullDistribZkTestBase.java
index 6e1613f..72caa55 100644
--- a/solr/test-framework/src/java/org/apache/solr/cloud/AbstractFullDistribZkTestBase.java
+++ b/solr/test-framework/src/java/org/apache/solr/cloud/AbstractFullDistribZkTestBase.java
@@ -295,11 +295,9 @@ public abstract class AbstractFullDistribZkTestBase extends AbstractDistribZkTes
   }
   
   protected CloudSolrClient createCloudClient(String defaultCollection) {
-    CloudSolrClient client = getCloudSolrClient(zkServer.getZkAddress(), random().nextBoolean());
+    CloudSolrClient client = getCloudSolrClient(zkServer.getZkAddress(), random().nextBoolean(), 30000, 60000);
     client.setParallelUpdates(random().nextBoolean());
     if (defaultCollection != null) client.setDefaultCollection(defaultCollection);
-    client.getLbClient().setConnectionTimeout(30000);
-    client.getLbClient().setSoTimeout(60000);
     return client;
   }
 
@@ -1708,15 +1706,29 @@ public abstract class AbstractFullDistribZkTestBase extends AbstractDistribZkTes
   protected SolrClient createNewSolrClient(int port) {
     return createNewSolrClient(DEFAULT_COLLECTION, port);
   }
+  
+  protected SolrClient createNewSolrClient(int port, int connectionTimeoutMillis, int socketTimeoutMillis) {
+    return createNewSolrClient(DEFAULT_COLLECTION, port, connectionTimeoutMillis, socketTimeoutMillis);
+  }
 
   protected SolrClient createNewSolrClient(String coreName, int port) {
     try {
       // setup the server...
       String baseUrl = buildUrl(port);
       String url = baseUrl + (baseUrl.endsWith("/") ? "" : "/") + coreName;
-      HttpSolrClient client = getHttpSolrClient(url);
-      client.setConnectionTimeout(DEFAULT_CONNECTION_TIMEOUT);
-      client.setSoTimeout(60000);
+      HttpSolrClient client = getHttpSolrClient(url, DEFAULT_CONNECTION_TIMEOUT, 60000);
+      return client;
+    } catch (Exception ex) {
+      throw new RuntimeException(ex);
+    }
+  }
+  
+  protected SolrClient createNewSolrClient(String coreName, int port, int connectionTimeoutMillis, int socketTimeoutMillis) {
+    try {
+      // setup the server...
+      String baseUrl = buildUrl(port);
+      String url = baseUrl + (baseUrl.endsWith("/") ? "" : "/") + coreName;
+      HttpSolrClient client = getHttpSolrClient(url, connectionTimeoutMillis, socketTimeoutMillis);
       return client;
     } catch (Exception ex) {
       throw new RuntimeException(ex);
@@ -1726,8 +1738,7 @@ public abstract class AbstractFullDistribZkTestBase extends AbstractDistribZkTes
   protected SolrClient createNewSolrClient(String collection, String baseUrl) {
     try {
       // setup the server...
-      HttpSolrClient client = getHttpSolrClient(baseUrl + "/" + collection);
-      client.setConnectionTimeout(DEFAULT_CONNECTION_TIMEOUT);
+      HttpSolrClient client = getHttpSolrClient(baseUrl + "/" + collection, DEFAULT_CONNECTION_TIMEOUT);
       return client;
     }
     catch (Exception ex) {
@@ -1811,9 +1822,7 @@ public abstract class AbstractFullDistribZkTestBase extends AbstractDistribZkTes
         boolean updatesToLeaders = random().nextBoolean();
         boolean parallelUpdates = random().nextBoolean();
         commonCloudSolrClient = getCloudSolrClient(zkServer.getZkAddress(),
-                updatesToLeaders);
-        commonCloudSolrClient.getLbClient().setConnectionTimeout(5000);
-        commonCloudSolrClient.getLbClient().setSoTimeout(120000);
+                updatesToLeaders, 5000, 120000);
         commonCloudSolrClient.setParallelUpdates(parallelUpdates);
         commonCloudSolrClient.setDefaultCollection(DEFAULT_COLLECTION);
         commonCloudSolrClient.connect();


[09/23] lucene-solr:jira/solr-10879: Remove FixBrokenOffsets since 7.0+ indices cannot have broken offsets.

Posted by ab...@apache.org.
Remove FixBrokenOffsets since 7.0+ indices cannot have broken offsets.


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

Branch: refs/heads/jira/solr-10879
Commit: 869a48ccacddd3ad9e1dac68eee6d66b78a7049b
Parents: e809e09
Author: Adrien Grand <jp...@gmail.com>
Authored: Mon Jul 3 14:12:25 2017 +0200
Committer: Adrien Grand <jp...@gmail.com>
Committed: Mon Jul 3 14:12:25 2017 +0200

----------------------------------------------------------------------
 .../apache/lucene/index/FixBrokenOffsets.java   | 138 -------------------
 .../lucene/index/TestFixBrokenOffsets.java      | 112 ---------------
 2 files changed, 250 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/869a48cc/lucene/backward-codecs/src/java/org/apache/lucene/index/FixBrokenOffsets.java
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/java/org/apache/lucene/index/FixBrokenOffsets.java b/lucene/backward-codecs/src/java/org/apache/lucene/index/FixBrokenOffsets.java
deleted file mode 100644
index 9b3615e..0000000
--- a/lucene/backward-codecs/src/java/org/apache/lucene/index/FixBrokenOffsets.java
+++ /dev/null
@@ -1,138 +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.index;
-
-import java.io.IOException;
-import java.nio.file.Files;
-import java.nio.file.Path;
-import java.nio.file.Paths;
-import java.util.List;
-
-import org.apache.lucene.store.Directory;
-import org.apache.lucene.store.FSDirectory;
-import org.apache.lucene.util.IOUtils;
-import org.apache.lucene.util.SuppressForbidden;
-
-/**
- * Command-line tool that reads from a source index and
- * writes to a dest index, correcting any broken offsets
- * in the process.
- *
- * @lucene.experimental
- */
-public class FixBrokenOffsets {
-  public SegmentInfos infos;
-
-  FSDirectory fsDir;
-
-  Path dir;
-
-  @SuppressForbidden(reason = "System.out required: command line tool")
-  public static void main(String[] args) throws IOException {
-    if (args.length < 2) {
-      System.err.println("Usage: FixBrokenOffsetse <srcDir> <destDir>");
-      return;
-    }
-    Path srcPath = Paths.get(args[0]);
-    if (!Files.exists(srcPath)) {
-      throw new RuntimeException("srcPath " + srcPath.toAbsolutePath() + " doesn't exist");
-    }
-    Path destPath = Paths.get(args[1]);
-    if (Files.exists(destPath)) {
-      throw new RuntimeException("destPath " + destPath.toAbsolutePath() + " already exists; please remove it and re-run");
-    }
-    Directory srcDir = FSDirectory.open(srcPath);
-    DirectoryReader reader = DirectoryReader.open(srcDir);
-
-    List<LeafReaderContext> leaves = reader.leaves();
-    CodecReader[] filtered = new CodecReader[leaves.size()];
-    for(int i=0;i<leaves.size();i++) {
-      filtered[i] = SlowCodecReaderWrapper.wrap(new FilterLeafReader(leaves.get(i).reader()) {
-          @Override
-          public Fields getTermVectors(int docID) throws IOException {
-            Fields termVectors = in.getTermVectors(docID);
-            if (termVectors == null) {
-              return null;
-            }
-            return new FilterFields(termVectors) {
-              @Override
-              public Terms terms(String field) throws IOException {
-                return new FilterTerms(super.terms(field)) {
-                  @Override
-                  public TermsEnum iterator() throws IOException {
-                    return new FilterTermsEnum(super.iterator()) {
-                      @Override
-                      public PostingsEnum postings(PostingsEnum reuse, int flags) throws IOException {
-                        return new FilterPostingsEnum(super.postings(reuse, flags)) {
-                          int nextLastStartOffset = 0;
-                          int lastStartOffset = 0;
-
-                          @Override
-                          public int nextPosition() throws IOException {
-                            int pos = super.nextPosition();
-                            lastStartOffset = nextLastStartOffset;
-                            nextLastStartOffset = startOffset();
-                            return pos;
-                          }
-                          
-                          @Override
-                          public int startOffset() throws IOException {
-                            int offset = super.startOffset();
-                            if (offset < lastStartOffset) {
-                              offset = lastStartOffset;
-                            }
-                            return offset;
-                          }
-                          
-                          @Override
-                          public int endOffset() throws IOException {
-                            int offset = super.endOffset();
-                            if (offset < lastStartOffset) {
-                              offset = lastStartOffset;
-                            }
-                            return offset;
-                          }
-                        };
-                      }
-                    };
-                  }
-                };
-              }
-            };
-          }
-
-          @Override
-          public CacheHelper getCoreCacheHelper() {
-            return null;
-          }
-
-          @Override
-          public CacheHelper getReaderCacheHelper() {
-            return null;
-          }
-        });
-    }
-
-    Directory destDir = FSDirectory.open(destPath);
-    // We need to maintain the same major version
-    int createdMajor = SegmentInfos.readLatestCommit(srcDir).getIndexCreatedVersionMajor();
-    new SegmentInfos(createdMajor).commit(destDir);
-    IndexWriter writer = new IndexWriter(destDir, new IndexWriterConfig());
-    writer.addIndexes(filtered);
-    IOUtils.close(writer, reader, srcDir, destDir);
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/869a48cc/lucene/backward-codecs/src/test/org/apache/lucene/index/TestFixBrokenOffsets.java
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/index/TestFixBrokenOffsets.java b/lucene/backward-codecs/src/test/org/apache/lucene/index/TestFixBrokenOffsets.java
deleted file mode 100644
index 46b30d3..0000000
--- a/lucene/backward-codecs/src/test/org/apache/lucene/index/TestFixBrokenOffsets.java
+++ /dev/null
@@ -1,112 +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.index;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.nio.file.Path;
-import java.util.List;
-
-import org.apache.lucene.store.Directory;
-import org.apache.lucene.store.FSDirectory;
-import org.apache.lucene.store.MockDirectoryWrapper;
-import org.apache.lucene.util.LuceneTestCase;
-import org.apache.lucene.util.TestUtil;
-
-public class TestFixBrokenOffsets extends LuceneTestCase {
-
-  // Run this in Lucene 6.x:
-  //
-  //     ant test -Dtestcase=TestFixBrokenOffsets -Dtestmethod=testCreateBrokenOffsetsIndex -Dtests.codec=default -Dtests.useSecurityManager=false
-  /*
-  public void testCreateBrokenOffsetsIndex() throws IOException {
-
-    Path indexDir = Paths.get("/tmp/brokenoffsets");
-    Files.deleteIfExists(indexDir);
-    Directory dir = newFSDirectory(indexDir);
-    IndexWriter writer = new IndexWriter(dir, new IndexWriterConfig());
-
-    Document doc = new Document();
-    FieldType fieldType = new FieldType(TextField.TYPE_STORED);
-    fieldType.setStoreTermVectors(true);
-    fieldType.setStoreTermVectorPositions(true);
-    fieldType.setStoreTermVectorOffsets(true);
-    Field field = new Field("foo", "bar", fieldType);
-    field.setTokenStream(new CannedTokenStream(new Token("foo", 10, 13), new Token("foo", 7, 9)));
-    doc.add(field);
-    writer.addDocument(doc);
-    writer.commit();
-
-    // 2nd segment
-    doc = new Document();
-    field = new Field("foo", "bar", fieldType);
-    field.setTokenStream(new CannedTokenStream(new Token("bar", 15, 17), new Token("bar", 1, 5)));
-    doc.add(field);
-    writer.addDocument(doc);
-    
-    writer.close();
-
-    dir.close();
-  }
-  */
-
-  public void testFixBrokenOffsetsIndex() throws IOException {
-    InputStream resource = getClass().getResourceAsStream("index.630.brokenoffsets.zip");
-    assertNotNull("Broken offsets index not found", resource);
-    Path path = createTempDir("brokenoffsets");
-    TestUtil.unzip(resource, path);
-    Directory dir = newFSDirectory(path);
-
-    // OK: index is 6.3.0 so offsets not checked:
-    TestUtil.checkIndex(dir);
-    
-    MockDirectoryWrapper tmpDir = newMockDirectory();
-    tmpDir.setCheckIndexOnClose(false);
-    IndexWriter w = new IndexWriter(tmpDir, new IndexWriterConfig());
-    IndexWriter finalW = w;
-    IllegalArgumentException e = expectThrows(IllegalArgumentException.class, () -> finalW.addIndexes(dir));
-    assertTrue(e.getMessage(), e.getMessage().startsWith("Cannot use addIndexes(Directory) with indexes that have been created by a different Lucene version."));
-    w.close();
-    // OK: addIndexes(Directory...) refuses to execute if the index creation version is different so broken offsets are not carried over
-    tmpDir.close();
-
-    final MockDirectoryWrapper tmpDir2 = newMockDirectory();
-    tmpDir2.setCheckIndexOnClose(false);
-    w = new IndexWriter(tmpDir2, new IndexWriterConfig());
-    DirectoryReader reader = DirectoryReader.open(dir);
-    List<LeafReaderContext> leaves = reader.leaves();
-    CodecReader[] codecReaders = new CodecReader[leaves.size()];
-    for(int i=0;i<leaves.size();i++) {
-      codecReaders[i] = (CodecReader) leaves.get(i).reader();
-    }
-    IndexWriter finalW2 = w;
-    e = expectThrows(IllegalArgumentException.class, () -> finalW2.addIndexes(codecReaders));
-    assertEquals("Cannot merge a segment that has been created with major version 6 into this index which has been created by major version 7", e.getMessage());
-    reader.close();
-    w.close();
-    tmpDir2.close();
-
-    // Now run the tool and confirm the broken offsets are fixed:
-    Path path2 = createTempDir("fixedbrokenoffsets").resolve("subdir");
-    FixBrokenOffsets.main(new String[] {path.toString(), path2.toString()});
-    Directory tmpDir3 = FSDirectory.open(path2);
-    TestUtil.checkIndex(tmpDir3);
-    tmpDir3.close();
-    
-    dir.close();
-  }
-}


[21/23] lucene-solr:jira/solr-10879: SOLR-10878: MOVEREPLICA command may lose data when replicationFactor==1.

Posted by ab...@apache.org.
SOLR-10878: MOVEREPLICA command may lose data when replicationFactor==1.


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

Branch: refs/heads/jira/solr-10879
Commit: bc37e8b4cc45794af9271c158ec36b2902cddc2f
Parents: e9d33ee
Author: Andrzej Bialecki <ab...@apache.org>
Authored: Tue Jul 4 11:33:41 2017 +0200
Committer: Andrzej Bialecki <ab...@apache.org>
Committed: Tue Jul 4 13:25:47 2017 +0200

----------------------------------------------------------------------
 solr/CHANGES.txt                                |  2 +
 .../org/apache/solr/cloud/AddReplicaCmd.java    |  2 +-
 .../org/apache/solr/cloud/DeleteReplicaCmd.java |  2 +-
 .../org/apache/solr/cloud/MoveReplicaCmd.java   | 43 ++++++++--
 .../cloud/OverseerCollectionMessageHandler.java | 23 +++---
 .../org/apache/solr/cloud/ReplaceNodeCmd.java   | 43 +++++++---
 .../apache/solr/cloud/MoveReplicaHDFSTest.java  | 53 ++++++++++++
 .../org/apache/solr/cloud/MoveReplicaTest.java  | 84 +++++++++++++++++++-
 8 files changed, 220 insertions(+), 32 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bc37e8b4/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 360916b..403b6cd 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -299,6 +299,8 @@ Bug Fixes
 * SOLR-6807: CloudSolrClient's ZK state version check with the server was ignored when handleSelect=false
   (David Smiley)
 
+* SOLR-10878: MOVEREPLICA command may lose data when replicationFactor is 1. (ab, shalin)
+
 Optimizations
 ----------------------
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bc37e8b4/solr/core/src/java/org/apache/solr/cloud/AddReplicaCmd.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/AddReplicaCmd.java b/solr/core/src/java/org/apache/solr/cloud/AddReplicaCmd.java
index 63acdd1..c42d073 100644
--- a/solr/core/src/java/org/apache/solr/cloud/AddReplicaCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/AddReplicaCmd.java
@@ -68,7 +68,7 @@ public class AddReplicaCmd implements OverseerCollectionMessageHandler.Cmd {
 
   ZkNodeProps addReplica(ClusterState clusterState, ZkNodeProps message, NamedList results, Runnable onComplete)
       throws KeeperException, InterruptedException {
-    log.info("addReplica() : {}", Utils.toJSONString(message));
+    log.debug("addReplica() : {}", Utils.toJSONString(message));
     String collection = message.getStr(COLLECTION_PROP);
     String node = message.getStr(CoreAdminParams.NODE);
     String shard = message.getStr(SHARD_ID_PROP);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bc37e8b4/solr/core/src/java/org/apache/solr/cloud/DeleteReplicaCmd.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/DeleteReplicaCmd.java b/solr/core/src/java/org/apache/solr/cloud/DeleteReplicaCmd.java
index b79fa46..e71d7e8 100644
--- a/solr/core/src/java/org/apache/solr/cloud/DeleteReplicaCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/DeleteReplicaCmd.java
@@ -265,7 +265,7 @@ public class DeleteReplicaCmd implements Cmd {
       try {
         if (!callable.call())
           throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
-                  "Could not  remove replica : " + collectionName + "/" + shard + "/" + replicaName);
+                  "Could not remove replica : " + collectionName + "/" + shard + "/" + replicaName);
       } catch (InterruptedException | KeeperException e) {
         throw e;
       } catch (Exception ex) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bc37e8b4/solr/core/src/java/org/apache/solr/cloud/MoveReplicaCmd.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/MoveReplicaCmd.java b/solr/core/src/java/org/apache/solr/cloud/MoveReplicaCmd.java
index fed1398..53d05e1 100644
--- a/solr/core/src/java/org/apache/solr/cloud/MoveReplicaCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/MoveReplicaCmd.java
@@ -22,6 +22,8 @@ import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
 import java.util.Locale;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
 
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.cloud.ClusterState;
@@ -29,6 +31,7 @@ import org.apache.solr.common.cloud.DocCollection;
 import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.common.cloud.Slice;
 import org.apache.solr.common.cloud.ZkNodeProps;
+import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.params.CoreAdminParams;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.Utils;
@@ -56,10 +59,11 @@ public class MoveReplicaCmd implements Cmd{
   }
 
   private void moveReplica(ClusterState clusterState, ZkNodeProps message, NamedList results) throws Exception {
-    log.info("moveReplica() : {}", Utils.toJSONString(message));
+    log.debug("moveReplica() : {}", Utils.toJSONString(message));
     ocmh.checkRequired(message, COLLECTION_PROP, "targetNode");
     String collection = message.getStr(COLLECTION_PROP);
     String targetNode = message.getStr("targetNode");
+    int timeout = message.getInt("timeout", 10 * 60); // 10 minutes
 
     String async = message.getStr(ASYNC);
 
@@ -103,14 +107,14 @@ public class MoveReplicaCmd implements Cmd{
     assert slice != null;
     Object dataDir = replica.get("dataDir");
     if (dataDir != null && dataDir.toString().startsWith("hdfs:/")) {
-      moveHdfsReplica(clusterState, results, dataDir.toString(), targetNode, async, coll, replica, slice);
+      moveHdfsReplica(clusterState, results, dataDir.toString(), targetNode, async, coll, replica, slice, timeout);
     } else {
-      moveNormalReplica(clusterState, results, targetNode, async, coll, replica, slice);
+      moveNormalReplica(clusterState, results, targetNode, async, coll, replica, slice, timeout);
     }
   }
 
   private void moveHdfsReplica(ClusterState clusterState, NamedList results, String dataDir, String targetNode, String async,
-                                 DocCollection coll, Replica replica, Slice slice) throws Exception {
+                                 DocCollection coll, Replica replica, Slice slice, int timeout) throws Exception {
     String newCoreName = Assign.buildCoreName(coll, slice.getName(), replica.getType());
 
     ZkNodeProps removeReplicasProps = new ZkNodeProps(
@@ -154,7 +158,7 @@ public class MoveReplicaCmd implements Cmd{
   }
 
   private void moveNormalReplica(ClusterState clusterState, NamedList results, String targetNode, String async,
-                                 DocCollection coll, Replica replica, Slice slice) throws Exception {
+                                 DocCollection coll, Replica replica, Slice slice, int timeout) throws Exception {
     String newCoreName = Assign.buildCoreName(coll, slice.getName(), replica.getType());
     ZkNodeProps addReplicasProps = new ZkNodeProps(
         COLLECTION_PROP, coll.getName(),
@@ -163,20 +167,47 @@ public class MoveReplicaCmd implements Cmd{
         CoreAdminParams.NAME, newCoreName);
     if(async!=null) addReplicasProps.getProperties().put(ASYNC, async);
     NamedList addResult = new NamedList();
+    CountDownLatch countDownLatch = new CountDownLatch(1);
+    ReplaceNodeCmd.RecoveryWatcher watcher = null;
+    if (replica.equals(slice.getLeader())) {
+      watcher = new ReplaceNodeCmd.RecoveryWatcher(coll.getName(), slice.getName(),
+          replica.getName(), null, countDownLatch);
+      ocmh.zkStateReader.registerCollectionStateWatcher(coll.getName(), watcher);
+    }
     ocmh.addReplica(clusterState, addReplicasProps, addResult, null);
     if (addResult.get("failure") != null) {
       String errorString = String.format(Locale.ROOT, "Failed to create replica for collection=%s shard=%s" +
           " on node=%s", coll.getName(), slice.getName(), targetNode);
       log.warn(errorString);
       results.add("failure", errorString);
+      if (watcher != null) { // unregister
+        ocmh.zkStateReader.registerCollectionStateWatcher(coll.getName(), watcher);
+      }
       return;
     }
+    // wait for the other replica to be active if the source replica was a leader
+    if (watcher != null) {
+      try {
+        log.debug("Waiting for leader's replica to recover.");
+        if (!countDownLatch.await(timeout, TimeUnit.SECONDS)) {
+          String errorString = String.format(Locale.ROOT, "Timed out waiting for leader's replica to recover, collection=%s shard=%s" +
+              " on node=%s", coll.getName(), slice.getName(), targetNode);
+          log.warn(errorString);
+          results.add("failure", errorString);
+          return;
+        } else {
+          log.debug("Replica " + watcher.getRecoveredReplica() + " is active - deleting the source...");
+        }
+      } finally {
+        ocmh.zkStateReader.removeCollectionStateWatcher(coll.getName(), watcher);
+      }
+    }
 
     ZkNodeProps removeReplicasProps = new ZkNodeProps(
         COLLECTION_PROP, coll.getName(),
         SHARD_ID_PROP, slice.getName(),
         REPLICA_PROP, replica.getName());
-    if(async!=null) removeReplicasProps.getProperties().put(ASYNC, async);
+    if (async != null) removeReplicasProps.getProperties().put(ASYNC, async);
     NamedList deleteResult = new NamedList();
     ocmh.deleteReplica(clusterState, removeReplicasProps, deleteResult, null);
     if (deleteResult.get("failure") != null) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bc37e8b4/solr/core/src/java/org/apache/solr/cloud/OverseerCollectionMessageHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/OverseerCollectionMessageHandler.java b/solr/core/src/java/org/apache/solr/cloud/OverseerCollectionMessageHandler.java
index a8d74e8..2c55f3c 100644
--- a/solr/core/src/java/org/apache/solr/cloud/OverseerCollectionMessageHandler.java
+++ b/solr/core/src/java/org/apache/solr/cloud/OverseerCollectionMessageHandler.java
@@ -419,20 +419,25 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler
 
   boolean waitForCoreNodeGone(String collectionName, String shard, String replicaName, int timeoutms) throws InterruptedException {
     TimeOut timeout = new TimeOut(timeoutms, TimeUnit.MILLISECONDS);
-    boolean deleted = false;
-    while (! timeout.hasTimedOut()) {
-      Thread.sleep(100);
-      DocCollection docCollection = zkStateReader.getClusterState().getCollection(collectionName);
-      if(docCollection != null) {
+    // TODO: remove this workaround for SOLR-9440
+    zkStateReader.registerCore(collectionName);
+    try {
+      while (! timeout.hasTimedOut()) {
+        Thread.sleep(100);
+        DocCollection docCollection = zkStateReader.getClusterState().getCollection(collectionName);
+        if (docCollection == null) { // someone already deleted the collection
+          return true;
+        }
         Slice slice = docCollection.getSlice(shard);
         if(slice == null || slice.getReplica(replicaName) == null) {
-          deleted =  true;
+          return true;
         }
       }
-      // Return true if either someone already deleted the collection/slice/replica.
-      if (docCollection == null || deleted) break;
+      // replica still exists after the timeout
+      return false;
+    } finally {
+      zkStateReader.unregisterCore(collectionName);
     }
-    return deleted;
   }
 
   void deleteCoreNode(String collectionName, String replicaName, Replica replica, String core) throws KeeperException, InterruptedException {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bc37e8b4/solr/core/src/java/org/apache/solr/cloud/ReplaceNodeCmd.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/ReplaceNodeCmd.java b/solr/core/src/java/org/apache/solr/cloud/ReplaceNodeCmd.java
index 5adbe8c..ba60908 100644
--- a/solr/core/src/java/org/apache/solr/cloud/ReplaceNodeCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/ReplaceNodeCmd.java
@@ -93,15 +93,6 @@ public class ReplaceNodeCmd implements OverseerCollectionMessageHandler.Cmd {
     CountDownLatch replicasToRecover = new CountDownLatch(numLeaders);
 
     for (ZkNodeProps sourceReplica : sourceReplicas) {
-      if (sourceReplica.getBool(ZkStateReader.LEADER_PROP, false)) {
-        String shardName = sourceReplica.getStr(SHARD_ID_PROP);
-        String replicaName = sourceReplica.getStr(ZkStateReader.REPLICA_PROP);
-        String collectionName = sourceReplica.getStr(COLLECTION_PROP);
-        String key = collectionName + "_" + replicaName;
-        RecoveryWatcher watcher = new RecoveryWatcher(collectionName, shardName, replicaName, replicasToRecover);
-        watchers.put(key, watcher);
-        zkStateReader.registerCollectionStateWatcher(collectionName, watcher);
-      }
       NamedList nl = new NamedList();
       log.info("Going to create replica for collection={} shard={} on node={}", sourceReplica.getStr(COLLECTION_PROP), sourceReplica.getStr(SHARD_ID_PROP), target);
       ZkNodeProps msg = sourceReplica.plus("parallel", String.valueOf(parallel)).plus(CoreAdminParams.NODE, target);
@@ -128,6 +119,16 @@ public class ReplaceNodeCmd implements OverseerCollectionMessageHandler.Cmd {
 
       if (addedReplica != null) {
         createdReplicas.add(addedReplica);
+        if (sourceReplica.getBool(ZkStateReader.LEADER_PROP, false)) {
+          String shardName = sourceReplica.getStr(SHARD_ID_PROP);
+          String replicaName = sourceReplica.getStr(ZkStateReader.REPLICA_PROP);
+          String collectionName = sourceReplica.getStr(COLLECTION_PROP);
+          String key = collectionName + "_" + replicaName;
+          RecoveryWatcher watcher = new RecoveryWatcher(collectionName, shardName, replicaName,
+              addedReplica.getStr(ZkStateReader.CORE_NAME_PROP), replicasToRecover);
+          watchers.put(key, watcher);
+          zkStateReader.registerCollectionStateWatcher(collectionName, watcher);
+        }
       }
     }
 
@@ -208,16 +209,27 @@ public class ReplaceNodeCmd implements OverseerCollectionMessageHandler.Cmd {
   }
 
   // we use this watcher to wait for replicas to recover
-  private static class RecoveryWatcher implements CollectionStateWatcher {
+  static class RecoveryWatcher implements CollectionStateWatcher {
     String collectionId;
     String shardId;
     String replicaId;
+    String targetCore;
     CountDownLatch countDownLatch;
+    Replica recovered;
 
-    RecoveryWatcher(String collectionId, String shardId, String replicaId, CountDownLatch countDownLatch) {
+    /**
+     * Watch for recovery of a replica
+     * @param collectionId collection name
+     * @param shardId shard id
+     * @param replicaId source replica name (coreNodeName)
+     * @param targetCore specific target core name - if null then any active replica will do
+     * @param countDownLatch countdown when recovered
+     */
+    RecoveryWatcher(String collectionId, String shardId, String replicaId, String targetCore, CountDownLatch countDownLatch) {
       this.collectionId = collectionId;
       this.shardId = shardId;
       this.replicaId = replicaId;
+      this.targetCore = targetCore;
       this.countDownLatch = countDownLatch;
     }
 
@@ -241,7 +253,12 @@ public class ReplaceNodeCmd implements OverseerCollectionMessageHandler.Cmd {
             continue;
           }
           // check its state
+          String coreName = replica.getStr(ZkStateReader.CORE_NAME_PROP);
+          if (targetCore != null && !targetCore.equals(coreName)) {
+            continue;
+          }
           if (replica.isActive(liveNodes)) { // recovered - stop waiting
+            recovered = replica;
             countDownLatch.countDown();
             return true;
           }
@@ -250,5 +267,9 @@ public class ReplaceNodeCmd implements OverseerCollectionMessageHandler.Cmd {
       // set the watch again to wait for the new replica to recover
       return false;
     }
+
+    public Replica getRecoveredReplica() {
+      return recovered;
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bc37e8b4/solr/core/src/test/org/apache/solr/cloud/MoveReplicaHDFSTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/MoveReplicaHDFSTest.java b/solr/core/src/test/org/apache/solr/cloud/MoveReplicaHDFSTest.java
new file mode 100644
index 0000000..884d49e
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/cloud/MoveReplicaHDFSTest.java
@@ -0,0 +1,53 @@
+package org.apache.solr.cloud;
+
+import com.carrotsearch.randomizedtesting.ThreadFilter;
+import com.carrotsearch.randomizedtesting.annotations.ThreadLeakFilters;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.solr.cloud.hdfs.HdfsTestUtil;
+import org.apache.solr.common.cloud.ZkConfigManager;
+import org.apache.solr.util.BadHdfsThreadsFilter;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+
+/**
+ *
+ */
+@ThreadLeakFilters(defaultFilters = true, filters = {
+    BadHdfsThreadsFilter.class, // hdfs currently leaks thread(s)
+    MoveReplicaHDFSTest.ForkJoinThreadsFilter.class
+})
+public class MoveReplicaHDFSTest extends MoveReplicaTest {
+
+  private static MiniDFSCluster dfsCluster;
+
+  @BeforeClass
+  public static void setupClass() throws Exception {
+    dfsCluster = HdfsTestUtil.setupClass(createTempDir().toFile().getAbsolutePath());
+
+    ZkConfigManager configManager = new ZkConfigManager(zkClient());
+    configManager.uploadConfigDir(configset("cloud-hdfs"), "conf1");
+
+    System.setProperty("solr.hdfs.home", HdfsTestUtil.getDataDir(dfsCluster, "data"));
+  }
+
+  @AfterClass
+  public static void teardownClass() throws Exception {
+    cluster.shutdown(); // need to close before the MiniDFSCluster
+    HdfsTestUtil.teardownClass(dfsCluster);
+    dfsCluster = null;
+  }
+
+
+  public static class ForkJoinThreadsFilter implements ThreadFilter {
+
+    @Override
+    public boolean reject(Thread t) {
+      String name = t.getName();
+      if (name.startsWith("ForkJoinPool.commonPool")) {
+        return true;
+      }
+      return false;
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bc37e8b4/solr/core/src/test/org/apache/solr/cloud/MoveReplicaTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/MoveReplicaTest.java b/solr/core/src/test/org/apache/solr/cloud/MoveReplicaTest.java
index 4368fea..8f00431 100644
--- a/solr/core/src/test/org/apache/solr/cloud/MoveReplicaTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/MoveReplicaTest.java
@@ -20,6 +20,7 @@ package org.apache.solr.cloud;
 import java.io.IOException;
 import java.lang.invoke.MethodHandles;
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.Collections;
 import java.util.List;
 import java.util.Set;
@@ -31,8 +32,10 @@ import org.apache.solr.client.solrj.request.CollectionAdminRequest;
 import org.apache.solr.client.solrj.request.CoreAdminRequest;
 import org.apache.solr.client.solrj.response.CoreAdminResponse;
 import org.apache.solr.client.solrj.response.RequestStatusState;
+import org.apache.solr.common.cloud.DocCollection;
 import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.common.cloud.Slice;
+import org.apache.solr.common.cloud.ZkStateReader;
 import org.junit.BeforeClass;
 import org.junit.Test;
 import org.slf4j.Logger;
@@ -40,6 +43,7 @@ import org.slf4j.LoggerFactory;
 
 public class MoveReplicaTest extends SolrCloudTestCase {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
   @BeforeClass
   public static void setupCluster() throws Exception {
     configureCluster(4)
@@ -56,10 +60,11 @@ public class MoveReplicaTest extends SolrCloudTestCase {
     cluster.waitForAllNodes(5000);
     String coll = "movereplicatest_coll";
     log.info("total_jettys: " + cluster.getJettySolrRunners().size());
+    int REPLICATION = 2;
 
     CloudSolrClient cloudClient = cluster.getSolrClient();
 
-    CollectionAdminRequest.Create create = CollectionAdminRequest.createCollection(coll, "conf1", 2, 2);
+    CollectionAdminRequest.Create create = CollectionAdminRequest.createCollection(coll, "conf1", 2, REPLICATION);
     create.setMaxShardsPerNode(2);
     cloudClient.request(create);
 
@@ -94,16 +99,87 @@ public class MoveReplicaTest extends SolrCloudTestCase {
         break;
       }
       assertFalse(rsp.getRequestStatus() == RequestStatusState.FAILED);
-      Thread.sleep(50);
+      Thread.sleep(500);
     }
     assertTrue(success);
     checkNumOfCores(cloudClient, replica.getNodeName(), 0);
-    checkNumOfCores(cloudClient, targetNode, 2);
+    assertTrue("should be at least one core on target node!", getNumOfCores(cloudClient, targetNode) > 0);
+    // wait for recovery
+    boolean recovered = false;
+    for (int i = 0; i < 300; i++) {
+      DocCollection collState = getCollectionState(coll);
+      log.debug("###### " + collState);
+      Collection<Replica> replicas = collState.getSlice(shardId).getReplicas();
+      boolean allActive = true;
+      boolean hasLeaders = true;
+      if (replicas != null && !replicas.isEmpty()) {
+        for (Replica r : replicas) {
+          if (!r.getNodeName().equals(targetNode)) {
+            continue;
+          }
+          if (!r.isActive(Collections.singleton(targetNode))) {
+            log.info("Not active: " + r);
+            allActive = false;
+          }
+        }
+      } else {
+        allActive = false;
+      }
+      for (Slice slice : collState.getSlices()) {
+        if (slice.getLeader() == null) {
+          hasLeaders = false;
+        }
+      }
+      if (allActive && hasLeaders) {
+        // check the number of active replicas
+        assertEquals("total number of replicas", REPLICATION, replicas.size());
+        recovered = true;
+        break;
+      } else {
+        log.info("--- waiting, allActive=" + allActive + ", hasLeaders=" + hasLeaders);
+        Thread.sleep(1000);
+      }
+    }
+    assertTrue("replica never fully recovered", recovered);
 
     moveReplica = new CollectionAdminRequest.MoveReplica(coll, shardId, targetNode, replica.getNodeName());
     moveReplica.process(cloudClient);
     checkNumOfCores(cloudClient, replica.getNodeName(), 1);
-    checkNumOfCores(cloudClient, targetNode, 1);
+    // wait for recovery
+    recovered = false;
+    for (int i = 0; i < 300; i++) {
+      DocCollection collState = getCollectionState(coll);
+      log.debug("###### " + collState);
+      Collection<Replica> replicas = collState.getSlice(shardId).getReplicas();
+      boolean allActive = true;
+      boolean hasLeaders = true;
+      if (replicas != null && !replicas.isEmpty()) {
+        for (Replica r : replicas) {
+          if (!r.getNodeName().equals(replica.getNodeName())) {
+            continue;
+          }
+          if (!r.isActive(Collections.singleton(replica.getNodeName()))) {
+            log.info("Not active yet: " + r);
+            allActive = false;
+          }
+        }
+      } else {
+        allActive = false;
+      }
+      for (Slice slice : collState.getSlices()) {
+        if (slice.getLeader() == null) {
+          hasLeaders = false;
+        }
+      }
+      if (allActive && hasLeaders) {
+        assertEquals("total number of replicas", REPLICATION, replicas.size());
+        recovered = true;
+        break;
+      } else {
+        Thread.sleep(1000);
+      }
+    }
+    assertTrue("replica never fully recovered", recovered);
   }
 
   private Replica getRandomReplica(String coll, CloudSolrClient cloudClient) {


[13/23] lucene-solr:jira/solr-10879: LUCENE-7871: fixing CHANGES.txt, mark it as Lucene 7.0 bug fix.

Posted by ab...@apache.org.
LUCENE-7871: fixing CHANGES.txt, mark it as Lucene 7.0 bug fix.


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

Branch: refs/heads/jira/solr-10879
Commit: bb2d6c128ff74d6164c5c60ac952074c1b5a5b94
Parents: 5966f75
Author: Mikhail Khludnev <mk...@apache.org>
Authored: Mon Jul 3 15:53:57 2017 +0300
Committer: Mikhail Khludnev <mk...@apache.org>
Committed: Mon Jul 3 15:53:57 2017 +0300

----------------------------------------------------------------------
 lucene/CHANGES.txt                                                | 3 +++
 .../src/java/org/apache/lucene/search/join/BlockJoinSelector.java | 2 +-
 solr/CHANGES.txt                                                  | 3 ---
 3 files changed, 4 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bb2d6c12/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 909b6ce..88e1779 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -105,6 +105,9 @@ Bug Fixes
 * LUCENE-7859: Spatial-extras PackedQuadPrefixTree bug that only revealed itself
   with the new pointsOnly optimizations in LUCENE-7845. (David Smiley)
 
+* LUCENE-7871: fix false positive match in BlockJoinSelector when children have no value, introducing 
+  wrap methods accepting children as DISI. Extracting ToParentDocValues (Mikhail Khludnev)
+
 Improvements
 
 * LUCENE-7489: Better storage of sparse doc-values fields with the default

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bb2d6c12/lucene/join/src/java/org/apache/lucene/search/join/BlockJoinSelector.java
----------------------------------------------------------------------
diff --git a/lucene/join/src/java/org/apache/lucene/search/join/BlockJoinSelector.java b/lucene/join/src/java/org/apache/lucene/search/join/BlockJoinSelector.java
index a81230d..79c35b8 100644
--- a/lucene/join/src/java/org/apache/lucene/search/join/BlockJoinSelector.java
+++ b/lucene/join/src/java/org/apache/lucene/search/join/BlockJoinSelector.java
@@ -126,7 +126,7 @@ public class BlockJoinSelector {
     return wrap(sortedNumerics, selection, parents, toIter(children));
   }
 
-  /** creates an interator for the given bitset */
+  /** creates an iterator for the given bitset */
   protected static BitSetIterator toIter(BitSet children) {
     return new BitSetIterator(children, 0);
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bb2d6c12/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 8c06107..aadc6db 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -501,9 +501,6 @@ when using one of Exact*StatsCache (Mikhail Khludnev)
 * SOLR-10910: Clean up a few details left over from pluggable transient core and untangling
   CoreDescriptor/CoreContainer references (Erick Erickson)
 
-* LUCENE-7871: fix false positive match in BlockJoinSelector when children have no value, introducing 
-  wrap methods accepting children as DISI. Extracting ToParentDocValues (Mikhail Khludnev)
-
 Optimizations
 ----------------------
 * SOLR-10634: JSON Facet API: When a field/terms facet will retrieve all buckets (i.e. limit:-1)


[06/23] lucene-solr:jira/solr-10879: Changes for version upgrade to 8.0 on master for the 7.0 release

Posted by ab...@apache.org.
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/189f2b3f/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene54/Lucene54DocValuesProducer.java
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene54/Lucene54DocValuesProducer.java b/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene54/Lucene54DocValuesProducer.java
deleted file mode 100644
index a35f503..0000000
--- a/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene54/Lucene54DocValuesProducer.java
+++ /dev/null
@@ -1,1803 +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.lucene54;
-
-import java.io.Closeable;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.atomic.AtomicLong;
-
-import org.apache.lucene.codecs.CodecUtil;
-import org.apache.lucene.codecs.DocValuesProducer;
-import org.apache.lucene.codecs.lucene54.Lucene54DocValuesConsumer.NumberType;
-import org.apache.lucene.index.*;
-import org.apache.lucene.index.NumericDocValues;
-import org.apache.lucene.store.ChecksumIndexInput;
-import org.apache.lucene.store.IndexInput;
-import org.apache.lucene.store.RandomAccessInput;
-import org.apache.lucene.util.Accountable;
-import org.apache.lucene.util.Accountables;
-import org.apache.lucene.util.Bits;
-import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.IOUtils;
-import org.apache.lucene.util.LongValues;
-import org.apache.lucene.util.PagedBytes;
-import org.apache.lucene.util.RamUsageEstimator;
-import org.apache.lucene.util.packed.DirectMonotonicReader;
-import org.apache.lucene.util.packed.DirectReader;
-import org.apache.lucene.util.packed.MonotonicBlockPackedReader;
-
-import static org.apache.lucene.codecs.lucene54.Lucene54DocValuesFormat.*;
-
-/** reader for {@link Lucene54DocValuesFormat} */
-final class Lucene54DocValuesProducer extends DocValuesProducer implements Closeable {
-  private final Map<String,NumericEntry> numerics = new HashMap<>();
-  private final Map<String,BinaryEntry> binaries = new HashMap<>();
-  private final Map<String,SortedSetEntry> sortedSets = new HashMap<>();
-  private final Map<String,SortedSetEntry> sortedNumerics = new HashMap<>();
-  private final Map<String,NumericEntry> ords = new HashMap<>();
-  private final Map<String,NumericEntry> ordIndexes = new HashMap<>();
-  private final int numFields;
-  private final AtomicLong ramBytesUsed;
-  private final IndexInput data;
-  private final int maxDoc;
-
-  // memory-resident structures
-  private final Map<String,MonotonicBlockPackedReader> addressInstances = new HashMap<>();
-  private final Map<String,ReverseTermsIndex> reverseIndexInstances = new HashMap<>();
-  private final Map<String,DirectMonotonicReader.Meta> directAddressesMeta = new HashMap<>();
-
-  private final boolean merging;
-
-  // clone for merge: when merging we don't do any instances.put()s
-  Lucene54DocValuesProducer(Lucene54DocValuesProducer original) throws IOException {
-    assert Thread.holdsLock(original);
-    numerics.putAll(original.numerics);
-    binaries.putAll(original.binaries);
-    sortedSets.putAll(original.sortedSets);
-    sortedNumerics.putAll(original.sortedNumerics);
-    ords.putAll(original.ords);
-    ordIndexes.putAll(original.ordIndexes);
-    numFields = original.numFields;
-    ramBytesUsed = new AtomicLong(original.ramBytesUsed.get());
-    data = original.data.clone();
-    maxDoc = original.maxDoc;
-
-    addressInstances.putAll(original.addressInstances);
-    reverseIndexInstances.putAll(original.reverseIndexInstances);
-    merging = true;
-  }
-
-  /** expert: instantiates a new reader */
-  Lucene54DocValuesProducer(SegmentReadState state, String dataCodec, String dataExtension, String metaCodec, String metaExtension) throws IOException {
-    String metaName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, metaExtension);
-    this.maxDoc = state.segmentInfo.maxDoc();
-    merging = false;
-    ramBytesUsed = new AtomicLong(RamUsageEstimator.shallowSizeOfInstance(getClass()));
-
-    int version = -1;
-    int numFields = -1;
-
-    // read in the entries from the metadata file.
-    try (ChecksumIndexInput in = state.directory.openChecksumInput(metaName, state.context)) {
-      Throwable priorE = null;
-      try {
-        version = CodecUtil.checkIndexHeader(in, metaCodec,
-                                        Lucene54DocValuesFormat.VERSION_START,
-                                        Lucene54DocValuesFormat.VERSION_CURRENT,
-                                        state.segmentInfo.getId(),
-                                        state.segmentSuffix);
-        numFields = readFields(in, state.fieldInfos);
-      } catch (Throwable exception) {
-        priorE = exception;
-      } finally {
-        CodecUtil.checkFooter(in, priorE);
-      }
-    }
-
-    this.numFields = numFields;
-    String dataName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, dataExtension);
-    this.data = state.directory.openInput(dataName, state.context);
-    boolean success = false;
-    try {
-      final int version2 = CodecUtil.checkIndexHeader(data, dataCodec,
-                                                 Lucene54DocValuesFormat.VERSION_START,
-                                                 Lucene54DocValuesFormat.VERSION_CURRENT,
-                                                 state.segmentInfo.getId(),
-                                                 state.segmentSuffix);
-      if (version != version2) {
-        throw new CorruptIndexException("Format versions mismatch: meta=" + version + ", data=" + version2, data);
-      }
-
-      // NOTE: data file is too costly to verify checksum against all the bytes on open,
-      // but for now we at least verify proper structure of the checksum footer: which looks
-      // for FOOTER_MAGIC + algorithmID. This is cheap and can detect some forms of corruption
-      // such as file truncation.
-      CodecUtil.retrieveChecksum(data);
-
-      success = true;
-    } finally {
-      if (!success) {
-        IOUtils.closeWhileHandlingException(this.data);
-      }
-    }
-  }
-
-  private void readSortedField(FieldInfo info, IndexInput meta) throws IOException {
-    // sorted = binary + numeric
-    if (meta.readVInt() != info.number) {
-      throw new CorruptIndexException("sorted entry for field: " + info.name + " is corrupt", meta);
-    }
-    if (meta.readByte() != Lucene54DocValuesFormat.BINARY) {
-      throw new CorruptIndexException("sorted entry for field: " + info.name + " is corrupt", meta);
-    }
-    BinaryEntry b = readBinaryEntry(info, meta);
-    binaries.put(info.name, b);
-
-    if (meta.readVInt() != info.number) {
-      throw new CorruptIndexException("sorted entry for field: " + info.name + " is corrupt", meta);
-    }
-    if (meta.readByte() != Lucene54DocValuesFormat.NUMERIC) {
-      throw new CorruptIndexException("sorted entry for field: " + info.name + " is corrupt", meta);
-    }
-    NumericEntry n = readNumericEntry(info, meta);
-    ords.put(info.name, n);
-  }
-
-  private void readSortedSetFieldWithAddresses(FieldInfo info, IndexInput meta) throws IOException {
-    // sortedset = binary + numeric (addresses) + ordIndex
-    if (meta.readVInt() != info.number) {
-      throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt", meta);
-    }
-    if (meta.readByte() != Lucene54DocValuesFormat.BINARY) {
-      throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt", meta);
-    }
-    BinaryEntry b = readBinaryEntry(info, meta);
-    binaries.put(info.name, b);
-
-    if (meta.readVInt() != info.number) {
-      throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt", meta);
-    }
-    if (meta.readByte() != Lucene54DocValuesFormat.NUMERIC) {
-      throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt", meta);
-    }
-    NumericEntry n1 = readNumericEntry(info, meta);
-    ords.put(info.name, n1);
-
-    if (meta.readVInt() != info.number) {
-      throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt", meta);
-    }
-    if (meta.readByte() != Lucene54DocValuesFormat.NUMERIC) {
-      throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt", meta);
-    }
-    NumericEntry n2 = readNumericEntry(info, meta);
-    ordIndexes.put(info.name, n2);
-  }
-
-  private void readSortedSetFieldWithTable(FieldInfo info, IndexInput meta) throws IOException {
-    // sortedset table = binary + ordset table + ordset index
-    if (meta.readVInt() != info.number) {
-      throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt", meta);
-    }
-    if (meta.readByte() != Lucene54DocValuesFormat.BINARY) {
-      throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt", meta);
-    }
-
-    BinaryEntry b = readBinaryEntry(info, meta);
-    binaries.put(info.name, b);
-
-    if (meta.readVInt() != info.number) {
-      throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt", meta);
-    }
-    if (meta.readByte() != Lucene54DocValuesFormat.NUMERIC) {
-      throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt", meta);
-    }
-    NumericEntry n = readNumericEntry(info, meta);
-    ords.put(info.name, n);
-  }
-
-  private int readFields(IndexInput meta, FieldInfos infos) throws IOException {
-    int numFields = 0;
-    int fieldNumber = meta.readVInt();
-    while (fieldNumber != -1) {
-      numFields++;
-      FieldInfo info = infos.fieldInfo(fieldNumber);
-      if (info == null) {
-        // trickier to validate more: because we use multiple entries for "composite" types like sortedset, etc.
-        throw new CorruptIndexException("Invalid field number: " + fieldNumber, meta);
-      }
-      byte type = meta.readByte();
-      if (type == Lucene54DocValuesFormat.NUMERIC) {
-        numerics.put(info.name, readNumericEntry(info, meta));
-      } else if (type == Lucene54DocValuesFormat.BINARY) {
-        BinaryEntry b = readBinaryEntry(info, meta);
-        binaries.put(info.name, b);
-      } else if (type == Lucene54DocValuesFormat.SORTED) {
-        readSortedField(info, meta);
-      } else if (type == Lucene54DocValuesFormat.SORTED_SET) {
-        SortedSetEntry ss = readSortedSetEntry(meta);
-        sortedSets.put(info.name, ss);
-        if (ss.format == SORTED_WITH_ADDRESSES) {
-          readSortedSetFieldWithAddresses(info, meta);
-        } else if (ss.format == SORTED_SET_TABLE) {
-          readSortedSetFieldWithTable(info, meta);
-        } else if (ss.format == SORTED_SINGLE_VALUED) {
-          if (meta.readVInt() != fieldNumber) {
-            throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt", meta);
-          }
-          if (meta.readByte() != Lucene54DocValuesFormat.SORTED) {
-            throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt", meta);
-          }
-          readSortedField(info, meta);
-        } else {
-          throw new AssertionError();
-        }
-      } else if (type == Lucene54DocValuesFormat.SORTED_NUMERIC) {
-        SortedSetEntry ss = readSortedSetEntry(meta);
-        sortedNumerics.put(info.name, ss);
-        if (ss.format == SORTED_WITH_ADDRESSES) {
-          if (meta.readVInt() != fieldNumber) {
-            throw new CorruptIndexException("sortednumeric entry for field: " + info.name + " is corrupt", meta);
-          }
-          if (meta.readByte() != Lucene54DocValuesFormat.NUMERIC) {
-            throw new CorruptIndexException("sortednumeric entry for field: " + info.name + " is corrupt", meta);
-          }
-          numerics.put(info.name, readNumericEntry(info, meta));
-          if (meta.readVInt() != fieldNumber) {
-            throw new CorruptIndexException("sortednumeric entry for field: " + info.name + " is corrupt", meta);
-          }
-          if (meta.readByte() != Lucene54DocValuesFormat.NUMERIC) {
-            throw new CorruptIndexException("sortednumeric entry for field: " + info.name + " is corrupt", meta);
-          }
-          NumericEntry ordIndex = readNumericEntry(info, meta);
-          ordIndexes.put(info.name, ordIndex);
-        } else if (ss.format == SORTED_SET_TABLE) {
-          if (meta.readVInt() != info.number) {
-            throw new CorruptIndexException("sortednumeric entry for field: " + info.name + " is corrupt", meta);
-          }
-          if (meta.readByte() != Lucene54DocValuesFormat.NUMERIC) {
-            throw new CorruptIndexException("sortednumeric entry for field: " + info.name + " is corrupt", meta);
-          }
-          NumericEntry n = readNumericEntry(info, meta);
-          ords.put(info.name, n);
-        } else if (ss.format == SORTED_SINGLE_VALUED) {
-          if (meta.readVInt() != fieldNumber) {
-            throw new CorruptIndexException("sortednumeric entry for field: " + info.name + " is corrupt", meta);
-          }
-          if (meta.readByte() != Lucene54DocValuesFormat.NUMERIC) {
-            throw new CorruptIndexException("sortednumeric entry for field: " + info.name + " is corrupt", meta);
-          }
-          numerics.put(info.name, readNumericEntry(info, meta));
-        } else {
-          throw new AssertionError();
-        }
-      } else {
-        throw new CorruptIndexException("invalid type: " + type, meta);
-      }
-      fieldNumber = meta.readVInt();
-    }
-    return numFields;
-  }
-
-  private NumericEntry readNumericEntry(FieldInfo info, IndexInput meta) throws IOException {
-    NumericEntry entry = new NumericEntry();
-    entry.format = meta.readVInt();
-    entry.missingOffset = meta.readLong();
-    if (entry.format == SPARSE_COMPRESSED) {
-      // sparse bits need a bit more metadata
-      entry.numDocsWithValue = meta.readVLong();
-      final int blockShift = meta.readVInt();
-      entry.monotonicMeta = DirectMonotonicReader.loadMeta(meta, entry.numDocsWithValue, blockShift);
-      ramBytesUsed.addAndGet(entry.monotonicMeta.ramBytesUsed());
-      directAddressesMeta.put(info.name, entry.monotonicMeta);
-    }
-    entry.offset = meta.readLong();
-    entry.count = meta.readVLong();
-    switch(entry.format) {
-      case CONST_COMPRESSED:
-        entry.minValue = meta.readLong();
-        if (entry.count > Integer.MAX_VALUE) {
-          // currently just a limitation e.g. of bits interface and so on.
-          throw new CorruptIndexException("illegal CONST_COMPRESSED count: " + entry.count, meta);
-        }
-        break;
-      case GCD_COMPRESSED:
-        entry.minValue = meta.readLong();
-        entry.gcd = meta.readLong();
-        entry.bitsPerValue = meta.readVInt();
-        break;
-      case TABLE_COMPRESSED:
-        final int uniqueValues = meta.readVInt();
-        if (uniqueValues > 256) {
-          throw new CorruptIndexException("TABLE_COMPRESSED cannot have more than 256 distinct values, got=" + uniqueValues, meta);
-        }
-        entry.table = new long[uniqueValues];
-        for (int i = 0; i < uniqueValues; ++i) {
-          entry.table[i] = meta.readLong();
-        }
-        ramBytesUsed.addAndGet(RamUsageEstimator.sizeOf(entry.table));
-        entry.bitsPerValue = meta.readVInt();
-        break;
-      case DELTA_COMPRESSED:
-        entry.minValue = meta.readLong();
-        entry.bitsPerValue = meta.readVInt();
-        break;
-      case MONOTONIC_COMPRESSED:
-        final int blockShift = meta.readVInt();
-        entry.monotonicMeta = DirectMonotonicReader.loadMeta(meta, maxDoc + 1, blockShift);
-        ramBytesUsed.addAndGet(entry.monotonicMeta.ramBytesUsed());
-        directAddressesMeta.put(info.name, entry.monotonicMeta);
-        break;
-      case SPARSE_COMPRESSED:
-        final byte numberType = meta.readByte();
-        switch (numberType) {
-          case 0:
-            entry.numberType = NumberType.VALUE;
-            break;
-          case 1:
-            entry.numberType = NumberType.ORDINAL;
-            break;
-          default:
-            throw new CorruptIndexException("Number type can only be 0 or 1, got=" + numberType, meta);
-        }
-
-        // now read the numeric entry for non-missing values
-        final int fieldNumber = meta.readVInt();
-        if (fieldNumber != info.number) {
-          throw new CorruptIndexException("Field numbers mistmatch: " + fieldNumber + " != " + info.number, meta);
-        }
-        final int dvFormat = meta.readByte();
-        if (dvFormat != NUMERIC) {
-          throw new CorruptIndexException("Formats mistmatch: " + dvFormat + " != " + NUMERIC, meta);
-        }
-        entry.nonMissingValues = readNumericEntry(info, meta);
-        break;
-      default:
-        throw new CorruptIndexException("Unknown format: " + entry.format + ", input=", meta);
-    }
-    entry.endOffset = meta.readLong();
-    return entry;
-  }
-
-  private BinaryEntry readBinaryEntry(FieldInfo info, IndexInput meta) throws IOException {
-    BinaryEntry entry = new BinaryEntry();
-    entry.format = meta.readVInt();
-    entry.missingOffset = meta.readLong();
-    entry.minLength = meta.readVInt();
-    entry.maxLength = meta.readVInt();
-    entry.count = meta.readVLong();
-    entry.offset = meta.readLong();
-    switch(entry.format) {
-      case BINARY_FIXED_UNCOMPRESSED:
-        break;
-      case BINARY_PREFIX_COMPRESSED:
-        entry.addressesOffset = meta.readLong();
-        entry.packedIntsVersion = meta.readVInt();
-        entry.blockSize = meta.readVInt();
-        entry.reverseIndexOffset = meta.readLong();
-        break;
-      case BINARY_VARIABLE_UNCOMPRESSED:
-        entry.addressesOffset = meta.readLong();
-        final int blockShift = meta.readVInt();
-        entry.addressesMeta = DirectMonotonicReader.loadMeta(meta, entry.count + 1, blockShift);
-        ramBytesUsed.addAndGet(entry.addressesMeta.ramBytesUsed());
-        directAddressesMeta.put(info.name, entry.addressesMeta);
-        entry.addressesEndOffset = meta.readLong();
-        break;
-      default:
-        throw new CorruptIndexException("Unknown format: " + entry.format, meta);
-    }
-    return entry;
-  }
-
-  SortedSetEntry readSortedSetEntry(IndexInput meta) throws IOException {
-    SortedSetEntry entry = new SortedSetEntry();
-    entry.format = meta.readVInt();
-    if (entry.format == SORTED_SET_TABLE) {
-      final int totalTableLength = meta.readInt();
-      if (totalTableLength > 256) {
-        throw new CorruptIndexException("SORTED_SET_TABLE cannot have more than 256 values in its dictionary, got=" + totalTableLength, meta);
-      }
-      entry.table = new long[totalTableLength];
-      for (int i = 0; i < totalTableLength; ++i) {
-        entry.table[i] = meta.readLong();
-      }
-      ramBytesUsed.addAndGet(RamUsageEstimator.sizeOf(entry.table));
-      final int tableSize = meta.readInt();
-      if (tableSize > totalTableLength + 1) { // +1 because of the empty set
-        throw new CorruptIndexException("SORTED_SET_TABLE cannot have more set ids than ords in its dictionary, got " + totalTableLength + " ords and " + tableSize + " sets", meta);
-      }
-      entry.tableOffsets = new int[tableSize + 1];
-      for (int i = 1; i < entry.tableOffsets.length; ++i) {
-        entry.tableOffsets[i] = entry.tableOffsets[i - 1] + meta.readInt();
-      }
-      ramBytesUsed.addAndGet(RamUsageEstimator.sizeOf(entry.tableOffsets));
-    } else if (entry.format != SORTED_SINGLE_VALUED && entry.format != SORTED_WITH_ADDRESSES) {
-      throw new CorruptIndexException("Unknown format: " + entry.format, meta);
-    }
-    return entry;
-  }
-
-  @Override
-  public NumericDocValues getNumeric(FieldInfo field) throws IOException {
-    NumericEntry entry = numerics.get(field.name);
-    Bits docsWithField;
-
-    if (entry.format == SPARSE_COMPRESSED) {
-      return getSparseNumericDocValues(entry);
-    } else {
-      if (entry.missingOffset == ALL_MISSING) {
-        return DocValues.emptyNumeric();
-      } else if (entry.missingOffset == ALL_LIVE) {
-        LongValues values = getNumeric(entry);
-        return new NumericDocValues() {
-          private int docID = -1;
-
-          @Override
-          public int docID() {
-            return docID;
-          }
-
-          @Override
-          public int nextDoc() {
-            docID++;
-            if (docID == maxDoc) {
-              docID = NO_MORE_DOCS;
-            }
-            return docID;
-          }
-
-          @Override
-          public int advance(int target) {
-            if (target >= maxDoc) {
-              docID = NO_MORE_DOCS;
-            } else {
-              docID = target;
-            }
-            return docID;
-          }
-
-          @Override
-          public boolean advanceExact(int target) throws IOException {
-            docID = target;
-            return true;
-          }
-
-          @Override
-          public long cost() {
-            // TODO
-            return 0;
-          }
-
-          @Override
-          public long longValue() {
-            return values.get(docID);
-          }
-        };
-      } else {
-        docsWithField = getLiveBits(entry.missingOffset, maxDoc);
-      }
-    }
-    final LongValues values = getNumeric(entry);
-    return new NumericDocValues() {
-
-      int doc = -1;
-      long value;
-
-      @Override
-      public long longValue() throws IOException {
-        return value;
-      }
-
-      @Override
-      public int docID() {
-        return doc;
-      }
-
-      @Override
-      public int nextDoc() throws IOException {
-        return advance(doc + 1);
-      }
-
-      @Override
-      public int advance(int target) throws IOException {
-        for (int doc = target; doc < maxDoc; ++doc) {
-          value = values.get(doc);
-          if (value != 0 || docsWithField.get(doc)) {
-            return this.doc = doc;
-          }
-        }
-        return doc = NO_MORE_DOCS;
-      }
-
-      @Override
-      public boolean advanceExact(int target) throws IOException {
-        doc = target;
-        value = values.get(doc);
-        return value != 0 || docsWithField.get(doc);
-      }
-
-      @Override
-      public long cost() {
-        return maxDoc;
-      }
-
-    };
-  }
-
-  @Override
-  public long ramBytesUsed() {
-    return ramBytesUsed.get();
-  }
-
-  @Override
-  public synchronized Collection<Accountable> getChildResources() {
-    List<Accountable> resources = new ArrayList<>();
-    resources.addAll(Accountables.namedAccountables("addresses field", addressInstances));
-    resources.addAll(Accountables.namedAccountables("reverse index field", reverseIndexInstances));
-    resources.addAll(Accountables.namedAccountables("direct addresses meta field", directAddressesMeta));
-    return Collections.unmodifiableList(resources);
-  }
-
-  @Override
-  public void checkIntegrity() throws IOException {
-    CodecUtil.checksumEntireFile(data);
-  }
-
-  @Override
-  public String toString() {
-    return getClass().getSimpleName() + "(fields=" + numFields + ")";
-  }
-
-  LongValues getNumeric(NumericEntry entry) throws IOException {
-    switch (entry.format) {
-      case CONST_COMPRESSED: {
-        final long constant = entry.minValue;
-        final Bits live = getLiveBits(entry.missingOffset, (int)entry.count);
-        return new LongValues() {
-          @Override
-          public long get(long index) {
-            return live.get((int)index) ? constant : 0;
-          }
-        };
-      }
-      case DELTA_COMPRESSED: {
-        RandomAccessInput slice = this.data.randomAccessSlice(entry.offset, entry.endOffset - entry.offset);
-        final long delta = entry.minValue;
-        final LongValues values = DirectReader.getInstance(slice, entry.bitsPerValue, 0);
-        return new LongValues() {
-          @Override
-          public long get(long id) {
-            return delta + values.get(id);
-          }
-        };
-      }
-      case GCD_COMPRESSED: {
-        RandomAccessInput slice = this.data.randomAccessSlice(entry.offset, entry.endOffset - entry.offset);
-        final long min = entry.minValue;
-        final long mult = entry.gcd;
-        final LongValues quotientReader = DirectReader.getInstance(slice, entry.bitsPerValue, 0);
-        return new LongValues() {
-          @Override
-          public long get(long id) {
-            return min + mult * quotientReader.get(id);
-          }
-        };
-      }
-      case TABLE_COMPRESSED: {
-        RandomAccessInput slice = this.data.randomAccessSlice(entry.offset, entry.endOffset - entry.offset);
-        final long table[] = entry.table;
-        final LongValues ords = DirectReader.getInstance(slice, entry.bitsPerValue, 0);
-        return new LongValues() {
-          @Override
-          public long get(long id) {
-            return table[(int) ords.get(id)];
-          }
-        };
-      }
-      case SPARSE_COMPRESSED:
-        final SparseNumericDocValues values = getSparseNumericDocValues(entry);
-        final long missingValue;
-        switch (entry.numberType) {
-          case ORDINAL:
-            missingValue = -1L;
-            break;
-          case VALUE:
-            missingValue = 0L;
-            break;
-          default:
-            throw new AssertionError();
-        }
-        return new SparseNumericDocValuesRandomAccessWrapper(values, missingValue);
-      default:
-        throw new AssertionError();
-    }
-  }
-
-  static final class SparseNumericDocValues extends NumericDocValues {
-
-    final int docIDsLength;
-    final LongValues docIds, values;
-
-    int index, doc;
-
-    SparseNumericDocValues(int docIDsLength, LongValues docIDs, LongValues values) {
-      this.docIDsLength = docIDsLength;
-      this.docIds = docIDs;
-      this.values = values;
-      reset();
-    }
-
-    void reset() {
-      index = -1;
-      doc = -1;
-    }
-
-    @Override
-    public int docID() {
-      return doc;
-    }
-
-    @Override
-    public int nextDoc() throws IOException {
-      if (index >= docIDsLength - 1) {
-        index = docIDsLength;
-        return doc = NO_MORE_DOCS;
-      }
-      return doc = (int) docIds.get(++index);
-    }
-
-    @Override
-    public int advance(int target) throws IOException {
-      long loIndex = index;
-      long step = 1;
-      long hiIndex;
-      int hiDoc;
-
-      // gallop forward by exponentially growing the interval
-      // in order to find an interval so that the target doc
-      // is in ]lo, hi]. Compared to a regular binary search,
-      // this optimizes the case that the caller performs many
-      // advance calls by small deltas
-      do {
-        hiIndex = index + step;
-        if (hiIndex >= docIDsLength) {
-          hiIndex = docIDsLength;
-          hiDoc = NO_MORE_DOCS;
-          break;
-        }
-        hiDoc = (int) docIds.get(hiIndex);
-        if (hiDoc >= target) {
-          break;
-        }
-        step <<= 1;
-      } while (true);
-
-      // now binary search
-      while (loIndex + 1 < hiIndex) {
-        final long midIndex = (loIndex + 1 + hiIndex) >>> 1;
-        final int midDoc = (int) docIds.get(midIndex);
-        if (midDoc >= target) {
-          hiIndex = midIndex;
-          hiDoc = midDoc;
-        } else {
-          loIndex = midIndex;
-        }
-      }
-
-      index = (int) hiIndex;
-      return doc = hiDoc;
-    }
-
-    @Override
-    public boolean advanceExact(int target) throws IOException {
-      if (advance(target) == target) {
-        return true;
-      }
-      --index;
-      doc = target;
-      return index >= 0 && docIds.get(index) == target;
-    }
-
-    @Override
-    public long longValue() {
-      assert index >= 0;
-      assert index < docIDsLength;
-      return values.get(index);
-    }
-
-    @Override
-    public long cost() {
-      return docIDsLength;
-    }
-  }
-
-  static class SparseNumericDocValuesRandomAccessWrapper extends LongValues {
-
-    final SparseNumericDocValues values;
-    final long missingValue;
-
-    SparseNumericDocValuesRandomAccessWrapper(SparseNumericDocValues values, long missingValue) {
-      this.values = values;
-      this.missingValue = missingValue;
-    }
-
-    @Override
-    public long get(long longIndex) {
-      final int index = Math.toIntExact(longIndex);
-      int doc = values.docID();
-      if (doc >= index) {
-        values.reset();
-      }
-      assert values.docID() < index;
-      try {
-        doc = values.advance(index);
-      } catch (IOException e) {
-        throw new RuntimeException(e);
-      }
-      if (doc == index) {
-        return values.longValue();
-      } else {
-        return missingValue;
-      }
-    }
-
-  }
-
-  LegacyBinaryDocValues getLegacyBinary(FieldInfo field) throws IOException {
-    BinaryEntry bytes = binaries.get(field.name);
-    switch(bytes.format) {
-      case BINARY_FIXED_UNCOMPRESSED:
-        return getFixedBinary(field, bytes);
-      case BINARY_VARIABLE_UNCOMPRESSED:
-        return getVariableBinary(field, bytes);
-      case BINARY_PREFIX_COMPRESSED:
-        return getCompressedBinary(field, bytes);
-      default:
-        throw new AssertionError();
-    }
-  }
-
-  @Override
-  public BinaryDocValues getBinary(FieldInfo field) throws IOException {
-    BinaryEntry be = binaries.get(field.name);
-    return new LegacyBinaryDocValuesWrapper(getLiveBits(be.missingOffset, maxDoc), getLegacyBinary(field));
-  }
-
-  private LegacyBinaryDocValues getFixedBinary(FieldInfo field, final BinaryEntry bytes) throws IOException {
-    final IndexInput data = this.data.slice("fixed-binary", bytes.offset, bytes.count * bytes.maxLength);
-
-    final BytesRef term = new BytesRef(bytes.maxLength);
-    final byte[] buffer = term.bytes;
-    final int length = term.length = bytes.maxLength;
-
-    return new LongBinaryDocValues() {
-      @Override
-      public BytesRef get(long id) {
-        try {
-          data.seek(id * length);
-          data.readBytes(buffer, 0, buffer.length);
-          return term;
-        } catch (IOException e) {
-          throw new RuntimeException(e);
-        }
-      }
-    };
-  }
-
-  private LegacyBinaryDocValues getVariableBinary(FieldInfo field, final BinaryEntry bytes) throws IOException {
-    final RandomAccessInput addressesData = this.data.randomAccessSlice(bytes.addressesOffset, bytes.addressesEndOffset - bytes.addressesOffset);
-    final LongValues addresses = DirectMonotonicReader.getInstance(bytes.addressesMeta, addressesData);
-
-    final IndexInput data = this.data.slice("var-binary", bytes.offset, bytes.addressesOffset - bytes.offset);
-    final BytesRef term = new BytesRef(Math.max(0, bytes.maxLength));
-    final byte buffer[] = term.bytes;
-
-    return new LongBinaryDocValues() {
-      @Override
-      public BytesRef get(long id) {
-        long startAddress = addresses.get(id);
-        long endAddress = addresses.get(id+1);
-        int length = (int) (endAddress - startAddress);
-        try {
-          data.seek(startAddress);
-          data.readBytes(buffer, 0, length);
-          term.length = length;
-          return term;
-        } catch (IOException e) {
-          throw new RuntimeException(e);
-        }
-      }
-    };
-  }
-
-  /** returns an address instance for prefix-compressed binary values. */
-  private synchronized MonotonicBlockPackedReader getIntervalInstance(FieldInfo field, BinaryEntry bytes) throws IOException {
-    MonotonicBlockPackedReader addresses = addressInstances.get(field.name);
-    if (addresses == null) {
-      data.seek(bytes.addressesOffset);
-      final long size = (bytes.count + INTERVAL_MASK) >>> INTERVAL_SHIFT;
-      addresses = MonotonicBlockPackedReader.of(data, bytes.packedIntsVersion, bytes.blockSize, size, false);
-      if (!merging) {
-        addressInstances.put(field.name, addresses);
-        ramBytesUsed.addAndGet(addresses.ramBytesUsed() + Integer.BYTES);
-      }
-    }
-    return addresses;
-  }
-
-  /** returns a reverse lookup instance for prefix-compressed binary values. */
-  private synchronized ReverseTermsIndex getReverseIndexInstance(FieldInfo field, BinaryEntry bytes) throws IOException {
-    ReverseTermsIndex index = reverseIndexInstances.get(field.name);
-    if (index == null) {
-      index = new ReverseTermsIndex();
-      data.seek(bytes.reverseIndexOffset);
-      long size = (bytes.count + REVERSE_INTERVAL_MASK) >>> REVERSE_INTERVAL_SHIFT;
-      index.termAddresses = MonotonicBlockPackedReader.of(data, bytes.packedIntsVersion, bytes.blockSize, size, false);
-      long dataSize = data.readVLong();
-      PagedBytes pagedBytes = new PagedBytes(15);
-      pagedBytes.copy(data, dataSize);
-      index.terms = pagedBytes.freeze(true);
-      if (!merging) {
-        reverseIndexInstances.put(field.name, index);
-        ramBytesUsed.addAndGet(index.ramBytesUsed());
-      }
-    }
-    return index;
-  }
-
-  private LegacyBinaryDocValues getCompressedBinary(FieldInfo field, final BinaryEntry bytes) throws IOException {
-    final MonotonicBlockPackedReader addresses = getIntervalInstance(field, bytes);
-    final ReverseTermsIndex index = getReverseIndexInstance(field, bytes);
-    assert addresses.size() > 0; // we don't have to handle empty case
-    IndexInput slice = data.slice("terms", bytes.offset, bytes.addressesOffset - bytes.offset);
-    return new CompressedBinaryDocValues(bytes, addresses, index, slice);
-  }
-
-  @Override
-  public SortedDocValues getSorted(FieldInfo field) throws IOException {
-    final int valueCount = (int) binaries.get(field.name).count;
-    final LegacyBinaryDocValues binary = getLegacyBinary(field);
-    NumericEntry entry = ords.get(field.name);
-    final LongValues ordinals = getNumeric(entry);
-    if (entry.format == SPARSE_COMPRESSED) {
-      final SparseNumericDocValues sparseValues = ((SparseNumericDocValuesRandomAccessWrapper) ordinals).values;
-      return new SortedDocValues() {
-
-        @Override
-        public int ordValue() {
-          return (int) sparseValues.longValue();
-        }
-
-        @Override
-        public BytesRef lookupOrd(int ord) {
-          return binary.get(ord);
-        }
-
-        @Override
-        public int getValueCount() {
-          return valueCount;
-        }
-
-        @Override
-        public int docID() {
-          return sparseValues.docID();
-        }
-
-        @Override
-        public int nextDoc() throws IOException {
-          return sparseValues.nextDoc();
-        }
-
-        @Override
-        public int advance(int target) throws IOException {
-          return sparseValues.advance(target);
-        }
-
-        @Override
-        public boolean advanceExact(int target) throws IOException {
-          return sparseValues.advanceExact(target);
-        }
-
-        @Override
-        public long cost() {
-          return sparseValues.cost();
-        }
-
-      };
-    }
-    return new SortedDocValues() {
-      private int docID = -1;
-      private int ord;
-
-      @Override
-      public int docID() {
-        return docID;
-      }
-
-      @Override
-      public int nextDoc() throws IOException {
-        assert docID != NO_MORE_DOCS;
-        while (true) {
-          docID++;
-          if (docID == maxDoc) {
-            docID = NO_MORE_DOCS;
-            break;
-          }
-          ord = (int) ordinals.get(docID);
-          if (ord != -1) {
-            break;
-          }
-        }
-        return docID;
-      }
-
-      @Override
-      public int advance(int target) throws IOException {
-        if (target >= maxDoc) {
-          docID = NO_MORE_DOCS;
-          return docID;
-        } else {
-          docID = target-1;
-          return nextDoc();
-        }
-      }
-
-      @Override
-      public boolean advanceExact(int target) throws IOException {
-        docID = target;
-        ord = (int) ordinals.get(target);
-        return ord != -1;
-      }
-
-      @Override
-      public int ordValue() {
-        return ord;
-      }
-
-      @Override
-      public long cost() {
-        // TODO
-        return 0;
-      }
-
-      @Override
-      public BytesRef lookupOrd(int ord) {
-        return binary.get(ord);
-      }
-
-      @Override
-      public int getValueCount() {
-        return valueCount;
-      }
-
-      @Override
-      public int lookupTerm(BytesRef key) throws IOException {
-        if (binary instanceof CompressedBinaryDocValues) {
-          return (int) ((CompressedBinaryDocValues)binary).lookupTerm(key);
-        } else {
-          return super.lookupTerm(key);
-        }
-      }
-
-      @Override
-      public TermsEnum termsEnum() throws IOException {
-        if (binary instanceof CompressedBinaryDocValues) {
-          return ((CompressedBinaryDocValues)binary).getTermsEnum();
-        } else {
-          return super.termsEnum();
-        }
-      }
-    };
-  }
-
-  /** returns an address instance for sortedset ordinal lists */
-  private LongValues getOrdIndexInstance(FieldInfo field, NumericEntry entry) throws IOException {
-    RandomAccessInput data = this.data.randomAccessSlice(entry.offset, entry.endOffset - entry.offset);
-    return DirectMonotonicReader.getInstance(entry.monotonicMeta, data);
-  }
-
-  @Override
-  public SortedNumericDocValues getSortedNumeric(FieldInfo field) throws IOException {
-    SortedSetEntry ss = sortedNumerics.get(field.name);
-    if (ss.format == SORTED_SINGLE_VALUED) {
-      NumericEntry numericEntry = numerics.get(field.name);
-      final LongValues values = getNumeric(numericEntry);
-      if (numericEntry.format == SPARSE_COMPRESSED) {
-        SparseNumericDocValues sparseValues = ((SparseNumericDocValuesRandomAccessWrapper) values).values;
-        return new SortedNumericDocValues() {
-
-          @Override
-          public long nextValue() throws IOException {
-            return sparseValues.longValue();
-          }
-
-          @Override
-          public int docValueCount() {
-            return 1;
-          }
-
-          @Override
-          public int docID() {
-            return sparseValues.docID();
-          }
-
-          @Override
-          public int nextDoc() throws IOException {
-            return sparseValues.nextDoc();
-          }
-
-          @Override
-          public int advance(int target) throws IOException {
-            return sparseValues.advance(target);
-          }
-
-          @Override
-          public boolean advanceExact(int target) throws IOException {
-            return sparseValues.advanceExact(target);
-          }
-
-          @Override
-          public long cost() {
-            return sparseValues.cost();
-          }
-
-        };
-      }
-      final Bits docsWithField = getLiveBits(numericEntry.missingOffset, maxDoc);
-      return new SortedNumericDocValues() {
-        int docID = -1;
-
-        @Override
-        public int docID() {
-          return docID;
-        }
-
-        @Override
-        public int nextDoc() {
-          while (true) {
-            docID++;
-            if (docID == maxDoc) {
-              docID = NO_MORE_DOCS;
-              break;
-            }
-
-            if (docsWithField.get(docID)) {
-              // TODO: use .nextSetBit here, at least!!
-              break;
-            }
-          }
-          return docID;
-        }
-
-        @Override
-        public int advance(int target) {
-          if (target >= maxDoc) {
-            docID = NO_MORE_DOCS;
-            return docID;
-          } else {
-            docID = target-1;
-            return nextDoc();
-          }
-        }
-
-        @Override
-        public boolean advanceExact(int target) throws IOException {
-          docID = target;
-          return docsWithField.get(docID);
-        }
-
-        @Override
-        public long cost() {
-          // TODO
-          return 0;
-        }
-
-        @Override
-        public int docValueCount() {
-          return 1;
-        }
-
-        @Override
-        public long nextValue() {
-          return values.get(docID);
-        }
-      };
-    } else if (ss.format == SORTED_WITH_ADDRESSES) {
-      NumericEntry numericEntry = numerics.get(field.name);
-      final LongValues values = getNumeric(numericEntry);
-      final LongValues ordIndex = getOrdIndexInstance(field, ordIndexes.get(field.name));
-
-      return new SortedNumericDocValues() {
-        long startOffset;
-        long endOffset;
-        int docID = -1;
-        long upto;
-
-        @Override
-        public int docID() {
-          return docID;
-        }
-
-        @Override
-        public int nextDoc() {
-          while (true) {
-            docID++;
-            if (docID == maxDoc) {
-              docID = NO_MORE_DOCS;
-              return docID;
-            }
-            startOffset = ordIndex.get(docID);
-            endOffset = ordIndex.get(docID+1L);
-            if (endOffset > startOffset) {
-              break;
-            }
-          }
-          upto = startOffset;
-          return docID;
-        }
-
-        @Override
-        public int advance(int target) {
-          if (target >= maxDoc) {
-            docID = NO_MORE_DOCS;
-            return docID;
-          } else {
-            docID = target-1;
-            return nextDoc();
-          }
-        }
-        
-        @Override
-        public boolean advanceExact(int target) throws IOException {
-          docID = target;
-          startOffset = ordIndex.get(docID);
-          endOffset = ordIndex.get(docID+1L);
-          return endOffset > startOffset;
-        }
-        
-        @Override
-        public long cost() {
-          // TODO
-          return 0;
-        }
-        
-        @Override
-        public int docValueCount() {
-          return (int) (endOffset - startOffset);
-        }
-        
-        @Override
-        public long nextValue() {
-          return values.get(upto++);
-        }
-      };
-    } else if (ss.format == SORTED_SET_TABLE) {
-      NumericEntry entry = ords.get(field.name);
-      final LongValues ordinals = getNumeric(entry);
-
-      final long[] table = ss.table;
-      final int[] offsets = ss.tableOffsets;
-      return new SortedNumericDocValues() {
-        int startOffset;
-        int endOffset;
-        int docID = -1;
-        int upto;
-
-        @Override
-        public int docID() {
-          return docID;
-        }
-
-        @Override
-        public int nextDoc() {
-          while (true) {
-            docID++;
-            if (docID == maxDoc) {
-              docID = NO_MORE_DOCS;
-              return docID;
-            }
-            int ord = (int) ordinals.get(docID);
-            startOffset = offsets[ord];
-            endOffset = offsets[ord+1];
-            if (endOffset > startOffset) {
-              break;
-            }
-          }
-          upto = startOffset;
-          return docID;
-        }
-
-        @Override
-        public int advance(int target) {
-          if (target >= maxDoc) {
-            docID = NO_MORE_DOCS;
-            return docID;
-          } else {
-            docID = target-1;
-            return nextDoc();
-          }
-        }
-        
-        @Override
-        public boolean advanceExact(int target) throws IOException {
-          docID = target;
-          int ord = (int) ordinals.get(docID);
-          startOffset = offsets[ord];
-          endOffset = offsets[ord+1];
-          return endOffset > startOffset;
-        }
-        
-        @Override
-        public long cost() {
-          // TODO
-          return 0;
-        }
-
-        @Override
-        public int docValueCount() {
-          return endOffset - startOffset;
-        }
-        
-        @Override
-        public long nextValue() {
-          return table[upto++];
-        }
-      };
-    } else {
-      throw new AssertionError();
-    }
-  }
-
-  @Override
-  public SortedSetDocValues getSortedSet(FieldInfo field) throws IOException {
-    SortedSetEntry ss = sortedSets.get(field.name);
-    switch (ss.format) {
-      case SORTED_SINGLE_VALUED:
-        return DocValues.singleton(getSorted(field));
-      case SORTED_WITH_ADDRESSES:
-        return getSortedSetWithAddresses(field);
-      case SORTED_SET_TABLE:
-        return getSortedSetTable(field, ss);
-      default:
-        throw new AssertionError();
-    }
-  }
-
-  private SortedSetDocValues getSortedSetWithAddresses(FieldInfo field) throws IOException {
-    final long valueCount = binaries.get(field.name).count;
-    // we keep the byte[]s and list of ords on disk, these could be large
-    final LongBinaryDocValues binary = (LongBinaryDocValues) getLegacyBinary(field);
-    final LongValues ordinals = getNumeric(ords.get(field.name));
-    // but the addresses to the ord stream are in RAM
-    final LongValues ordIndex = getOrdIndexInstance(field, ordIndexes.get(field.name));
-
-    return new LegacySortedSetDocValuesWrapper(new LegacySortedSetDocValues() {
-      long startOffset;
-      long offset;
-      long endOffset;
-
-      @Override
-      public long nextOrd() {
-        if (offset == endOffset) {
-          return NO_MORE_ORDS;
-        } else {
-          long ord = ordinals.get(offset);
-          offset++;
-          return ord;
-        }
-      }
-
-      @Override
-      public void setDocument(int docID) {
-        startOffset = offset = ordIndex.get(docID);
-        endOffset = ordIndex.get(docID+1L);
-      }
-
-      @Override
-      public BytesRef lookupOrd(long ord) {
-        return binary.get(ord);
-      }
-
-      @Override
-      public long getValueCount() {
-        return valueCount;
-      }
-
-      @Override
-      public long lookupTerm(BytesRef key) {
-        if (binary instanceof CompressedBinaryDocValues) {
-          return ((CompressedBinaryDocValues)binary).lookupTerm(key);
-        } else {
-          return super.lookupTerm(key);
-        }
-      }
-
-      @Override
-      public TermsEnum termsEnum() throws IOException {
-        if (binary instanceof CompressedBinaryDocValues) {
-          return ((CompressedBinaryDocValues)binary).getTermsEnum();
-        } else {
-          return super.termsEnum();
-        }
-      }
-      }, maxDoc);
-  }
-
-  private SortedSetDocValues getSortedSetTable(FieldInfo field, SortedSetEntry ss) throws IOException {
-    final long valueCount = binaries.get(field.name).count;
-    final LongBinaryDocValues binary = (LongBinaryDocValues) getLegacyBinary(field);
-    final NumericEntry ordinalsEntry = ords.get(field.name);
-    final LongValues ordinals = getNumeric(ordinalsEntry);
-
-    final long[] table = ss.table;
-    final int[] offsets = ss.tableOffsets;
-
-    return new LegacySortedSetDocValuesWrapper(new LegacySortedSetDocValues() {
-
-      int offset, startOffset, endOffset;
-
-      @Override
-      public void setDocument(int docID) {
-        final int ord = (int) ordinals.get(docID);
-        offset = startOffset = offsets[ord];
-        endOffset = offsets[ord + 1];
-      }
-
-      @Override
-      public long nextOrd() {
-        if (offset == endOffset) {
-          return NO_MORE_ORDS;
-        } else {
-          return table[offset++];
-        }
-      }
-
-      @Override
-      public BytesRef lookupOrd(long ord) {
-        return binary.get(ord);
-      }
-
-      @Override
-      public long getValueCount() {
-        return valueCount;
-      }
-
-      @Override
-      public long lookupTerm(BytesRef key) {
-        if (binary instanceof CompressedBinaryDocValues) {
-          return ((CompressedBinaryDocValues) binary).lookupTerm(key);
-        } else {
-          return super.lookupTerm(key);
-        }
-      }
-
-      @Override
-      public TermsEnum termsEnum() throws IOException {
-        if (binary instanceof CompressedBinaryDocValues) {
-          return ((CompressedBinaryDocValues) binary).getTermsEnum();
-        } else {
-          return super.termsEnum();
-        }
-      }
-      }, maxDoc);
-  }
-
-  private Bits getLiveBits(final long offset, final int count) throws IOException {
-    if (offset == ALL_MISSING) {
-      return new Bits.MatchNoBits(count);
-    } else if (offset == ALL_LIVE) {
-      return new Bits.MatchAllBits(count);
-    } else {
-      int length = (int) ((count + 7L) >>> 3);
-      final RandomAccessInput in = data.randomAccessSlice(offset, length);
-      return new Bits() {
-        @Override
-        public boolean get(int index) {
-          try {
-            return (in.readByte(index >> 3) & (1 << (index & 7))) != 0;
-          } catch (IOException e) {
-            throw new RuntimeException(e);
-          }
-        }
-
-        @Override
-        public int length() {
-          return count;
-        }
-      };
-    }
-  }
-
-  private SparseNumericDocValues getSparseNumericDocValues(NumericEntry entry) throws IOException {
-    final RandomAccessInput docIdsData = this.data.randomAccessSlice(entry.missingOffset, entry.offset - entry.missingOffset);
-    final LongValues docIDs = DirectMonotonicReader.getInstance(entry.monotonicMeta, docIdsData);
-    final LongValues values = getNumeric(entry.nonMissingValues); // cannot be sparse
-    return new SparseNumericDocValues(Math.toIntExact(entry.numDocsWithValue), docIDs, values);
-  }
-
-  @Override
-  public synchronized DocValuesProducer getMergeInstance() throws IOException {
-    return new Lucene54DocValuesProducer(this);
-  }
-
-  @Override
-  public void close() throws IOException {
-    data.close();
-  }
-
-  /** metadata entry for a numeric docvalues field */
-  static class NumericEntry {
-    private NumericEntry() {}
-    /** offset to the bitset representing docsWithField, or -1 if no documents have missing values */
-    long missingOffset;
-    /** offset to the actual numeric values */
-    public long offset;
-    /** end offset to the actual numeric values */
-    public long endOffset;
-    /** bits per value used to pack the numeric values */
-    public int bitsPerValue;
-
-    int format;
-    /** count of values written */
-    public long count;
-
-    /** monotonic meta */
-    public DirectMonotonicReader.Meta monotonicMeta;
-
-    long minValue;
-    long gcd;
-    long table[];
-
-    /** for sparse compression */
-    long numDocsWithValue;
-    NumericEntry nonMissingValues;
-    NumberType numberType;
-
-  }
-
-  /** metadata entry for a binary docvalues field */
-  static class BinaryEntry {
-    private BinaryEntry() {}
-    /** offset to the bitset representing docsWithField, or -1 if no documents have missing values */
-    long missingOffset;
-    /** offset to the actual binary values */
-    long offset;
-
-    int format;
-    /** count of values written */
-    public long count;
-    int minLength;
-    int maxLength;
-    /** offset to the addressing data that maps a value to its slice of the byte[] */
-    public long addressesOffset, addressesEndOffset;
-    /** meta data for addresses */
-    public DirectMonotonicReader.Meta addressesMeta;
-    /** offset to the reverse index */
-    public long reverseIndexOffset;
-    /** packed ints version used to encode addressing information */
-    public int packedIntsVersion;
-    /** packed ints blocksize */
-    public int blockSize;
-  }
-
-  /** metadata entry for a sorted-set docvalues field */
-  static class SortedSetEntry {
-    private SortedSetEntry() {}
-    int format;
-
-    long[] table;
-    int[] tableOffsets;
-  }
-
-  // internally we compose complex dv (sorted/sortedset) from other ones
-  static abstract class LongBinaryDocValues extends LegacyBinaryDocValues {
-    @Override
-    public final BytesRef get(int docID) {
-      return get((long)docID);
-    }
-
-    abstract BytesRef get(long id);
-  }
-
-  // used for reverse lookup to a small range of blocks
-  static class ReverseTermsIndex implements Accountable {
-    public MonotonicBlockPackedReader termAddresses;
-    public PagedBytes.Reader terms;
-
-    @Override
-    public long ramBytesUsed() {
-      return termAddresses.ramBytesUsed() + terms.ramBytesUsed();
-    }
-
-    @Override
-    public Collection<Accountable> getChildResources() {
-      List<Accountable> resources = new ArrayList<>();
-      resources.add(Accountables.namedAccountable("term bytes", terms));
-      resources.add(Accountables.namedAccountable("term addresses", termAddresses));
-      return Collections.unmodifiableList(resources);
-    }
-
-    @Override
-    public String toString() {
-      return getClass().getSimpleName() + "(size=" + termAddresses.size() + ")";
-    }
-  }
-
-  //in the compressed case, we add a few additional operations for
-  //more efficient reverse lookup and enumeration
-  static final class CompressedBinaryDocValues extends LongBinaryDocValues {
-    final long numValues;
-    final long numIndexValues;
-    final int maxTermLength;
-    final MonotonicBlockPackedReader addresses;
-    final IndexInput data;
-    final CompressedBinaryTermsEnum termsEnum;
-    final PagedBytes.Reader reverseTerms;
-    final MonotonicBlockPackedReader reverseAddresses;
-    final long numReverseIndexValues;
-
-    public CompressedBinaryDocValues(BinaryEntry bytes, MonotonicBlockPackedReader addresses, ReverseTermsIndex index, IndexInput data) throws IOException {
-      this.maxTermLength = bytes.maxLength;
-      this.numValues = bytes.count;
-      this.addresses = addresses;
-      this.numIndexValues = addresses.size();
-      this.data = data;
-      this.reverseTerms = index.terms;
-      this.reverseAddresses = index.termAddresses;
-      this.numReverseIndexValues = reverseAddresses.size();
-      this.termsEnum = getTermsEnum(data);
-    }
-
-    @Override
-    public BytesRef get(long id) {
-      try {
-        termsEnum.seekExact(id);
-        return termsEnum.term();
-      } catch (IOException e) {
-        throw new RuntimeException(e);
-      }
-    }
-
-    long lookupTerm(BytesRef key) {
-      try {
-        switch (termsEnum.seekCeil(key)) {
-          case FOUND: return termsEnum.ord();
-          case NOT_FOUND: return -termsEnum.ord()-1;
-          default: return -numValues-1;
-        }
-      } catch (IOException bogus) {
-        throw new RuntimeException(bogus);
-      }
-    }
-
-    TermsEnum getTermsEnum() throws IOException {
-      return getTermsEnum(data.clone());
-    }
-
-    private CompressedBinaryTermsEnum getTermsEnum(IndexInput input) throws IOException {
-      return new CompressedBinaryTermsEnum(input);
-    }
-
-    class CompressedBinaryTermsEnum extends TermsEnum {
-      private long currentOrd = -1;
-      // offset to the start of the current block
-      private long currentBlockStart;
-      private final IndexInput input;
-      // delta from currentBlockStart to start of each term
-      private final int offsets[] = new int[INTERVAL_COUNT];
-      private final byte buffer[] = new byte[2*INTERVAL_COUNT-1];
-
-      private final BytesRef term = new BytesRef(maxTermLength);
-      private final BytesRef firstTerm = new BytesRef(maxTermLength);
-      private final BytesRef scratch = new BytesRef();
-
-      CompressedBinaryTermsEnum(IndexInput input) throws IOException {
-        this.input = input;
-        input.seek(0);
-      }
-
-      private void readHeader() throws IOException {
-        firstTerm.length = input.readVInt();
-        input.readBytes(firstTerm.bytes, 0, firstTerm.length);
-        input.readBytes(buffer, 0, INTERVAL_COUNT-1);
-        if (buffer[0] == -1) {
-          readShortAddresses();
-        } else {
-          readByteAddresses();
-        }
-        currentBlockStart = input.getFilePointer();
-      }
-
-      // read single byte addresses: each is delta - 2
-      // (shared prefix byte and length > 0 are both implicit)
-      private void readByteAddresses() throws IOException {
-        int addr = 0;
-        for (int i = 1; i < offsets.length; i++) {
-          addr += 2 + (buffer[i-1] & 0xFF);
-          offsets[i] = addr;
-        }
-      }
-
-      // read double byte addresses: each is delta - 2
-      // (shared prefix byte and length > 0 are both implicit)
-      private void readShortAddresses() throws IOException {
-        input.readBytes(buffer, INTERVAL_COUNT-1, INTERVAL_COUNT);
-        int addr = 0;
-        for (int i = 1; i < offsets.length; i++) {
-          int x = i<<1;
-          addr += 2 + ((buffer[x-1] << 8) | (buffer[x] & 0xFF));
-          offsets[i] = addr;
-        }
-      }
-
-      // set term to the first term
-      private void readFirstTerm() throws IOException {
-        term.length = firstTerm.length;
-        System.arraycopy(firstTerm.bytes, firstTerm.offset, term.bytes, 0, term.length);
-      }
-
-      // read term at offset, delta encoded from first term
-      private void readTerm(int offset) throws IOException {
-        int start = input.readByte() & 0xFF;
-        System.arraycopy(firstTerm.bytes, firstTerm.offset, term.bytes, 0, start);
-        int suffix = offsets[offset] - offsets[offset-1] - 1;
-        input.readBytes(term.bytes, start, suffix);
-        term.length = start + suffix;
-      }
-
-      @Override
-      public BytesRef next() throws IOException {
-        currentOrd++;
-        if (currentOrd >= numValues) {
-          return null;
-        } else {
-          int offset = (int) (currentOrd & INTERVAL_MASK);
-          if (offset == 0) {
-            // switch to next block
-            readHeader();
-            readFirstTerm();
-          } else {
-            readTerm(offset);
-          }
-          return term;
-        }
-      }
-
-      // binary search reverse index to find smaller
-      // range of blocks to search
-      long binarySearchIndex(BytesRef text) throws IOException {
-        long low = 0;
-        long high = numReverseIndexValues - 1;
-        while (low <= high) {
-          long mid = (low + high) >>> 1;
-          reverseTerms.fill(scratch, reverseAddresses.get(mid));
-          int cmp = scratch.compareTo(text);
-
-          if (cmp < 0) {
-            low = mid + 1;
-          } else if (cmp > 0) {
-            high = mid - 1;
-          } else {
-            return mid;
-          }
-        }
-        return high;
-      }
-
-      // binary search against first term in block range
-      // to find term's block
-      long binarySearchBlock(BytesRef text, long low, long high) throws IOException {
-        while (low <= high) {
-          long mid = (low + high) >>> 1;
-          input.seek(addresses.get(mid));
-          term.length = input.readVInt();
-          input.readBytes(term.bytes, 0, term.length);
-          int cmp = term.compareTo(text);
-
-          if (cmp < 0) {
-            low = mid + 1;
-          } else if (cmp > 0) {
-            high = mid - 1;
-          } else {
-            return mid;
-          }
-        }
-        return high;
-      }
-
-      @Override
-      public SeekStatus seekCeil(BytesRef text) throws IOException {
-        // locate block: narrow to block range with index, then search blocks
-        final long block;
-        long indexPos = binarySearchIndex(text);
-        if (indexPos < 0) {
-          block = 0;
-        } else {
-          long low = indexPos << BLOCK_INTERVAL_SHIFT;
-          long high = Math.min(numIndexValues - 1, low + BLOCK_INTERVAL_MASK);
-          block = Math.max(low, binarySearchBlock(text, low, high));
-        }
-
-        // position before block, then scan to term.
-        input.seek(addresses.get(block));
-        currentOrd = (block << INTERVAL_SHIFT) - 1;
-
-        while (next() != null) {
-          int cmp = term.compareTo(text);
-          if (cmp == 0) {
-            return SeekStatus.FOUND;
-          } else if (cmp > 0) {
-            return SeekStatus.NOT_FOUND;
-          }
-        }
-        return SeekStatus.END;
-      }
-
-      @Override
-      public void seekExact(long ord) throws IOException {
-        long block = ord >>> INTERVAL_SHIFT;
-        if (block != currentOrd >>> INTERVAL_SHIFT) {
-          // switch to different block
-          input.seek(addresses.get(block));
-          readHeader();
-        }
-
-        currentOrd = ord;
-
-        int offset = (int) (ord & INTERVAL_MASK);
-        if (offset == 0) {
-          readFirstTerm();
-        } else {
-          input.seek(currentBlockStart + offsets[offset-1]);
-          readTerm(offset);
-        }
-      }
-
-      @Override
-      public BytesRef term() throws IOException {
-        return term;
-      }
-
-      @Override
-      public long ord() throws IOException {
-        return currentOrd;
-      }
-
-      @Override
-      public int docFreq() throws IOException {
-        throw new UnsupportedOperationException();
-      }
-
-      @Override
-      public long totalTermFreq() throws IOException {
-        return -1;
-      }
-
-      @Override
-      public PostingsEnum postings(PostingsEnum reuse, int flags) throws IOException {
-        throw new UnsupportedOperationException();
-      }
-
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/189f2b3f/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene54/package-info.java
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene54/package-info.java b/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene54/package-info.java
deleted file mode 100644
index 2c19222..0000000
--- a/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene54/package-info.java
+++ /dev/null
@@ -1,403 +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.
- */
-
-/**
- * Lucene 5.4 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.lucene62.Lucene62SegmentInfoFormat 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>
- * </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.lucene62.Lucene62SegmentInfoFormat 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>
- * </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>
- * </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>
- */
-package org.apache.lucene.codecs.lucene54;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/189f2b3f/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
deleted file mode 100644
index 4bed311..0000000
--- a/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene50/Lucene50RWSegmentInfoFormat.java
+++ /dev/null
@@ -1,125 +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.Map;
-import java.util.Set;
-
-import org.apache.lucene.codecs.CodecUtil;
-import org.apache.lucene.index.CorruptIndexException;
-import org.apache.lucene.index.IndexFileNames;
-import org.apache.lucene.index.SegmentInfo; // javadocs
-import org.apache.lucene.store.ChecksumIndexInput;
-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 {
-        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 = input.readMapOfStrings();
-        final Set<String> files = input.readSetOfStrings();
-        final Map<String,String> attributes = input.readMapOfStrings();
-        
-        si = new SegmentInfo(dir, version, null, 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);
-
-    assert si.getIndexSort() == null;
-
-    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_SAFE_MAPS = 1;
-  static final int VERSION_START = VERSION_SAFE_MAPS;
-  static final int VERSION_CURRENT = VERSION_SAFE_MAPS;
-}


[20/23] lucene-solr:jira/solr-10879: Adds 7.1.0 section to solr/CHANGES.txt file.

Posted by ab...@apache.org.
Adds 7.1.0 section to solr/CHANGES.txt file.


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

Branch: refs/heads/jira/solr-10879
Commit: e9d33ee5ac5c2bcd3fb64f0801d0856ba4bc5588
Parents: 6c16365
Author: Christine Poerschke <cp...@apache.org>
Authored: Tue Jul 4 11:14:45 2017 +0100
Committer: Christine Poerschke <cp...@apache.org>
Committed: Tue Jul 4 11:14:45 2017 +0100

----------------------------------------------------------------------
 solr/CHANGES.txt | 41 +++++++++++++++++++++++++++++++++++++++++
 1 file changed, 41 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e9d33ee5/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 61f72c4..360916b 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -33,6 +33,47 @@ Jetty 9.3.14.v20161028
 (No Changes)
 
 
+==================  7.1.0 ==================
+
+Consult the LUCENE_CHANGES.txt file for additional, low level, changes in this release.
+
+Versions of Major Components
+---------------------
+Apache Tika 1.13
+Carrot2 3.15.0
+Velocity 1.7 and Velocity Tools 2.0
+Apache UIMA 2.3.1
+Apache ZooKeeper 3.4.10
+Jetty 9.3.14.v20161028
+
+Detailed Change List
+----------------------
+
+Upgrade Notes
+----------------------
+
+(No Notes)
+
+New Features
+----------------------
+
+(No Changes)
+
+Bug Fixes
+----------------------
+
+(No Changes)
+
+Optimizations
+----------------------
+
+(No Changes)
+
+Other Changes
+----------------------
+
+(No Changes)
+
 ==================  7.0.0 ==================
 
 Versions of Major Components


[08/23] lucene-solr:jira/solr-10879: Merge branch 'master' into upgrade-master-to-8

Posted by ab...@apache.org.
Merge branch 'master' into upgrade-master-to-8


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

Branch: refs/heads/jira/solr-10879
Commit: e809e095f04309726cb282bbaac56fed2844a584
Parents: 189f2b3 20dcb56
Author: Adrien Grand <jp...@gmail.com>
Authored: Mon Jul 3 09:25:39 2017 +0200
Committer: Adrien Grand <jp...@gmail.com>
Committed: Mon Jul 3 09:25:39 2017 +0200

----------------------------------------------------------------------
 dev-tools/scripts/smokeTestRelease.py           | 12 ++--
 .../apache/lucene/index/DocumentsWriter.java    | 20 +++---
 .../lucene/index/DocumentsWriterFlushQueue.java |  4 ++
 .../lucene/index/FrozenBufferedUpdates.java     | 19 ++++--
 .../org/apache/lucene/index/IndexWriter.java    |  5 +-
 .../apache/lucene/index/ReadersAndUpdates.java  |  8 ++-
 .../lucene/index/TestIndexManyDocuments.java    | 71 ++++++++++++++++++++
 solr/CHANGES.txt                                |  1 +
 .../solr/common/cloud/ReplicaPosition.java      | 55 ---------------
 .../conf/bad-schema-dup-fieldType.xml           |  4 +-
 .../collection1/conf/schema-binaryfield.xml     | 36 ----------
 .../collection1/conf/schema-copyfield-test.xml  | 34 ++--------
 .../collection1/conf/schema-customfield.xml     | 29 +-------
 .../conf/schema-non-stored-docvalues.xml        | 47 ++-----------
 .../conf/schema-not-required-unique-key.xml     |  1 -
 .../collection1/conf/schema-replication1.xml    |  1 -
 .../collection1/conf/schema-replication2.xml    |  1 -
 .../collection1/conf/schema-required-fields.xml | 34 ++--------
 .../solr/collection1/conf/schema-reversed.xml   |  5 +-
 .../collection1/conf/schema-tokenizer-test.xml  | 21 +-----
 .../solr/collection1/conf/schema-version-dv.xml |  5 +-
 .../collection1/conf/schema-version-indexed.xml |  5 +-
 .../org/apache/solr/TestRandomDVFaceting.java   | 28 ++++++--
 .../apache/solr/core/DirectoryFactoryTest.java  | 15 +++--
 .../solr/common/cloud/ReplicaPosition.java      | 55 +++++++++++++++
 25 files changed, 232 insertions(+), 284 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e809e095/solr/CHANGES.txt
----------------------------------------------------------------------


[07/23] lucene-solr:jira/solr-10879: Changes for version upgrade to 8.0 on master for the 7.0 release

Posted by ab...@apache.org.
Changes for version upgrade to 8.0 on master for the 7.0 release


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

Branch: refs/heads/jira/solr-10879
Commit: 189f2b3f6e0dfb3c6f8e14a1a66c7bbaa4000970
Parents: 93fbfc2
Author: Anshum Gupta <an...@apple.com>
Authored: Fri Jun 30 11:18:28 2017 -0700
Committer: Anshum Gupta <an...@apple.com>
Committed: Fri Jun 30 11:18:28 2017 -0700

----------------------------------------------------------------------
 .../analysis/custom/TestCustomAnalyzer.java     |    8 +-
 .../lucene50/Lucene50SegmentInfoFormat.java     |   90 -
 .../apache/lucene/codecs/lucene50/package.html  |   25 -
 .../codecs/lucene53/Lucene53NormsFormat.java    |   91 -
 .../codecs/lucene53/Lucene53NormsProducer.java  |  236 ---
 .../lucene/codecs/lucene53/package-info.java    |   23 -
 .../lucene54/Lucene54DocValuesConsumer.java     |  797 --------
 .../lucene54/Lucene54DocValuesFormat.java       |  186 --
 .../lucene54/Lucene54DocValuesProducer.java     | 1803 ------------------
 .../lucene/codecs/lucene54/package-info.java    |  403 ----
 .../lucene50/Lucene50RWSegmentInfoFormat.java   |  125 --
 .../lucene50/TestLucene50SegmentInfoFormat.java |   50 -
 .../codecs/lucene53/Lucene53NormsConsumer.java  |  159 --
 .../codecs/lucene53/Lucene53RWNormsFormat.java  |   31 -
 .../lucene53/TestLucene53NormsFormat.java       |   44 -
 .../lucene54/TestLucene54DocValuesFormat.java   |  640 -------
 .../index/IndexFormatTooOldException.java       |    2 +-
 .../java/org/apache/lucene/util/Version.java    |   77 -
 .../org/apache/lucene/util/TestVersion.java     |   71 +-
 19 files changed, 41 insertions(+), 4820 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/189f2b3f/lucene/analysis/common/src/test/org/apache/lucene/analysis/custom/TestCustomAnalyzer.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/test/org/apache/lucene/analysis/custom/TestCustomAnalyzer.java b/lucene/analysis/common/src/test/org/apache/lucene/analysis/custom/TestCustomAnalyzer.java
index d929bfd..d9ea43c 100644
--- a/lucene/analysis/common/src/test/org/apache/lucene/analysis/custom/TestCustomAnalyzer.java
+++ b/lucene/analysis/common/src/test/org/apache/lucene/analysis/custom/TestCustomAnalyzer.java
@@ -107,7 +107,7 @@ public class TestCustomAnalyzer extends BaseTokenStreamTestCase {
 
   public void testFactoryHtmlStripClassicFolding() throws Exception {
     CustomAnalyzer a = CustomAnalyzer.builder()
-        .withDefaultMatchVersion(Version.LUCENE_6_0_0)
+        .withDefaultMatchVersion(Version.LUCENE_7_0_0)
         .addCharFilter(HTMLStripCharFilterFactory.class)
         .withTokenizer(ClassicTokenizerFactory.class)
         .addTokenFilter(ASCIIFoldingFilterFactory.class, "preserveOriginal", "true")
@@ -126,7 +126,7 @@ public class TestCustomAnalyzer extends BaseTokenStreamTestCase {
     assertSame(LowerCaseFilterFactory.class, tokenFilters.get(1).getClass());
     assertEquals(100, a.getPositionIncrementGap("dummy"));
     assertEquals(1000, a.getOffsetGap("dummy"));
-    assertSame(Version.LUCENE_6_0_0, a.getVersion());
+    assertSame(Version.LUCENE_7_0_0, a.getVersion());
 
     assertAnalyzesTo(a, "<p>foo bar</p> FOO BAR", 
         new String[] { "foo", "bar", "foo", "bar" },
@@ -139,7 +139,7 @@ public class TestCustomAnalyzer extends BaseTokenStreamTestCase {
   
   public void testHtmlStripClassicFolding() throws Exception {
     CustomAnalyzer a = CustomAnalyzer.builder()
-        .withDefaultMatchVersion(Version.LUCENE_6_0_0)
+        .withDefaultMatchVersion(Version.LUCENE_7_0_0)
         .addCharFilter("htmlstrip")
         .withTokenizer("classic")
         .addTokenFilter("asciifolding", "preserveOriginal", "true")
@@ -158,7 +158,7 @@ public class TestCustomAnalyzer extends BaseTokenStreamTestCase {
     assertSame(LowerCaseFilterFactory.class, tokenFilters.get(1).getClass());
     assertEquals(100, a.getPositionIncrementGap("dummy"));
     assertEquals(1000, a.getOffsetGap("dummy"));
-    assertSame(Version.LUCENE_6_0_0, a.getVersion());
+    assertSame(Version.LUCENE_7_0_0, a.getVersion());
 
     assertAnalyzesTo(a, "<p>foo bar</p> FOO BAR", 
         new String[] { "foo", "bar", "foo", "bar" },

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/189f2b3f/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
deleted file mode 100644
index d2a384e..0000000
--- a/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene50/Lucene50SegmentInfoFormat.java
+++ /dev/null
@@ -1,90 +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.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.SegmentInfo; // javadocs
-import org.apache.lucene.store.ChecksumIndexInput;
-import org.apache.lucene.store.Directory;
-import org.apache.lucene.store.IOContext;
-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 {
-        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 = input.readMapOfStrings();
-        final Set<String> files = input.readSetOfStrings();
-        final Map<String,String> attributes = input.readMapOfStrings();
-        
-        si = new SegmentInfo(dir, version, null, 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_SAFE_MAPS = 1;
-  static final int VERSION_START = VERSION_SAFE_MAPS;
-  static final int VERSION_CURRENT = VERSION_SAFE_MAPS;
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/189f2b3f/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene50/package.html
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene50/package.html b/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene50/package.html
deleted file mode 100644
index 352cef4..0000000
--- a/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene50/package.html
+++ /dev/null
@@ -1,25 +0,0 @@
-<!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.0 file format.
-</body>
-</html>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/189f2b3f/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene53/Lucene53NormsFormat.java
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene53/Lucene53NormsFormat.java b/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene53/Lucene53NormsFormat.java
deleted file mode 100644
index 1f7928f..0000000
--- a/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene53/Lucene53NormsFormat.java
+++ /dev/null
@@ -1,91 +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.lucene53;
-
-
-import java.io.IOException;
-
-import org.apache.lucene.codecs.CodecUtil;
-import org.apache.lucene.codecs.NormsConsumer;
-import org.apache.lucene.codecs.NormsFormat;
-import org.apache.lucene.codecs.NormsProducer;
-import org.apache.lucene.index.SegmentReadState;
-import org.apache.lucene.index.SegmentWriteState;
-import org.apache.lucene.store.DataOutput;
-
-/**
- * Lucene 5.3 Score normalization format.
- * <p>
- * Encodes normalization values by encoding each value with the minimum
- * number of bytes needed to represent the range (which can be zero).
- * <p>
- * Files:
- * <ol>
- *   <li><tt>.nvd</tt>: Norms data</li>
- *   <li><tt>.nvm</tt>: Norms metadata</li>
- * </ol>
- * <ol>
- *   <li><a name="nvm"></a>
- *   <p>The Norms metadata or .nvm file.</p>
- *   <p>For each norms field, this stores metadata, such as the offset into the 
- *      Norms data (.nvd)</p>
- *   <p>Norms metadata (.dvm) --&gt; Header,&lt;Entry&gt;<sup>NumFields</sup>,Footer</p>
- *   <ul>
- *     <li>Header --&gt; {@link CodecUtil#writeIndexHeader IndexHeader}</li>
- *     <li>Entry --&gt; FieldNumber,BytesPerValue, Address</li>
- *     <li>FieldNumber --&gt; {@link DataOutput#writeVInt vInt}</li>
- *     <li>BytesPerValue --&gt; {@link DataOutput#writeByte byte}</li>
- *     <li>Offset --&gt; {@link DataOutput#writeLong Int64}</li>
- *     <li>Footer --&gt; {@link CodecUtil#writeFooter CodecFooter}</li>
- *   </ul>
- *   <p>FieldNumber of -1 indicates the end of metadata.</p>
- *   <p>Offset is the pointer to the start of the data in the norms data (.nvd), or the singleton value 
- *      when BytesPerValue = 0</p>
- *   <li><a name="nvd"></a>
- *   <p>The Norms data or .nvd file.</p>
- *   <p>For each Norms field, this stores the actual per-document data (the heavy-lifting)</p>
- *   <p>Norms data (.nvd) --&gt; Header,&lt; Data &gt;<sup>NumFields</sup>,Footer</p>
- *   <ul>
- *     <li>Header --&gt; {@link CodecUtil#writeIndexHeader IndexHeader}</li>
- *     <li>Data --&gt; {@link DataOutput#writeByte(byte) byte}<sup>MaxDoc * BytesPerValue</sup></li>
- *     <li>Footer --&gt; {@link CodecUtil#writeFooter CodecFooter}</li>
- *   </ul>
- * </ol>
- * @lucene.experimental
- */
-public class Lucene53NormsFormat extends NormsFormat {
-
-  /** Sole Constructor */
-  public Lucene53NormsFormat() {}
-  
-  @Override
-  public NormsConsumer normsConsumer(SegmentWriteState state) throws IOException {
-    throw new UnsupportedOperationException("This format can only be used for reading");
-  }
-
-  @Override
-  public NormsProducer normsProducer(SegmentReadState state) throws IOException {
-    return new Lucene53NormsProducer(state, DATA_CODEC, DATA_EXTENSION, METADATA_CODEC, METADATA_EXTENSION);
-  }
-  
-  static final String DATA_CODEC = "Lucene53NormsData";
-  static final String DATA_EXTENSION = "nvd";
-  static final String METADATA_CODEC = "Lucene53NormsMetadata";
-  static final String METADATA_EXTENSION = "nvm";
-  static final int VERSION_START = 0;
-  static final int VERSION_CURRENT = VERSION_START;
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/189f2b3f/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene53/Lucene53NormsProducer.java
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene53/Lucene53NormsProducer.java b/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene53/Lucene53NormsProducer.java
deleted file mode 100644
index 718fcd6..0000000
--- a/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene53/Lucene53NormsProducer.java
+++ /dev/null
@@ -1,236 +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.lucene53;
-
-
-import java.io.IOException;
-import java.util.HashMap;
-import java.util.Map;
-
-import org.apache.lucene.codecs.CodecUtil;
-import org.apache.lucene.codecs.NormsProducer;
-import org.apache.lucene.index.CorruptIndexException;
-import org.apache.lucene.index.FieldInfo;
-import org.apache.lucene.index.FieldInfos;
-import org.apache.lucene.index.IndexFileNames;
-import org.apache.lucene.index.NumericDocValues;
-import org.apache.lucene.index.SegmentReadState;
-import org.apache.lucene.store.ChecksumIndexInput;
-import org.apache.lucene.store.IndexInput;
-import org.apache.lucene.store.RandomAccessInput;
-import org.apache.lucene.util.IOUtils;
-
-import static org.apache.lucene.codecs.lucene53.Lucene53NormsFormat.VERSION_CURRENT;
-import static org.apache.lucene.codecs.lucene53.Lucene53NormsFormat.VERSION_START;
-
-/**
- * Reader for {@link Lucene53NormsFormat}
- */
-class Lucene53NormsProducer extends NormsProducer {
-  // metadata maps (just file pointers and minimal stuff)
-  private final Map<Integer,NormsEntry> norms = new HashMap<>();
-  private final IndexInput data;
-  private final int maxDoc;
-
-  Lucene53NormsProducer(SegmentReadState state, String dataCodec, String dataExtension, String metaCodec, String metaExtension) throws IOException {
-    maxDoc = state.segmentInfo.maxDoc();
-    String metaName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, metaExtension);
-    int version = -1;
-
-    // read in the entries from the metadata file.
-    try (ChecksumIndexInput in = state.directory.openChecksumInput(metaName, state.context)) {
-      Throwable priorE = null;
-      try {
-        version = CodecUtil.checkIndexHeader(in, metaCodec, VERSION_START, VERSION_CURRENT, state.segmentInfo.getId(), state.segmentSuffix);
-        readFields(in, state.fieldInfos);
-      } catch (Throwable exception) {
-        priorE = exception;
-      } finally {
-        CodecUtil.checkFooter(in, priorE);
-      }
-    }
-
-    String dataName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, dataExtension);
-    data = state.directory.openInput(dataName, state.context);
-    boolean success = false;
-    try {
-      final int version2 = CodecUtil.checkIndexHeader(data, dataCodec, VERSION_START, VERSION_CURRENT, state.segmentInfo.getId(), state.segmentSuffix);
-      if (version != version2) {
-        throw new CorruptIndexException("Format versions mismatch: meta=" + version + ",data=" + version2, data);
-      }
-
-      // NOTE: data file is too costly to verify checksum against all the bytes on open,
-      // but for now we at least verify proper structure of the checksum footer: which looks
-      // for FOOTER_MAGIC + algorithmID. This is cheap and can detect some forms of corruption
-      // such as file truncation.
-      CodecUtil.retrieveChecksum(data);
-
-      success = true;
-    } finally {
-      if (!success) {
-        IOUtils.closeWhileHandlingException(this.data);
-      }
-    }
-  }
-
-  private void readFields(IndexInput meta, FieldInfos infos) throws IOException {
-    int fieldNumber = meta.readVInt();
-    while (fieldNumber != -1) {
-      FieldInfo info = infos.fieldInfo(fieldNumber);
-      if (info == null) {
-        throw new CorruptIndexException("Invalid field number: " + fieldNumber, meta);
-      } else if (!info.hasNorms()) {
-        throw new CorruptIndexException("Invalid field: " + info.name, meta);
-      }
-      NormsEntry entry = new NormsEntry();
-      entry.bytesPerValue = meta.readByte();
-      switch (entry.bytesPerValue) {
-        case 0: case 1: case 2: case 4: case 8:
-          break;
-        default:
-          throw new CorruptIndexException("Invalid bytesPerValue: " + entry.bytesPerValue + ", field: " + info.name, meta);
-      }
-      entry.offset = meta.readLong();
-      norms.put(info.number, entry);
-      fieldNumber = meta.readVInt();
-    }
-  }
-
-  @Override
-  public NumericDocValues getNorms(FieldInfo field) throws IOException {
-    final NormsEntry entry = norms.get(field.number);
-
-    if (entry.bytesPerValue == 0) {
-      final long value = entry.offset;
-      return new NormsIterator(maxDoc) {
-          @Override
-          public long longValue() {
-            return value;
-          }
-        };
-    } else {
-      RandomAccessInput slice;
-      synchronized (data) {
-        switch (entry.bytesPerValue) {
-        case 1: 
-          slice = data.randomAccessSlice(entry.offset, maxDoc);
-          return new NormsIterator(maxDoc) {
-            @Override
-            public long longValue() throws IOException {
-              return slice.readByte(docID);
-            }
-          };
-        case 2: 
-          slice = data.randomAccessSlice(entry.offset, maxDoc * 2L);
-          return new NormsIterator(maxDoc) {
-            @Override
-            public long longValue() throws IOException {
-              return slice.readShort(((long)docID) << 1L);
-            }
-          };
-        case 4: 
-          slice = data.randomAccessSlice(entry.offset, maxDoc * 4L);
-          return new NormsIterator(maxDoc) {
-            @Override
-            public long longValue() throws IOException {
-              return slice.readInt(((long)docID) << 2L);
-            }
-          };
-        case 8: 
-          slice = data.randomAccessSlice(entry.offset, maxDoc * 8L);
-          return new NormsIterator(maxDoc) {
-            @Override
-            public long longValue() throws IOException {
-              return slice.readLong(((long)docID) << 3L);
-            }
-          };
-        default:
-          throw new AssertionError();
-        }
-      }
-    }
-  }
-
-  @Override
-  public void close() throws IOException {
-    data.close();
-  }
-
-  @Override
-  public long ramBytesUsed() {
-    return 64L * norms.size(); // good enough
-  }
-
-  @Override
-  public void checkIntegrity() throws IOException {
-    CodecUtil.checksumEntireFile(data);
-  }
-
-  static class NormsEntry {
-    byte bytesPerValue;
-    long offset;
-  }
-
-  @Override
-  public String toString() {
-    return getClass().getSimpleName() + "(fields=" + norms.size() + ")";
-  }
-
-  private static abstract class NormsIterator extends NumericDocValues {
-    private final int maxDoc;
-    protected int docID = -1;
-  
-    public NormsIterator(int maxDoc) {
-      this.maxDoc = maxDoc;
-    }
-
-    @Override
-    public int docID() {
-      return docID;
-    }
-
-    @Override
-    public int nextDoc() {
-      docID++;
-      if (docID == maxDoc) {
-        docID = NO_MORE_DOCS;
-      }
-      return docID;
-    }
-
-    @Override
-    public int advance(int target) {
-      docID = target;
-      if (docID >= maxDoc) {
-        docID = NO_MORE_DOCS;
-      }
-      return docID;
-    }
-
-    @Override
-    public boolean advanceExact(int target) throws IOException {
-      docID = target;
-      return true;
-    }
-
-    @Override
-    public long cost() {
-      // TODO
-      return 0;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/189f2b3f/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene53/package-info.java
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene53/package-info.java b/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene53/package-info.java
deleted file mode 100644
index 93fefb8..0000000
--- a/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene53/package-info.java
+++ /dev/null
@@ -1,23 +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.
- */
-
-/**
- * Components from the Lucene 5.3 index format
- * See {@link org.apache.lucene.codecs.lucene53} for an overview
- * of the index format.
- */
-package org.apache.lucene.codecs.lucene53;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/189f2b3f/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene54/Lucene54DocValuesConsumer.java
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene54/Lucene54DocValuesConsumer.java b/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene54/Lucene54DocValuesConsumer.java
deleted file mode 100644
index 217cd43..0000000
--- a/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene54/Lucene54DocValuesConsumer.java
+++ /dev/null
@@ -1,797 +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.lucene54;
-
-
-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 java.util.stream.StreamSupport;
-
-import org.apache.lucene.codecs.CodecUtil;
-import org.apache.lucene.codecs.DocValuesConsumer;
-import org.apache.lucene.codecs.DocValuesProducer;
-import org.apache.lucene.codecs.LegacyDocValuesIterables;
-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.PagedBytesDataInput;
-import org.apache.lucene.util.PagedBytes;
-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 {
-
-  enum NumberType {
-    /** Dense ordinals */
-    ORDINAL,
-    /** Random long values */
-    VALUE;
-  }
-
-  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, DocValuesProducer valuesProducer) throws IOException {
-    addNumericField(field, LegacyDocValuesIterables.numericIterable(field, valuesProducer, maxDoc), NumberType.VALUE);
-  }
-
-  void addNumericField(FieldInfo field, Iterable<Number> values, NumberType numberType) 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;
-    long missingOrdCount = 0;
-    if (numberType == NumberType.VALUE) {
-      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();
-        if (v == -1L) {
-          missingOrdCount++;
-        }
-        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 boolean sparse; // 1% of docs or less have a value
-    switch (numberType) {
-      case VALUE:
-        sparse = (double) missingCount / count >= 0.99;
-        break;
-      case ORDINAL:
-        sparse = (double) missingOrdCount / count >= 0.99;
-        break;
-      default:
-        throw new AssertionError();
-    }
-
-    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 (sparse && count >= 1024) {
-      // require at least 1024 docs to avoid flipping back and forth when doing NRT search
-      format = SPARSE_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 (format == SPARSE_COMPRESSED) {
-      meta.writeLong(data.getFilePointer());
-      final long numDocsWithValue;
-      switch (numberType) {
-        case VALUE:
-          numDocsWithValue = count - missingCount;
-          break;
-        case ORDINAL:
-          numDocsWithValue = count - missingOrdCount;
-          break;
-        default:
-          throw new AssertionError();
-      }
-      final long maxDoc = writeSparseMissingBitset(values, numberType, numDocsWithValue);
-      assert maxDoc == count;
-    } else 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;
-      case SPARSE_COMPRESSED:
-        final Iterable<Number> filteredMissingValues;
-        switch (numberType) {
-          case VALUE:
-            meta.writeByte((byte) 0);
-            filteredMissingValues = new Iterable<Number>() {
-              @Override
-              public Iterator<Number> iterator() {
-                return StreamSupport
-                    .stream(values.spliterator(), false)
-                    .filter(value -> value != null)
-                    .iterator();
-              }
-            };
-            break;
-          case ORDINAL:
-            meta.writeByte((byte) 1);
-            filteredMissingValues = new Iterable<Number>() {
-              @Override
-              public Iterator<Number> iterator() {
-                return StreamSupport
-                    .stream(values.spliterator(), false)
-                    .filter(value -> value.longValue() != -1L)
-                    .iterator();
-              }
-            };
-            break;
-          default:
-            throw new AssertionError();
-        }
-        // Write non-missing values as a numeric field
-        addNumericField(field, filteredMissingValues, numberType);
-        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);
-    }
-  }
-
-  long writeSparseMissingBitset(Iterable<Number> values, NumberType numberType, long numDocsWithValue) throws IOException {
-    meta.writeVLong(numDocsWithValue);
-
-    // Write doc IDs that have a value
-    meta.writeVInt(DIRECT_MONOTONIC_BLOCK_SHIFT);
-    final DirectMonotonicWriter docIdsWriter = DirectMonotonicWriter.getInstance(meta, data, numDocsWithValue, DIRECT_MONOTONIC_BLOCK_SHIFT);
-    long docID = 0;
-    for (Number nv : values) {
-      switch (numberType) {
-        case VALUE:
-          if (nv != null) {
-            docIdsWriter.add(docID);
-          }
-          break;
-        case ORDINAL:
-          if (nv.longValue() != -1L) {
-            docIdsWriter.add(docID);
-          }
-          break;
-        default:
-          throw new AssertionError();
-      }
-      docID++;
-    }
-    docIdsWriter.finish();
-    return docID;
-  }
-
-  @Override
-  public void addBinaryField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException {
-    addBinaryField(field, LegacyDocValuesIterables.binaryIterable(field, valuesProducer, maxDoc));
-  }
-  
-  private 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, and compressibility if so
-    int minLength = Integer.MAX_VALUE;
-    int maxLength = Integer.MIN_VALUE;
-    long numValues = 0;
-    BytesRefBuilder previousValue = new BytesRefBuilder();
-    long prefixSum = 0; // only valid for fixed-width data, as we have a choice there
-    for (BytesRef v : values) {
-      minLength = Math.min(minLength, v.length);
-      maxLength = Math.max(maxLength, v.length);
-      if (minLength == maxLength) {
-        int termPosition = (int) (numValues & INTERVAL_MASK);
-        if (termPosition == 0) {
-          // first term in block, save it away to compare against the last term later
-          previousValue.copyBytes(v);
-        } else if (termPosition == INTERVAL_COUNT - 1) {
-          // last term in block, accumulate shared prefix against first term
-          prefixSum += StringHelper.bytesDifference(previousValue.get(), v);
-        }
-      }
-      numValues++;
-    }
-    // for fixed width data, look at the avg(shared prefix) before deciding how to encode:
-    // prefix compression "costs" worst case 2 bytes per term because we must store suffix lengths.
-    // so if we share at least 3 bytes on average, always compress.
-    if (minLength == maxLength && prefixSum <= 3*(numValues >> INTERVAL_SHIFT)) {
-      // no index needed: not very compressible, 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, DocValuesProducer valuesProducer) throws IOException {
-    meta.writeVInt(field.number);
-    meta.writeByte(Lucene54DocValuesFormat.SORTED);
-    addTermsDict(field, LegacyDocValuesIterables.valuesIterable(valuesProducer.getSorted(field)));
-    addNumericField(field, LegacyDocValuesIterables.sortedOrdIterable(valuesProducer, field, maxDoc), NumberType.ORDINAL);
-  }
-
-  private void addSortedField(FieldInfo field, Iterable<BytesRef> values, Iterable<Number> ords) throws IOException {
-    meta.writeVInt(field.number);
-    meta.writeByte(Lucene54DocValuesFormat.SORTED);
-    addTermsDict(field, values);
-    addNumericField(field, ords, NumberType.ORDINAL);
-  }
-
-  @Override
-  public void addSortedNumericField(FieldInfo field, final DocValuesProducer valuesProducer) throws IOException {
-
-    final Iterable<Number> docToValueCount = LegacyDocValuesIterables.sortedNumericToDocCount(valuesProducer, field, maxDoc);
-    final Iterable<Number> values = LegacyDocValuesIterables.sortedNumericToValues(valuesProducer, field);
-    
-    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), NumberType.VALUE);
-    } 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), NumberType.ORDINAL);
-      } else {
-        meta.writeVInt(SORTED_WITH_ADDRESSES);
-        // write the stream of values as a numeric field
-        addNumericField(field, values, NumberType.VALUE);
-        // write the doc -> ord count as a absolute index to the stream
-        addOrdIndex(field, docToValueCount);
-      }
-    }
-  }
-
-  @Override
-  public void addSortedSetField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException {
-
-    Iterable<BytesRef> values = LegacyDocValuesIterables.valuesIterable(valuesProducer.getSortedSet(field));
-    Iterable<Number> docToOrdCount = LegacyDocValuesIterables.sortedSetOrdCountIterable(valuesProducer, field, maxDoc);
-    Iterable<Number> ords = LegacyDocValuesIterables.sortedSetOrdsIterable(valuesProducer, field);
-
-    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), NumberType.ORDINAL);
-      } 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, NumberType.ORDINAL);
-
-        // 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;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/189f2b3f/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene54/Lucene54DocValuesFormat.java
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene54/Lucene54DocValuesFormat.java b/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene54/Lucene54DocValuesFormat.java
deleted file mode 100644
index 91ccfe2..0000000
--- a/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene54/Lucene54DocValuesFormat.java
+++ /dev/null
@@ -1,186 +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.lucene54;
-
-
-import java.io.IOException;
-
-import org.apache.lucene.codecs.DocValuesConsumer;
-import org.apache.lucene.codecs.DocValuesFormat;
-import org.apache.lucene.codecs.DocValuesProducer;
-import org.apache.lucene.codecs.lucene70.Lucene70DocValuesFormat;
-import org.apache.lucene.index.DocValuesType;
-import org.apache.lucene.index.SegmentReadState;
-import org.apache.lucene.index.SegmentWriteState;
-import org.apache.lucene.util.SmallFloat;
-import org.apache.lucene.util.packed.DirectWriter;
-
-/**
- * Lucene 5.4 DocValues format.
- * <p>
- * Encodes the five per-document value types (Numeric,Binary,Sorted,SortedSet,SortedNumeric) with these strategies:
- * <p>
- * {@link DocValuesType#NUMERIC NUMERIC}:
- * <ul>
- *    <li>Delta-compressed: per-document integers written as deltas from the minimum value,
- *        compressed with bitpacking. For more information, see {@link DirectWriter}.
- *    <li>Table-compressed: when the number of unique values is very small (&lt; 256), and
- *        when there are unused "gaps" in the range of values used (such as {@link SmallFloat}), 
- *        a lookup table is written instead. Each per-document entry is instead the ordinal 
- *        to this table, and those ordinals are compressed with bitpacking ({@link DirectWriter}). 
- *    <li>GCD-compressed: when all numbers share a common divisor, such as dates, the greatest
- *        common denominator (GCD) is computed, and quotients are stored using Delta-compressed Numerics.
- *    <li>Monotonic-compressed: when all numbers are monotonically increasing offsets, they are written
- *        as blocks of bitpacked integers, encoding the deviation from the expected delta.
- *    <li>Const-compressed: when there is only one possible non-missing value, only the missing
- *        bitset is encoded.
- *    <li>Sparse-compressed: only documents with a value are stored, and lookups are performed
- *        using binary search.
- * </ul>
- * <p>
- * {@link DocValuesType#BINARY BINARY}:
- * <ul>
- *    <li>Fixed-width Binary: one large concatenated byte[] is written, along with the fixed length.
- *        Each document's value can be addressed directly with multiplication ({@code docID * length}). 
- *    <li>Variable-width Binary: one large concatenated byte[] is written, along with end addresses 
- *        for each document. The addresses are written as Monotonic-compressed numerics.
- *    <li>Prefix-compressed Binary: values are written in chunks of 16, with the first value written
- *        completely and other values sharing prefixes. chunk addresses are written as Monotonic-compressed
- *        numerics. A reverse lookup index is written from a portion of every 1024th term.
- * </ul>
- * <p>
- * {@link DocValuesType#SORTED SORTED}:
- * <ul>
- *    <li>Sorted: a mapping of ordinals to deduplicated terms is written as Binary, 
- *        along with the per-document ordinals written using one of the numeric strategies above.
- * </ul>
- * <p>
- * {@link DocValuesType#SORTED_SET SORTED_SET}:
- * <ul>
- *    <li>Single: if all documents have 0 or 1 value, then data are written like SORTED.
- *    <li>SortedSet table: when there are few unique sets of values (&lt; 256) then each set is assigned
- *        an id, a lookup table is written and the mapping from document to set id is written using the
- *        numeric strategies above.
- *    <li>SortedSet: a mapping of ordinals to deduplicated terms is written as Binary, 
- *        an ordinal list and per-document index into this list are written using the numeric strategies 
- *        above.
- * </ul>
- * <p>
- * {@link DocValuesType#SORTED_NUMERIC SORTED_NUMERIC}:
- * <ul>
- *    <li>Single: if all documents have 0 or 1 value, then data are written like NUMERIC.
- *    <li>SortedSet table: when there are few unique sets of values (&lt; 256) then each set is assigned
- *        an id, a lookup table is written and the mapping from document to set id is written using the
- *        numeric strategies above.
- *    <li>SortedNumeric: a value list and per-document index into this list are written using the numeric
- *        strategies above.
- * </ul>
- * <p>
- * Files:
- * <ol>
- *   <li><tt>.dvd</tt>: DocValues data</li>
- *   <li><tt>.dvm</tt>: DocValues metadata</li>
- * </ol>
- * @lucene.experimental
- * @deprecated Use {@link Lucene70DocValuesFormat}.
- */
-@Deprecated
-public final class Lucene54DocValuesFormat extends DocValuesFormat {
-
-  /** Sole Constructor */
-  public Lucene54DocValuesFormat() {
-    super("Lucene54");
-  }
-
-  @Override
-  public DocValuesConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
-    return new Lucene54DocValuesConsumer(state, DATA_CODEC, DATA_EXTENSION, META_CODEC, META_EXTENSION);
-  }
-
-  @Override
-  public DocValuesProducer fieldsProducer(SegmentReadState state) throws IOException {
-    return new Lucene54DocValuesProducer(state, DATA_CODEC, DATA_EXTENSION, META_CODEC, META_EXTENSION);
-  }
-  
-  static final String DATA_CODEC = "Lucene54DocValuesData";
-  static final String DATA_EXTENSION = "dvd";
-  static final String META_CODEC = "Lucene54DocValuesMetadata";
-  static final String META_EXTENSION = "dvm";
-  static final int VERSION_START = 0;
-  static final int VERSION_CURRENT = VERSION_START;
-  
-  // indicates docvalues type
-  static final byte NUMERIC = 0;
-  static final byte BINARY = 1;
-  static final byte SORTED = 2;
-  static final byte SORTED_SET = 3;
-  static final byte SORTED_NUMERIC = 4;
-  
-  // address terms in blocks of 16 terms
-  static final int INTERVAL_SHIFT = 4;
-  static final int INTERVAL_COUNT = 1 << INTERVAL_SHIFT;
-  static final int INTERVAL_MASK = INTERVAL_COUNT - 1;
-  
-  // build reverse index from every 1024th term
-  static final int REVERSE_INTERVAL_SHIFT = 10;
-  static final int REVERSE_INTERVAL_COUNT = 1 << REVERSE_INTERVAL_SHIFT;
-  static final int REVERSE_INTERVAL_MASK = REVERSE_INTERVAL_COUNT - 1;
-  
-  // for conversion from reverse index to block
-  static final int BLOCK_INTERVAL_SHIFT = REVERSE_INTERVAL_SHIFT - INTERVAL_SHIFT;
-  static final int BLOCK_INTERVAL_COUNT = 1 << BLOCK_INTERVAL_SHIFT;
-  static final int BLOCK_INTERVAL_MASK = BLOCK_INTERVAL_COUNT - 1;
-
-  /** Compressed using packed blocks of ints. */
-  static final int DELTA_COMPRESSED = 0;
-  /** Compressed by computing the GCD. */
-  static final int GCD_COMPRESSED = 1;
-  /** Compressed by giving IDs to unique values. */
-  static final int TABLE_COMPRESSED = 2;
-  /** Compressed with monotonically increasing values */
-  static final int MONOTONIC_COMPRESSED = 3;
-  /** Compressed with constant value (uses only missing bitset) */
-  static final int CONST_COMPRESSED = 4;
-  /** Compressed with sparse arrays. */
-  static final int SPARSE_COMPRESSED = 5;
-
-  /** Uncompressed binary, written directly (fixed length). */
-  static final int BINARY_FIXED_UNCOMPRESSED = 0;
-  /** Uncompressed binary, written directly (variable length). */
-  static final int BINARY_VARIABLE_UNCOMPRESSED = 1;
-  /** Compressed binary with shared prefixes */
-  static final int BINARY_PREFIX_COMPRESSED = 2;
-
-  /** Standard storage for sorted set values with 1 level of indirection:
-   *  {@code docId -> address -> ord}. */
-  static final int SORTED_WITH_ADDRESSES = 0;
-  /** Single-valued sorted set values, encoded as sorted values, so no level
-   *  of indirection: {@code docId -> ord}. */
-  static final int SORTED_SINGLE_VALUED = 1;
-  /** Compressed giving IDs to unique sets of values:
-   * {@code docId -> setId -> ords} */
-  static final int SORTED_SET_TABLE = 2;
-  
-  /** placeholder for missing offset that means there are no missing values */
-  static final int ALL_LIVE = -1;
-  /** placeholder for missing offset that means all values are missing */
-  static final int ALL_MISSING = -2;
-  
-  // addressing uses 16k blocks
-  static final int MONOTONIC_BLOCK_SIZE = 16384;
-  static final int DIRECT_MONOTONIC_BLOCK_SHIFT = 16;
-}


[23/23] lucene-solr:jira/solr-10879: Update CHANGES.

Posted by ab...@apache.org.
Update CHANGES.


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

Branch: refs/heads/jira/solr-10879
Commit: c80c8cc59e5dc354d07be607bb9ea10defb28327
Parents: c751fc9
Author: Andrzej Bialecki <ab...@apache.org>
Authored: Tue Jul 4 18:37:02 2017 +0200
Committer: Andrzej Bialecki <ab...@apache.org>
Committed: Tue Jul 4 18:37:02 2017 +0200

----------------------------------------------------------------------
 solr/CHANGES.txt | 3 +++
 1 file changed, 3 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c80c8cc5/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 403b6cd..675aead 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -301,6 +301,9 @@ Bug Fixes
 
 * SOLR-10878: MOVEREPLICA command may lose data when replicationFactor is 1. (ab, shalin)
 
+* SOLR-10879: DELETEREPLICA and DELETENODE commands should prevent data loss when
+  replicationFactor is 1. (ab)
+
 Optimizations
 ----------------------
 


[17/23] lucene-solr:jira/solr-10879: Merge branch 'upgrade-master-to-8' of https://github.com/anshumg/lucene-solr for 7.0 release ( closes #119 )

Posted by ab...@apache.org.
Merge branch 'upgrade-master-to-8' of https://github.com/anshumg/lucene-solr for 7.0 release ( closes #119 )


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

Branch: refs/heads/jira/solr-10879
Commit: 8218a5b2c6ca78c05a6060af4770c504e28a6f99
Parents: 68a46ac bc9c414
Author: Anshum Gupta <an...@apple.com>
Authored: Mon Jul 3 12:11:52 2017 -0700
Committer: Anshum Gupta <an...@apple.com>
Committed: Mon Jul 3 12:11:52 2017 -0700

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |    3 +
 .../analysis/custom/TestCustomAnalyzer.java     |    8 +-
 .../lucene50/Lucene50SegmentInfoFormat.java     |   90 -
 .../apache/lucene/codecs/lucene50/package.html  |   25 -
 .../codecs/lucene53/Lucene53NormsFormat.java    |   91 -
 .../codecs/lucene53/Lucene53NormsProducer.java  |  236 ---
 .../lucene/codecs/lucene53/package-info.java    |   23 -
 .../lucene54/Lucene54DocValuesConsumer.java     |  797 --------
 .../lucene54/Lucene54DocValuesFormat.java       |  186 --
 .../lucene54/Lucene54DocValuesProducer.java     | 1803 ------------------
 .../lucene/codecs/lucene54/package-info.java    |  403 ----
 .../lucene/codecs/lucene60/Lucene60Codec.java   |  177 --
 .../apache/lucene/codecs/lucene60/package.html  |   25 -
 .../lucene/codecs/lucene62/Lucene62Codec.java   |  176 --
 .../apache/lucene/codecs/lucene62/package.html  |   25 -
 .../apache/lucene/index/FixBrokenOffsets.java   |  138 --
 .../java/org/apache/lucene/index/package.html   |   27 -
 .../services/org.apache.lucene.codecs.Codec     |    2 -
 .../org.apache.lucene.codecs.DocValuesFormat    |    1 -
 .../lucene50/Lucene50RWSegmentInfoFormat.java   |  125 --
 .../lucene50/TestLucene50SegmentInfoFormat.java |   50 -
 .../codecs/lucene53/Lucene53NormsConsumer.java  |  159 --
 .../codecs/lucene53/Lucene53RWNormsFormat.java  |   31 -
 .../lucene53/TestLucene53NormsFormat.java       |   44 -
 .../lucene54/TestLucene54DocValuesFormat.java   |  640 -------
 .../lucene/codecs/lucene60/Lucene60RWCodec.java |   38 -
 .../lucene/codecs/lucene62/Lucene62RWCodec.java |   44 -
 .../lucene62/Lucene62RWSegmentInfoFormat.java   |  193 --
 .../lucene62/TestLucene62SegmentInfoFormat.java |   48 -
 .../index/TestBackwardsCompatibility.java       |   64 +-
 .../lucene/index/TestFixBrokenOffsets.java      |  112 --
 .../lucene/index/TestIndexWriterOnOldIndex.java |    7 +-
 .../lucene/index/TestManyPointsInOldIndex.java  |    1 +
 .../org/apache/lucene/index/dvupdates.6.0.0.zip |  Bin 3420 -> 0 bytes
 .../org/apache/lucene/index/empty.6.0.0.zip     |  Bin 225 -> 0 bytes
 .../org/apache/lucene/index/index.6.0.0-cfs.zip |  Bin 15807 -> 0 bytes
 .../apache/lucene/index/index.6.0.0-nocfs.zip   |  Bin 15806 -> 0 bytes
 .../org/apache/lucene/index/index.6.0.1-cfs.zip |  Bin 15820 -> 0 bytes
 .../apache/lucene/index/index.6.0.1-nocfs.zip   |  Bin 15823 -> 0 bytes
 .../org/apache/lucene/index/index.6.1.0-cfs.zip |  Bin 15803 -> 0 bytes
 .../apache/lucene/index/index.6.1.0-nocfs.zip   |  Bin 15829 -> 0 bytes
 .../org/apache/lucene/index/index.6.2.0-cfs.zip |  Bin 15880 -> 0 bytes
 .../apache/lucene/index/index.6.2.0-nocfs.zip   |  Bin 15867 -> 0 bytes
 .../org/apache/lucene/index/index.6.2.1-cfs.zip |  Bin 15851 -> 0 bytes
 .../apache/lucene/index/index.6.2.1-nocfs.zip   |  Bin 15845 -> 0 bytes
 .../org/apache/lucene/index/index.6.3.0-cfs.zip |  Bin 15869 -> 0 bytes
 .../apache/lucene/index/index.6.3.0-nocfs.zip   |  Bin 15876 -> 0 bytes
 .../org/apache/lucene/index/index.6.4.0-cfs.zip |  Bin 15829 -> 0 bytes
 .../apache/lucene/index/index.6.4.0-nocfs.zip   |  Bin 15831 -> 0 bytes
 .../org/apache/lucene/index/index.6.4.1-cfs.zip |  Bin 15848 -> 0 bytes
 .../apache/lucene/index/index.6.4.1-nocfs.zip   |  Bin 15838 -> 0 bytes
 .../org/apache/lucene/index/index.6.4.2-cfs.zip |  Bin 15856 -> 0 bytes
 .../apache/lucene/index/index.6.4.2-nocfs.zip   |  Bin 15886 -> 0 bytes
 .../org/apache/lucene/index/index.6.5.0-cfs.zip |  Bin 15832 -> 0 bytes
 .../apache/lucene/index/index.6.5.0-nocfs.zip   |  Bin 15836 -> 0 bytes
 .../org/apache/lucene/index/index.6.5.1-cfs.zip |  Bin 15842 -> 0 bytes
 .../apache/lucene/index/index.6.5.1-nocfs.zip   |  Bin 15827 -> 0 bytes
 .../org/apache/lucene/index/index.6.6.0-cfs.zip |  Bin 15875 -> 0 bytes
 .../apache/lucene/index/index.6.6.0-nocfs.zip   |  Bin 15883 -> 0 bytes
 .../lucene/index/index.630.brokenoffsets.zip    |  Bin 3203 -> 0 bytes
 .../lucene/index/index.single-empty-doc.630.zip |  Bin 1363 -> 0 bytes
 .../org/apache/lucene/index/manypointsindex.zip |  Bin 3739 -> 0 bytes
 .../org/apache/lucene/index/maxposindex.zip     |  Bin 1702 -> 0 bytes
 .../org/apache/lucene/index/moreterms.6.0.0.zip |  Bin 157215 -> 0 bytes
 .../org/apache/lucene/index/sorted.6.2.0.zip    |  Bin 157007 -> 0 bytes
 .../org/apache/lucene/index/sorted.6.2.1.zip    |  Bin 100974 -> 0 bytes
 .../org/apache/lucene/index/sorted.6.3.0.zip    |  Bin 71090 -> 0 bytes
 .../lucene/index/unsupported.6.0.0-cfs.zip      |  Bin 0 -> 15807 bytes
 .../lucene/index/unsupported.6.0.0-nocfs.zip    |  Bin 0 -> 15806 bytes
 .../lucene/index/unsupported.6.0.1-cfs.zip      |  Bin 0 -> 15820 bytes
 .../lucene/index/unsupported.6.0.1-nocfs.zip    |  Bin 0 -> 15823 bytes
 .../lucene/index/unsupported.6.1.0-cfs.zip      |  Bin 0 -> 15803 bytes
 .../lucene/index/unsupported.6.1.0-nocfs.zip    |  Bin 0 -> 15829 bytes
 .../lucene/index/unsupported.6.2.0-cfs.zip      |  Bin 0 -> 15880 bytes
 .../lucene/index/unsupported.6.2.0-nocfs.zip    |  Bin 0 -> 15867 bytes
 .../lucene/index/unsupported.6.2.1-cfs.zip      |  Bin 0 -> 15851 bytes
 .../lucene/index/unsupported.6.2.1-nocfs.zip    |  Bin 0 -> 15845 bytes
 .../lucene/index/unsupported.6.3.0-cfs.zip      |  Bin 0 -> 15869 bytes
 .../lucene/index/unsupported.6.3.0-nocfs.zip    |  Bin 0 -> 15876 bytes
 .../lucene/index/unsupported.6.4.0-cfs.zip      |  Bin 0 -> 15829 bytes
 .../lucene/index/unsupported.6.4.0-nocfs.zip    |  Bin 0 -> 15831 bytes
 .../lucene/index/unsupported.6.4.1-cfs.zip      |  Bin 0 -> 15848 bytes
 .../lucene/index/unsupported.6.4.1-nocfs.zip    |  Bin 0 -> 15838 bytes
 .../lucene/index/unsupported.6.4.2-cfs.zip      |  Bin 0 -> 15856 bytes
 .../lucene/index/unsupported.6.4.2-nocfs.zip    |  Bin 0 -> 15886 bytes
 .../lucene/index/unsupported.6.5.0-cfs.zip      |  Bin 0 -> 15832 bytes
 .../lucene/index/unsupported.6.5.0-nocfs.zip    |  Bin 0 -> 15836 bytes
 .../lucene/index/unsupported.6.5.1-cfs.zip      |  Bin 0 -> 15842 bytes
 .../lucene/index/unsupported.6.5.1-nocfs.zip    |  Bin 0 -> 15827 bytes
 .../lucene/index/unsupported.6.6.0-cfs.zip      |  Bin 0 -> 15875 bytes
 .../lucene/index/unsupported.6.6.0-nocfs.zip    |  Bin 0 -> 15883 bytes
 .../org/apache/lucene/index/CheckIndex.java     |   47 +-
 .../index/IndexFormatTooOldException.java       |    2 +-
 .../org/apache/lucene/index/IndexWriter.java    |    6 +-
 .../org/apache/lucene/index/SegmentInfos.java   |    2 +-
 .../search/similarities/BM25Similarity.java     |   38 +-
 .../search/similarities/SimilarityBase.java     |   25 +-
 .../search/similarities/TFIDFSimilarity.java    |   25 +-
 .../java/org/apache/lucene/util/Version.java    |  101 +-
 .../apache/lucene/index/TestSegmentInfos.java   |   14 +-
 .../search/similarities/TestBM25Similarity.java |   42 -
 .../similarities/TestClassicSimilarity.java     |   59 +-
 .../search/similarities/TestSimilarityBase.java |   39 -
 .../org/apache/lucene/util/TestVersion.java     |   71 +-
 .../java/org/apache/lucene/util/TestUtil.java   |    4 +-
 lucene/version.properties                       |    2 +-
 solr/CHANGES.txt                                |   17 +
 .../solr/schema/FieldTypePluginLoader.java      |    6 +-
 .../org/apache/solr/schema/IndexSchema.java     |    4 +-
 .../apache/solr/schema/IndexSchemaFactory.java  |    9 +-
 .../similarities/SchemaSimilarityFactory.java   |    4 +-
 .../configsets/_default/conf/solrconfig.xml     |    2 +-
 .../TestNonDefinedSimilarityFactory.java        |   10 -
 .../example-DIH/solr/atom/conf/solrconfig.xml   |    2 +-
 .../example-DIH/solr/db/conf/solrconfig.xml     |    2 +-
 .../example-DIH/solr/mail/conf/solrconfig.xml   |    2 +-
 .../example-DIH/solr/solr/conf/solrconfig.xml   |    2 +-
 .../example-DIH/solr/tika/conf/solrconfig.xml   |    2 +-
 solr/example/files/conf/solrconfig.xml          |    2 +-
 .../configsets/_default/conf/solrconfig.xml     |    2 +-
 .../conf/solrconfig.xml                         |    2 +-
 121 files changed, 179 insertions(+), 6158 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8218a5b2/lucene/CHANGES.txt
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8218a5b2/solr/CHANGES.txt
----------------------------------------------------------------------
diff --cc solr/CHANGES.txt
index fac639b,c65671a..5e1481f
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@@ -16,17 -16,25 +16,34 @@@ In this release, there is an example So
  servlet container in the directory named "example".
  See the Quick Start guide at http://lucene.apache.org/solr/quickstart.html
  
+ ==================  8.0.0 ==================
+ 
+ Consult the LUCENE_CHANGES.txt file for additional, low level, changes in this release.
+ 
+ Versions of Major Components
+ ---------------------
+ Apache Tika 1.13
+ Carrot2 3.15.0
+ Velocity 1.7 and Velocity Tools 2.0
+ Apache UIMA 2.3.1
+ Apache ZooKeeper 3.4.10
+ Jetty 9.3.14.v20161028
+ 
+ 
+ (No Changes)
+ 
+ 
  ==================  7.0.0 ==================
  
 +Versions of Major Components
 +---------------------
 +Apache Tika 1.13
 +Carrot2 3.15.0
 +Velocity 1.7 and Velocity Tools 2.0
 +Apache UIMA 2.3.1
 +Apache ZooKeeper 3.4.10
 +Jetty 9.3.14.v20161028
 +
  Upgrading from Solr 6.x
  ----------------------
  


[02/23] lucene-solr:jira/solr-10879: Solr 7.0 release, bumping up version to 8 on master

Posted by ab...@apache.org.
Solr 7.0 release, bumping up version to 8 on master


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

Branch: refs/heads/jira/solr-10879
Commit: 3b6d741016acc4a2f25a816fc56ba7486759968b
Parents: 2e1c506
Author: Anshum Gupta <an...@apple.com>
Authored: Fri Jun 30 10:40:27 2017 -0700
Committer: Anshum Gupta <an...@apple.com>
Committed: Fri Jun 30 10:40:27 2017 -0700

----------------------------------------------------------------------
 lucene/CHANGES.txt                                 |  3 +++
 .../src/java/org/apache/lucene/util/Version.java   | 15 +++++++++++----
 lucene/version.properties                          |  2 +-
 solr/CHANGES.txt                                   | 17 +++++++++++++++++
 .../example-DIH/solr/atom/conf/solrconfig.xml      |  2 +-
 .../example-DIH/solr/db/conf/solrconfig.xml        |  2 +-
 .../example-DIH/solr/mail/conf/solrconfig.xml      |  2 +-
 .../example-DIH/solr/solr/conf/solrconfig.xml      |  2 +-
 .../example-DIH/solr/tika/conf/solrconfig.xml      |  2 +-
 solr/example/files/conf/solrconfig.xml             |  2 +-
 .../solr/configsets/_default/conf/solrconfig.xml   |  2 +-
 .../conf/solrconfig.xml                            |  2 +-
 12 files changed, 40 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3b6d7410/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 909b6ce..a0a6815 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -3,6 +3,9 @@ Lucene Change Log
 For more information on past and future Lucene versions, please see:
 http://s.apache.org/luceneversions
 
+======================= Lucene 8.0.0 =======================
+(No Changes)
+
 ======================= Lucene 7.0.0 =======================
 
 New Features

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3b6d7410/lucene/core/src/java/org/apache/lucene/util/Version.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/util/Version.java b/lucene/core/src/java/org/apache/lucene/util/Version.java
index 94dcc8b..0b10d95 100644
--- a/lucene/core/src/java/org/apache/lucene/util/Version.java
+++ b/lucene/core/src/java/org/apache/lucene/util/Version.java
@@ -124,12 +124,19 @@ public final class Version {
 
   /**
    * Match settings and bugs in Lucene's 7.0.0 release.
-   *  <p>
-   *  Use this to get the latest &amp; greatest settings, bug
-   *  fixes, etc, for Lucene.
+   * @deprecated (8.0.0) Use latest
    */
+  @Deprecated
   public static final Version LUCENE_7_0_0 = new Version(7, 0, 0);
 
+  /**
+   * Match settings and bugs in Lucene's 8.0.0 release.
+   * <p>
+   * Use this to get the latest &amp; greatest settings, bug
+   * fixes, etc, for Lucene.
+   */
+  public static final Version LUCENE_8_0_0 = new Version(8, 0, 0);
+
   // To add a new version:
   //  * Only add above this comment
   //  * If the new version is the newest, change LATEST below and deprecate the previous LATEST
@@ -149,7 +156,7 @@ public final class Version {
    * some defaults may have changed and may break functionality 
    * in your application.
    */
-  public static final Version LATEST = LUCENE_7_0_0;
+  public static final Version LATEST = LUCENE_8_0_0;
 
   /**
    * Constant for backwards compatibility.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3b6d7410/lucene/version.properties
----------------------------------------------------------------------
diff --git a/lucene/version.properties b/lucene/version.properties
index 55730a2..98380f4 100644
--- a/lucene/version.properties
+++ b/lucene/version.properties
@@ -2,7 +2,7 @@
 
 # RELEASE MANAGER must change this file after creating a release and
 # enter new base version (format "x.y.z", no prefix/appendix): 
-version.base=7.0.0
+version.base=8.0.0
 
 # Other version property defaults, don't change:
 version.suffix=SNAPSHOT

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3b6d7410/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 0d2226f..2bb599a 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -16,6 +16,23 @@ In this release, there is an example Solr server including a bundled
 servlet container in the directory named "example".
 See the Quick Start guide at http://lucene.apache.org/solr/quickstart.html
 
+==================  8.0.0 ==================
+
+Consult the LUCENE_CHANGES.txt file for additional, low level, changes in this release.
+
+Versions of Major Components
+---------------------
+Apache Tika 1.13
+Carrot2 3.15.0
+Velocity 1.7 and Velocity Tools 2.0
+Apache UIMA 2.3.1
+Apache ZooKeeper 3.4.10
+Jetty 9.3.14.v20161028
+
+
+(No Changes)
+
+
 ==================  7.0.0 ==================
 
 Upgrading from Solr 6.x

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3b6d7410/solr/example/example-DIH/solr/atom/conf/solrconfig.xml
----------------------------------------------------------------------
diff --git a/solr/example/example-DIH/solr/atom/conf/solrconfig.xml b/solr/example/example-DIH/solr/atom/conf/solrconfig.xml
index 22005dd..2b44ca6 100644
--- a/solr/example/example-DIH/solr/atom/conf/solrconfig.xml
+++ b/solr/example/example-DIH/solr/atom/conf/solrconfig.xml
@@ -36,7 +36,7 @@
     that you fully re-index after changing this setting as it can
     affect both how text is indexed and queried.
   -->
-  <luceneMatchVersion>7.0.0</luceneMatchVersion>
+  <luceneMatchVersion>8.0.0</luceneMatchVersion>
 
   <lib dir="${solr.install.dir:../../../..}/dist/" regex="solr-dataimporthandler-.*\.jar"/>
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3b6d7410/solr/example/example-DIH/solr/db/conf/solrconfig.xml
----------------------------------------------------------------------
diff --git a/solr/example/example-DIH/solr/db/conf/solrconfig.xml b/solr/example/example-DIH/solr/db/conf/solrconfig.xml
index c03529e..aaef5b8 100644
--- a/solr/example/example-DIH/solr/db/conf/solrconfig.xml
+++ b/solr/example/example-DIH/solr/db/conf/solrconfig.xml
@@ -35,7 +35,7 @@
        that you fully re-index after changing this setting as it can
        affect both how text is indexed and queried.
   -->
-  <luceneMatchVersion>7.0.0</luceneMatchVersion>
+  <luceneMatchVersion>8.0.0</luceneMatchVersion>
 
   <!-- <lib/> directives can be used to instruct Solr to load any Jars
        identified and use them to resolve any "plugins" specified in

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3b6d7410/solr/example/example-DIH/solr/mail/conf/solrconfig.xml
----------------------------------------------------------------------
diff --git a/solr/example/example-DIH/solr/mail/conf/solrconfig.xml b/solr/example/example-DIH/solr/mail/conf/solrconfig.xml
index 28f415c..5bd9f7a 100644
--- a/solr/example/example-DIH/solr/mail/conf/solrconfig.xml
+++ b/solr/example/example-DIH/solr/mail/conf/solrconfig.xml
@@ -35,7 +35,7 @@
        that you fully re-index after changing this setting as it can
        affect both how text is indexed and queried.
   -->
-  <luceneMatchVersion>7.0.0</luceneMatchVersion>
+  <luceneMatchVersion>8.0.0</luceneMatchVersion>
 
   <!-- <lib/> directives can be used to instruct Solr to load any Jars
        identified and use them to resolve any "plugins" specified in

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3b6d7410/solr/example/example-DIH/solr/solr/conf/solrconfig.xml
----------------------------------------------------------------------
diff --git a/solr/example/example-DIH/solr/solr/conf/solrconfig.xml b/solr/example/example-DIH/solr/solr/conf/solrconfig.xml
index 7019be7..eea6999 100644
--- a/solr/example/example-DIH/solr/solr/conf/solrconfig.xml
+++ b/solr/example/example-DIH/solr/solr/conf/solrconfig.xml
@@ -35,7 +35,7 @@
        that you fully re-index after changing this setting as it can
        affect both how text is indexed and queried.
   -->
-  <luceneMatchVersion>7.0.0</luceneMatchVersion>
+  <luceneMatchVersion>8.0.0</luceneMatchVersion>
 
   <!-- <lib/> directives can be used to instruct Solr to load any Jars
        identified and use them to resolve any "plugins" specified in

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3b6d7410/solr/example/example-DIH/solr/tika/conf/solrconfig.xml
----------------------------------------------------------------------
diff --git a/solr/example/example-DIH/solr/tika/conf/solrconfig.xml b/solr/example/example-DIH/solr/tika/conf/solrconfig.xml
index 38d5d8b..65ed56f 100644
--- a/solr/example/example-DIH/solr/tika/conf/solrconfig.xml
+++ b/solr/example/example-DIH/solr/tika/conf/solrconfig.xml
@@ -36,7 +36,7 @@
    that you fully re-index after changing this setting as it can
    affect both how text is indexed and queried.
   -->
-  <luceneMatchVersion>7.0.0</luceneMatchVersion>
+  <luceneMatchVersion>8.0.0</luceneMatchVersion>
 
   <!-- Load Data Import Handler and Apache Tika (extraction) libraries -->
   <lib dir="${solr.install.dir:../../../..}/dist/" regex="solr-dataimporthandler-.*\.jar"/>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3b6d7410/solr/example/files/conf/solrconfig.xml
----------------------------------------------------------------------
diff --git a/solr/example/files/conf/solrconfig.xml b/solr/example/files/conf/solrconfig.xml
index 71fdd47..a2de3e4 100644
--- a/solr/example/files/conf/solrconfig.xml
+++ b/solr/example/files/conf/solrconfig.xml
@@ -35,7 +35,7 @@
        that you fully re-index after changing this setting as it can
        affect both how text is indexed and queried.
   -->
-  <luceneMatchVersion>7.0.0</luceneMatchVersion>
+  <luceneMatchVersion>8.0.0</luceneMatchVersion>
 
   <!-- <lib/> directives can be used to instruct Solr to load any Jars
        identified and use them to resolve any "plugins" specified in

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3b6d7410/solr/server/solr/configsets/_default/conf/solrconfig.xml
----------------------------------------------------------------------
diff --git a/solr/server/solr/configsets/_default/conf/solrconfig.xml b/solr/server/solr/configsets/_default/conf/solrconfig.xml
index f75303f..c885f06 100644
--- a/solr/server/solr/configsets/_default/conf/solrconfig.xml
+++ b/solr/server/solr/configsets/_default/conf/solrconfig.xml
@@ -35,7 +35,7 @@
        that you fully re-index after changing this setting as it can
        affect both how text is indexed and queried.
   -->
-  <luceneMatchVersion>7.0.0</luceneMatchVersion>
+  <luceneMatchVersion>8.0.0</luceneMatchVersion>
 
   <!-- <lib/> directives can be used to instruct Solr to load any Jars
        identified and use them to resolve any "plugins" specified in

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3b6d7410/solr/server/solr/configsets/sample_techproducts_configs/conf/solrconfig.xml
----------------------------------------------------------------------
diff --git a/solr/server/solr/configsets/sample_techproducts_configs/conf/solrconfig.xml b/solr/server/solr/configsets/sample_techproducts_configs/conf/solrconfig.xml
index 6d0ce68..8adf13e 100644
--- a/solr/server/solr/configsets/sample_techproducts_configs/conf/solrconfig.xml
+++ b/solr/server/solr/configsets/sample_techproducts_configs/conf/solrconfig.xml
@@ -35,7 +35,7 @@
        that you fully re-index after changing this setting as it can
        affect both how text is indexed and queried.
   -->
-  <luceneMatchVersion>7.0.0</luceneMatchVersion>
+  <luceneMatchVersion>8.0.0</luceneMatchVersion>
 
   <!-- <lib/> directives can be used to instruct Solr to load any Jars
        identified and use them to resolve any "plugins" specified in


[12/23] lucene-solr:jira/solr-10879: SOLR-10565 Use unique scope and domain names in this test to avoid collisions.

Posted by ab...@apache.org.
SOLR-10565 Use unique scope and domain names in this test to avoid collisions.


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

Branch: refs/heads/jira/solr-10879
Commit: 5966f7566390d00643024dd3344de79695f984e0
Parents: 706d201
Author: Andrzej Bialecki <ab...@apache.org>
Authored: Mon Jul 3 14:21:12 2017 +0200
Committer: Andrzej Bialecki <ab...@apache.org>
Committed: Mon Jul 3 14:21:12 2017 +0200

----------------------------------------------------------------------
 .../metrics/reporters/SolrJmxReporterTest.java    | 18 ++++++++++--------
 1 file changed, 10 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/5966f756/solr/core/src/test/org/apache/solr/metrics/reporters/SolrJmxReporterTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/metrics/reporters/SolrJmxReporterTest.java b/solr/core/src/test/org/apache/solr/metrics/reporters/SolrJmxReporterTest.java
index f9a32c0..11cdf37 100644
--- a/solr/core/src/test/org/apache/solr/metrics/reporters/SolrJmxReporterTest.java
+++ b/solr/core/src/test/org/apache/solr/metrics/reporters/SolrJmxReporterTest.java
@@ -49,6 +49,7 @@ public class SolrJmxReporterTest extends SolrTestCaseJ4 {
   private static final int MAX_ITERATIONS = 20;
 
   private static int jmxPort;
+  private static String PREFIX;
 
   private String domain;
 
@@ -64,6 +65,7 @@ public class SolrJmxReporterTest extends SolrTestCaseJ4 {
     jmxPort = getNextAvailablePort();
     assertFalse(jmxPort == -1);
     LocateRegistry.createRegistry(jmxPort);
+    PREFIX = getSimpleClassName() + "-";
   }
 
   @Before
@@ -72,7 +74,7 @@ public class SolrJmxReporterTest extends SolrTestCaseJ4 {
 
     final SolrCore core = h.getCore();
     domain = core.getName();
-    rootName = TestUtil.randomSimpleString(random(), 5, 10);
+    rootName = PREFIX + TestUtil.randomSimpleString(random(), 5, 10);
 
     coreMetricManager = core.getCoreMetricManager();
     metricManager = core.getCoreContainer().getMetricManager();
@@ -95,7 +97,7 @@ public class SolrJmxReporterTest extends SolrTestCaseJ4 {
   private PluginInfo createReporterPluginInfo(String rootName, boolean enabled) {
     Random random = random();
     String className = SolrJmxReporter.class.getName();
-    String reporterName = TestUtil.randomSimpleString(random, 5, 10);
+    String reporterName = PREFIX + TestUtil.randomSimpleString(random, 5, 10);
 
     Map<String, Object> attrs = new HashMap<>();
     attrs.put(FieldType.CLASS_NAME, className);
@@ -106,7 +108,7 @@ public class SolrJmxReporterTest extends SolrTestCaseJ4 {
 
     boolean shouldOverrideDomain = random.nextBoolean();
     if (shouldOverrideDomain) {
-      domain = TestUtil.randomSimpleString(random);
+      domain = PREFIX + TestUtil.randomSimpleString(random);
       attrs.put("domain", domain);
     }
 
@@ -129,7 +131,7 @@ public class SolrJmxReporterTest extends SolrTestCaseJ4 {
     Random random = random();
 
     Map<String, Counter> registered = new HashMap<>();
-    String scope = SolrMetricTestUtils.getRandomScope(random, true);
+    String scope = PREFIX + SolrMetricTestUtils.getRandomScope(random, true);
     SolrInfoBean.Category category = SolrMetricTestUtils.getRandomCategory(random, true);
 
     int iterations = TestUtil.nextInt(random, 0, MAX_ITERATIONS);
@@ -150,7 +152,7 @@ public class SolrJmxReporterTest extends SolrTestCaseJ4 {
   public void testReloadCore() throws Exception {
     Random random = random();
 
-    String scope = SolrMetricTestUtils.getRandomScope(random, true);
+    String scope = PREFIX + SolrMetricTestUtils.getRandomScope(random, true);
     SolrInfoBean.Category category = SolrMetricTestUtils.getRandomCategory(random, true);
     Map<String, Counter> metrics = SolrMetricTestUtils.getRandomMetrics(random, true);
     SolrMetricProducer producer = SolrMetricTestUtils.getProducerOf(metricManager, category, scope, metrics);
@@ -174,12 +176,12 @@ public class SolrJmxReporterTest extends SolrTestCaseJ4 {
 
   @Test
   public void testEnabled() throws Exception {
-    String root1 = TestUtil.randomSimpleString(random(), 5, 10);
+    String root1 = PREFIX + TestUtil.randomSimpleString(random(), 5, 10);
     PluginInfo pluginInfo1 = createReporterPluginInfo(root1, true);
     metricManager.loadReporter(coreMetricManager.getRegistryName(), coreMetricManager.getCore().getResourceLoader(),
         pluginInfo1, coreMetricManager.getTag());
 
-    String root2 = TestUtil.randomSimpleString(random(), 5, 10);
+    String root2 = PREFIX + TestUtil.randomSimpleString(random(), 5, 10);
     assertFalse(root2.equals(root1));
     PluginInfo pluginInfo2 = createReporterPluginInfo(root2, false);
     metricManager.loadReporter(coreMetricManager.getRegistryName(), coreMetricManager.getCore().getResourceLoader(),
@@ -189,7 +191,7 @@ public class SolrJmxReporterTest extends SolrTestCaseJ4 {
     assertTrue(reporters.containsKey(pluginInfo1.name + "@" + coreMetricManager.getTag()));
     assertTrue(reporters.containsKey(pluginInfo2.name + "@" + coreMetricManager.getTag()));
 
-    String scope = SolrMetricTestUtils.getRandomScope(random(), true);
+    String scope = PREFIX + SolrMetricTestUtils.getRandomScope(random(), true);
     SolrInfoBean.Category category = SolrMetricTestUtils.getRandomCategory(random(), true);
     Map<String, Counter> metrics = SolrMetricTestUtils.getRandomMetrics(random(), true);
     SolrMetricProducer producer = SolrMetricTestUtils.getProducerOf(metricManager, category, scope, metrics);


[11/23] lucene-solr:jira/solr-10879: Remove unnecessary backward compatibility.

Posted by ab...@apache.org.
Remove unnecessary backward compatibility.


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

Branch: refs/heads/jira/solr-10879
Commit: bc9c4144dfbe209d8982f786f6265c9f8d0167fb
Parents: 9ca3dd2
Author: Adrien Grand <jp...@gmail.com>
Authored: Mon Jul 3 14:17:09 2017 +0200
Committer: Adrien Grand <jp...@gmail.com>
Committed: Mon Jul 3 14:17:09 2017 +0200

----------------------------------------------------------------------
 .../org/apache/lucene/index/CheckIndex.java     | 47 +++++++++-----------
 .../org/apache/lucene/index/IndexWriter.java    |  6 +--
 .../org/apache/lucene/index/SegmentInfos.java   |  2 +-
 .../search/similarities/BM25Similarity.java     | 38 ++++------------
 .../search/similarities/SimilarityBase.java     | 25 +++--------
 .../search/similarities/TFIDFSimilarity.java    | 25 ++---------
 .../java/org/apache/lucene/util/Version.java    | 13 ------
 .../search/similarities/TestBM25Similarity.java | 16 -------
 .../similarities/TestClassicSimilarity.java     | 33 +-------------
 .../search/similarities/TestSimilarityBase.java | 10 -----
 .../java/org/apache/lucene/util/TestUtil.java   |  4 +-
 .../solr/schema/FieldTypePluginLoader.java      |  6 +--
 .../org/apache/solr/schema/IndexSchema.java     |  4 +-
 .../apache/solr/schema/IndexSchemaFactory.java  |  9 +---
 .../similarities/SchemaSimilarityFactory.java   |  4 +-
 .../configsets/_default/conf/solrconfig.xml     |  2 +-
 .../TestNonDefinedSimilarityFactory.java        | 10 -----
 17 files changed, 54 insertions(+), 200 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bc9c4144/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java b/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
index e448d81..e605e98 100644
--- a/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
+++ b/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
@@ -744,13 +744,13 @@ public final class CheckIndex implements Closeable {
           segInfoStat.fieldNormStatus = testFieldNorms(reader, infoStream, failFast);
 
           // Test the Term Index
-          segInfoStat.termIndexStatus = testPostings(reader, infoStream, verbose, failFast, version);
+          segInfoStat.termIndexStatus = testPostings(reader, infoStream, verbose, failFast);
 
           // Test Stored Fields
           segInfoStat.storedFieldStatus = testStoredFields(reader, infoStream, failFast);
 
           // Test Term Vectors
-          segInfoStat.termVectorStatus = testTermVectors(reader, infoStream, verbose, crossCheckTermVectors, failFast, version);
+          segInfoStat.termVectorStatus = testTermVectors(reader, infoStream, verbose, crossCheckTermVectors, failFast);
 
           // Test Docvalues
           segInfoStat.docValuesStatus = testDocValues(reader, infoStream, failFast);
@@ -1209,7 +1209,7 @@ public final class CheckIndex implements Closeable {
    * checks Fields api is consistent with itself.
    * searcher is optional, to verify with queries. Can be null.
    */
-  private static Status.TermIndexStatus checkFields(Fields fields, Bits liveDocs, int maxDoc, FieldInfos fieldInfos, boolean doPrint, boolean isVectors, PrintStream infoStream, boolean verbose, Version version) throws IOException {
+  private static Status.TermIndexStatus checkFields(Fields fields, Bits liveDocs, int maxDoc, FieldInfos fieldInfos, boolean doPrint, boolean isVectors, PrintStream infoStream, boolean verbose) throws IOException {
     // TODO: we should probably return our own stats thing...?!
     long startNS;
     if (doPrint) {
@@ -1465,20 +1465,17 @@ public final class CheckIndex implements Closeable {
               if (hasOffsets) {
                 int startOffset = postings.startOffset();
                 int endOffset = postings.endOffset();
-                // In Lucene 7 we fixed IndexWriter to also enforce term vector offsets
-                if (isVectors == false || version.onOrAfter(Version.LUCENE_7_0_0)) {
-                  if (startOffset < 0) {
-                    throw new RuntimeException("term " + term + ": doc " + doc + ": pos " + pos + ": startOffset " + startOffset + " is out of bounds");
-                  }
-                  if (startOffset < lastOffset) {
-                    throw new RuntimeException("term " + term + ": doc " + doc + ": pos " + pos + ": startOffset " + startOffset + " < lastStartOffset " + lastOffset + "; consider using the FixBrokenOffsets tool in Lucene's backward-codecs module to correct your index");
-                  }
-                  if (endOffset < 0) {
-                    throw new RuntimeException("term " + term + ": doc " + doc + ": pos " + pos + ": endOffset " + endOffset + " is out of bounds");
-                  }
-                  if (endOffset < startOffset) {
-                    throw new RuntimeException("term " + term + ": doc " + doc + ": pos " + pos + ": endOffset " + endOffset + " < startOffset " + startOffset);
-                  }
+                if (startOffset < 0) {
+                  throw new RuntimeException("term " + term + ": doc " + doc + ": pos " + pos + ": startOffset " + startOffset + " is out of bounds");
+                }
+                if (startOffset < lastOffset) {
+                  throw new RuntimeException("term " + term + ": doc " + doc + ": pos " + pos + ": startOffset " + startOffset + " < lastStartOffset " + lastOffset + "; consider using the FixBrokenOffsets tool in Lucene's backward-codecs module to correct your index");
+                }
+                if (endOffset < 0) {
+                  throw new RuntimeException("term " + term + ": doc " + doc + ": pos " + pos + ": endOffset " + endOffset + " is out of bounds");
+                }
+                if (endOffset < startOffset) {
+                  throw new RuntimeException("term " + term + ": doc " + doc + ": pos " + pos + ": endOffset " + endOffset + " < startOffset " + startOffset);
                 }
                 lastOffset = startOffset;
               }
@@ -1745,15 +1742,15 @@ public final class CheckIndex implements Closeable {
    * Test the term index.
    * @lucene.experimental
    */
-  public static Status.TermIndexStatus testPostings(CodecReader reader, PrintStream infoStream, Version version) throws IOException {
-    return testPostings(reader, infoStream, false, false, version);
+  public static Status.TermIndexStatus testPostings(CodecReader reader, PrintStream infoStream) throws IOException {
+    return testPostings(reader, infoStream, false, false);
   }
   
   /**
    * Test the term index.
    * @lucene.experimental
    */
-  public static Status.TermIndexStatus testPostings(CodecReader reader, PrintStream infoStream, boolean verbose, boolean failFast, Version version) throws IOException {
+  public static Status.TermIndexStatus testPostings(CodecReader reader, PrintStream infoStream, boolean verbose, boolean failFast) throws IOException {
 
     // TODO: we should go and verify term vectors match, if
     // crossCheckTermVectors is on...
@@ -1768,7 +1765,7 @@ public final class CheckIndex implements Closeable {
 
       final Fields fields = reader.getPostingsReader().getMergeInstance();
       final FieldInfos fieldInfos = reader.getFieldInfos();
-      status = checkFields(fields, reader.getLiveDocs(), maxDoc, fieldInfos, true, false, infoStream, verbose, version);
+      status = checkFields(fields, reader.getLiveDocs(), maxDoc, fieldInfos, true, false, infoStream, verbose);
     } catch (Throwable e) {
       if (failFast) {
         throw IOUtils.rethrowAlways(e);
@@ -2377,15 +2374,15 @@ public final class CheckIndex implements Closeable {
    * Test term vectors.
    * @lucene.experimental
    */
-  public static Status.TermVectorStatus testTermVectors(CodecReader reader, PrintStream infoStream, Version version) throws IOException {
-    return testTermVectors(reader, infoStream, false, false, false, version);
+  public static Status.TermVectorStatus testTermVectors(CodecReader reader, PrintStream infoStream) throws IOException {
+    return testTermVectors(reader, infoStream, false, false, false);
   }
 
   /**
    * Test term vectors.
    * @lucene.experimental
    */
-  public static Status.TermVectorStatus testTermVectors(CodecReader reader, PrintStream infoStream, boolean verbose, boolean crossCheckTermVectors, boolean failFast, Version version) throws IOException {
+  public static Status.TermVectorStatus testTermVectors(CodecReader reader, PrintStream infoStream, boolean verbose, boolean crossCheckTermVectors, boolean failFast) throws IOException {
     long startNS = System.nanoTime();
     final Status.TermVectorStatus status = new Status.TermVectorStatus();
     final FieldInfos fieldInfos = reader.getFieldInfos();
@@ -2425,7 +2422,7 @@ public final class CheckIndex implements Closeable {
           
           if (tfv != null) {
             // First run with no deletions:
-            checkFields(tfv, null, 1, fieldInfos, false, true, infoStream, verbose, version);
+            checkFields(tfv, null, 1, fieldInfos, false, true, infoStream, verbose);
             
             // Only agg stats if the doc is live:
             final boolean doStats = liveDocs == null || liveDocs.get(j);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bc9c4144/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java b/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
index 26a25a5..c2cded7 100644
--- a/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
@@ -1170,9 +1170,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
     }
   }
 
-  /** Confirms that the incoming index sort (if any) matches the existing index sort (if any).
-   *  This is unfortunately just best effort, because it could be the old index only has unsorted flushed segments built
-   *  before {@link Version#LUCENE_6_5_0} (flushed segments are sorted in Lucene 7.0).  */
+  /** Confirms that the incoming index sort (if any) matches the existing index sort (if any).  */
   private void validateIndexSort() throws CorruptIndexException {
     Sort indexSort = config.getIndexSort();
     if (indexSort != null) {
@@ -1180,7 +1178,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
         Sort segmentIndexSort = info.info.getIndexSort();
         if (segmentIndexSort != null && indexSort.equals(segmentIndexSort) == false) {
           throw new IllegalArgumentException("cannot change previous indexSort=" + segmentIndexSort + " (from segment=" + info + ") to new indexSort=" + indexSort);
-        } else if (segmentIndexSort == null && info.info.getVersion().onOrAfter(Version.LUCENE_6_5_0)) {
+        } else if (segmentIndexSort == null) {
           // Flushed segments are not sorted if they were built with a version prior to 6.5.0
           throw new CorruptIndexException("segment not sorted with indexSort=" + segmentIndexSort, info.info.toString());
         }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bc9c4144/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java b/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java
index e463259..e3761cc 100644
--- a/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java
+++ b/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java
@@ -312,7 +312,7 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentCommitInfo
     CodecUtil.checkIndexHeaderSuffix(input, Long.toString(generation, Character.MAX_RADIX));
 
     Version luceneVersion = Version.fromBits(input.readVInt(), input.readVInt(), input.readVInt());
-    if (luceneVersion.onOrAfter(Version.LUCENE_6_0_0) == false) {
+    if (luceneVersion.onOrAfter(Version.LUCENE_7_0_0) == false) {
       // TODO: should we check indexCreatedVersion instead?
       throw new IndexFormatTooOldException(input, "this index is too old (version: " + luceneVersion + ")");
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bc9c4144/lucene/core/src/java/org/apache/lucene/search/similarities/BM25Similarity.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/similarities/BM25Similarity.java b/lucene/core/src/java/org/apache/lucene/search/similarities/BM25Similarity.java
index e693b2b..35554e2 100644
--- a/lucene/core/src/java/org/apache/lucene/search/similarities/BM25Similarity.java
+++ b/lucene/core/src/java/org/apache/lucene/search/similarities/BM25Similarity.java
@@ -118,16 +118,9 @@ public class BM25Similarity extends Similarity {
   }
   
   /** Cache of decoded bytes. */
-  private static final float[] OLD_LENGTH_TABLE = new float[256];
   private static final float[] LENGTH_TABLE = new float[256];
 
   static {
-    for (int i = 1; i < 256; i++) {
-      float f = SmallFloat.byte315ToFloat((byte)i);
-      OLD_LENGTH_TABLE[i] = 1.0f / (f*f);
-    }
-    OLD_LENGTH_TABLE[0] = 1.0f / OLD_LENGTH_TABLE[255]; // otherwise inf
-
     for (int i = 0; i < 256; i++) {
       LENGTH_TABLE[i] = SmallFloat.byte4ToInt((byte) i);
     }
@@ -137,12 +130,7 @@ public class BM25Similarity extends Similarity {
   @Override
   public final long computeNorm(FieldInvertState state) {
     final int numTerms = discountOverlaps ? state.getLength() - state.getNumOverlap() : state.getLength();
-    int indexCreatedVersionMajor = state.getIndexCreatedVersionMajor();
-    if (indexCreatedVersionMajor >= 7) {
-      return SmallFloat.intToByte4(numTerms);
-    } else {
-      return SmallFloat.floatToByte315((float) (1 / Math.sqrt(numTerms)));
-    }
+    return SmallFloat.intToByte4(numTerms);
   }
 
   /**
@@ -205,19 +193,17 @@ public class BM25Similarity extends Similarity {
     Explanation idf = termStats.length == 1 ? idfExplain(collectionStats, termStats[0]) : idfExplain(collectionStats, termStats);
     float avgdl = avgFieldLength(collectionStats);
 
-    float[] oldCache = new float[256];
     float[] cache = new float[256];
     for (int i = 0; i < cache.length; i++) {
-      oldCache[i] = k1 * ((1 - b) + b * OLD_LENGTH_TABLE[i] / avgdl);
       cache[i] = k1 * ((1 - b) + b * LENGTH_TABLE[i] / avgdl);
     }
-    return new BM25Stats(collectionStats.field(), boost, idf, avgdl, oldCache, cache);
+    return new BM25Stats(collectionStats.field(), boost, idf, avgdl, cache);
   }
 
   @Override
   public final SimScorer simScorer(SimWeight stats, LeafReaderContext context) throws IOException {
     BM25Stats bm25stats = (BM25Stats) stats;
-    return new BM25DocScorer(bm25stats, context.reader().getMetaData().getCreatedVersionMajor(), context.reader().getNormValues(bm25stats.field));
+    return new BM25DocScorer(bm25stats, context.reader().getNormValues(bm25stats.field));
   }
   
   private class BM25DocScorer extends SimScorer {
@@ -229,17 +215,12 @@ public class BM25Similarity extends Similarity {
     /** precomputed norm[256] with k1 * ((1 - b) + b * dl / avgdl) */
     private final float[] cache;
     
-    BM25DocScorer(BM25Stats stats, int indexCreatedVersionMajor, NumericDocValues norms) throws IOException {
+    BM25DocScorer(BM25Stats stats, NumericDocValues norms) throws IOException {
       this.stats = stats;
       this.weightValue = stats.weight * (k1 + 1);
       this.norms = norms;
-      if (indexCreatedVersionMajor >= 7) {
-        lengthCache = LENGTH_TABLE;
-        cache = stats.cache;
-      } else {
-        lengthCache = OLD_LENGTH_TABLE;
-        cache = stats.oldCache;
-      }
+      lengthCache = LENGTH_TABLE;
+      cache = stats.cache;
     }
     
     @Override
@@ -287,16 +268,15 @@ public class BM25Similarity extends Similarity {
     /** field name, for pulling norms */
     private final String field;
     /** precomputed norm[256] with k1 * ((1 - b) + b * dl / avgdl)
-     *  for both OLD_LENGTH_TABLE and LENGTH_TABLE */
-    private final float[] oldCache, cache;
+     *  for LENGTH_TABLE */
+    private final float[] cache;
 
-    BM25Stats(String field, float boost, Explanation idf, float avgdl, float[] oldCache, float[] cache) {
+    BM25Stats(String field, float boost, Explanation idf, float avgdl, float[] cache) {
       this.field = field;
       this.boost = boost;
       this.idf = idf;
       this.avgdl = avgdl;
       this.weight = idf.getValue() * boost;
-      this.oldCache = oldCache;
       this.cache = cache;
     }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bc9c4144/lucene/core/src/java/org/apache/lucene/search/similarities/SimilarityBase.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/similarities/SimilarityBase.java b/lucene/core/src/java/org/apache/lucene/search/similarities/SimilarityBase.java
index fb34f07..46899a3 100644
--- a/lucene/core/src/java/org/apache/lucene/search/similarities/SimilarityBase.java
+++ b/lucene/core/src/java/org/apache/lucene/search/similarities/SimilarityBase.java
@@ -191,7 +191,6 @@ public abstract class SimilarityBase extends Similarity {
   
   @Override
   public final SimScorer simScorer(SimWeight stats, LeafReaderContext context) throws IOException {
-    int indexCreatedVersionMajor = context.reader().getMetaData().getCreatedVersionMajor();
     if (stats instanceof MultiSimilarity.MultiStats) {
       // a multi term query (e.g. phrase). return the summation, 
       // scoring almost as if it were boolean query
@@ -199,12 +198,12 @@ public abstract class SimilarityBase extends Similarity {
       SimScorer subScorers[] = new SimScorer[subStats.length];
       for (int i = 0; i < subScorers.length; i++) {
         BasicStats basicstats = (BasicStats) subStats[i];
-        subScorers[i] = new BasicSimScorer(basicstats, indexCreatedVersionMajor, context.reader().getNormValues(basicstats.field));
+        subScorers[i] = new BasicSimScorer(basicstats, context.reader().getNormValues(basicstats.field));
       }
       return new MultiSimilarity.MultiSimScorer(subScorers);
     } else {
       BasicStats basicstats = (BasicStats) stats;
-      return new BasicSimScorer(basicstats, indexCreatedVersionMajor, context.reader().getNormValues(basicstats.field));
+      return new BasicSimScorer(basicstats, context.reader().getNormValues(basicstats.field));
     }
   }
   
@@ -218,16 +217,9 @@ public abstract class SimilarityBase extends Similarity {
   // ------------------------------ Norm handling ------------------------------
   
   /** Cache of decoded bytes. */
-  private static final float[] OLD_LENGTH_TABLE = new float[256];
   private static final float[] LENGTH_TABLE = new float[256];
 
   static {
-    for (int i = 1; i < 256; i++) {
-      float f = SmallFloat.byte315ToFloat((byte)i);
-      OLD_LENGTH_TABLE[i] = 1.0f / (f*f);
-    }
-    OLD_LENGTH_TABLE[0] = 1.0f / OLD_LENGTH_TABLE[255]; // otherwise inf
-
     for (int i = 0; i < 256; i++) {
       LENGTH_TABLE[i] = SmallFloat.byte4ToInt((byte) i);
     }
@@ -241,12 +233,7 @@ public abstract class SimilarityBase extends Similarity {
       numTerms = state.getLength() - state.getNumOverlap();
     else
       numTerms = state.getLength();
-    int indexCreatedVersionMajor = state.getIndexCreatedVersionMajor();
-    if (indexCreatedVersionMajor >= 7) {
-      return SmallFloat.intToByte4(numTerms);
-    } else {
-      return SmallFloat.floatToByte315((float) (1 / Math.sqrt(numTerms)));
-    }
+    return SmallFloat.intToByte4(numTerms);
   }
 
   // ----------------------------- Static methods ------------------------------
@@ -268,12 +255,10 @@ public abstract class SimilarityBase extends Similarity {
   final class BasicSimScorer extends SimScorer {
     private final BasicStats stats;
     private final NumericDocValues norms;
-    private final float[] normCache;
     
-    BasicSimScorer(BasicStats stats, int indexCreatedVersionMajor, NumericDocValues norms) throws IOException {
+    BasicSimScorer(BasicStats stats, NumericDocValues norms) throws IOException {
       this.stats = stats;
       this.norms = norms;
-      this.normCache = indexCreatedVersionMajor >= 7 ? LENGTH_TABLE : OLD_LENGTH_TABLE;
     }
 
     float getLengthValue(int doc) throws IOException {
@@ -281,7 +266,7 @@ public abstract class SimilarityBase extends Similarity {
         return 1F;
       }
       if (norms.advanceExact(doc)) {
-        return normCache[Byte.toUnsignedInt((byte) norms.longValue())];
+        return LENGTH_TABLE[Byte.toUnsignedInt((byte) norms.longValue())];
       } else {
         return 0;
       }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bc9c4144/lucene/core/src/java/org/apache/lucene/search/similarities/TFIDFSimilarity.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/similarities/TFIDFSimilarity.java b/lucene/core/src/java/org/apache/lucene/search/similarities/TFIDFSimilarity.java
index 14b3c3f..dba1c61 100644
--- a/lucene/core/src/java/org/apache/lucene/search/similarities/TFIDFSimilarity.java
+++ b/lucene/core/src/java/org/apache/lucene/search/similarities/TFIDFSimilarity.java
@@ -376,15 +376,6 @@ import org.apache.lucene.util.SmallFloat;
  */
 public abstract class TFIDFSimilarity extends Similarity {
 
-  /** Cache of decoded bytes. */
-  static final float[] OLD_NORM_TABLE = new float[256];
-
-  static {
-    for (int i = 0; i < 256; i++) {
-      OLD_NORM_TABLE[i] = SmallFloat.byte315ToFloat((byte)i);
-    }
-  }
-
   /**
    * Sole constructor. (For invocation by subclass 
    * constructors, typically implicit.)
@@ -516,11 +507,7 @@ public abstract class TFIDFSimilarity extends Similarity {
       numTerms = state.getLength() - state.getNumOverlap();
     else
       numTerms = state.getLength();
-    if (state.getIndexCreatedVersionMajor() >= 7) {
-      return SmallFloat.intToByte4(numTerms);
-    } else {
-      return SmallFloat.floatToByte315(lengthNorm(numTerms));
-    }
+    return SmallFloat.intToByte4(numTerms);
   }
  
   /** Computes the amount of a sloppy phrase match, based on an edit distance.
@@ -569,14 +556,8 @@ public abstract class TFIDFSimilarity extends Similarity {
   @Override
   public final SimScorer simScorer(SimWeight stats, LeafReaderContext context) throws IOException {
     IDFStats idfstats = (IDFStats) stats;
-    final float[] normTable;
-    if (context.reader().getMetaData().getCreatedVersionMajor() >= 7) {
-      // the norms only encode the length, we need a translation table that depends on how lengthNorm is implemented
-      normTable = idfstats.normTable;
-    } else {
-      // the norm is directly encoded in the index
-      normTable = OLD_NORM_TABLE;
-    }
+    // the norms only encode the length, we need a translation table that depends on how lengthNorm is implemented
+    final float[] normTable = idfstats.normTable;
     return new TFIDFSimScorer(idfstats, context.reader().getNormValues(idfstats.field), normTable);
   }
   

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bc9c4144/lucene/core/src/java/org/apache/lucene/util/Version.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/util/Version.java b/lucene/core/src/java/org/apache/lucene/util/Version.java
index 634caee..81d02f2 100644
--- a/lucene/core/src/java/org/apache/lucene/util/Version.java
+++ b/lucene/core/src/java/org/apache/lucene/util/Version.java
@@ -32,19 +32,6 @@ import java.util.Locale;
  */
 public final class Version {
 
-  /** Match settings and bugs in Lucene's 6.0 release.
-   * @deprecated (7.0.0) Use latest
-   */
-  @Deprecated
-  public static final Version LUCENE_6_0_0 = new Version(6, 0, 0);
-
-  /**
-   * Match settings and bugs in Lucene's 6.5.0 release.
-   * @deprecated Use latest
-   */
-  @Deprecated
-  public static final Version LUCENE_6_5_0 = new Version(6, 5, 0);
-
   /**
    * Match settings and bugs in Lucene's 7.0.0 release.
    * @deprecated (8.0.0) Use latest

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bc9c4144/lucene/core/src/test/org/apache/lucene/search/similarities/TestBM25Similarity.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/search/similarities/TestBM25Similarity.java b/lucene/core/src/test/org/apache/lucene/search/similarities/TestBM25Similarity.java
index 7c11d6b..4c6382b 100644
--- a/lucene/core/src/test/org/apache/lucene/search/similarities/TestBM25Similarity.java
+++ b/lucene/core/src/test/org/apache/lucene/search/similarities/TestBM25Similarity.java
@@ -17,24 +17,8 @@
 package org.apache.lucene.search.similarities;
 
 
-import java.io.IOException;
-import java.util.stream.Collectors;
-import java.util.stream.IntStream;
-
-import org.apache.lucene.document.Document;
-import org.apache.lucene.document.Field.Store;
-import org.apache.lucene.document.TextField;
-import org.apache.lucene.index.DirectoryReader;
-import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.index.IndexWriter;
-import org.apache.lucene.index.SegmentInfos;
-import org.apache.lucene.index.Term;
 import org.apache.lucene.search.Explanation;
-import org.apache.lucene.search.IndexSearcher;
-import org.apache.lucene.search.TermQuery;
-import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.LuceneTestCase;
-import org.apache.lucene.util.Version;
 
 public class TestBM25Similarity extends LuceneTestCase {
   

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bc9c4144/lucene/core/src/test/org/apache/lucene/search/similarities/TestClassicSimilarity.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/search/similarities/TestClassicSimilarity.java b/lucene/core/src/test/org/apache/lucene/search/similarities/TestClassicSimilarity.java
index 2b36ed2..e7a5606 100644
--- a/lucene/core/src/test/org/apache/lucene/search/similarities/TestClassicSimilarity.java
+++ b/lucene/core/src/test/org/apache/lucene/search/similarities/TestClassicSimilarity.java
@@ -19,29 +19,24 @@ package org.apache.lucene.search.similarities;
 
 import java.io.IOException;
 import java.util.Arrays;
-import java.util.stream.Collectors;
-import java.util.stream.IntStream;
 
 import org.apache.lucene.document.Document;
-import org.apache.lucene.document.StringField;
-import org.apache.lucene.document.TextField;
 import org.apache.lucene.document.Field.Store;
+import org.apache.lucene.document.StringField;
 import org.apache.lucene.index.DirectoryReader;
 import org.apache.lucene.index.FieldInvertState;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.IndexWriter;
 import org.apache.lucene.index.MultiReader;
-import org.apache.lucene.index.SegmentInfos;
 import org.apache.lucene.index.Term;
+import org.apache.lucene.search.BooleanClause.Occur;
 import org.apache.lucene.search.BooleanQuery;
 import org.apache.lucene.search.DisjunctionMaxQuery;
-import org.apache.lucene.search.Explanation;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.TermQuery;
 import org.apache.lucene.search.TopDocs;
 import org.apache.lucene.search.similarities.TFIDFSimilarity.IDFStats;
-import org.apache.lucene.search.BooleanClause.Occur;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.LuceneTestCase;
@@ -163,16 +158,6 @@ public class TestClassicSimilarity extends LuceneTestCase {
   
   public void testSaneNormValues() throws IOException {
     ClassicSimilarity sim = new ClassicSimilarity();
-    for (int i = 0; i < 256; i++) {
-      float boost = TFIDFSimilarity.OLD_NORM_TABLE[i];
-      assertFalse("negative boost: " + boost + ", byte=" + i, boost < 0.0f);
-      assertFalse("inf bost: " + boost + ", byte=" + i, Float.isInfinite(boost));
-      assertFalse("nan boost for byte=" + i, Float.isNaN(boost));
-      if (i > 0) {
-        assertTrue("boost is not increasing: " + boost + ",byte=" + i, boost > TFIDFSimilarity.OLD_NORM_TABLE[i-1]);
-      }
-    }
-
     TFIDFSimilarity.IDFStats stats = (IDFStats) sim.computeWeight(1f, new IndexSearcher(new MultiReader()).collectionStatistics("foo"));
     for (int i = 0; i < 256; i++) {
       float boost = stats.normTable[i];
@@ -185,20 +170,6 @@ public class TestClassicSimilarity extends LuceneTestCase {
     }
   }
 
-  private static Explanation findExplanation(Explanation expl, String text) {
-    if (expl.getDescription().startsWith(text)) {
-      return expl;
-    } else {
-      for (Explanation sub : expl.getDetails()) {
-        Explanation match = findExplanation(sub, text);
-        if (match != null) {
-          return match;
-        }
-      }
-    }
-    return null;
-  }
-
   public void testSameNormsAsBM25() {
     ClassicSimilarity sim1 = new ClassicSimilarity();
     BM25Similarity sim2 = new BM25Similarity();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bc9c4144/lucene/core/src/test/org/apache/lucene/search/similarities/TestSimilarityBase.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/search/similarities/TestSimilarityBase.java b/lucene/core/src/test/org/apache/lucene/search/similarities/TestSimilarityBase.java
index c5472fa..8fbd69d 100644
--- a/lucene/core/src/test/org/apache/lucene/search/similarities/TestSimilarityBase.java
+++ b/lucene/core/src/test/org/apache/lucene/search/similarities/TestSimilarityBase.java
@@ -20,23 +20,16 @@ package org.apache.lucene.search.similarities;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
-import java.util.stream.Collectors;
-import java.util.stream.IntStream;
 
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
 import org.apache.lucene.document.FieldType;
 import org.apache.lucene.document.TextField;
-import org.apache.lucene.document.Field.Store;
-import org.apache.lucene.index.DirectoryReader;
 import org.apache.lucene.index.FieldInvertState;
 import org.apache.lucene.index.IndexOptions;
 import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.index.IndexWriter;
 import org.apache.lucene.index.RandomIndexWriter;
-import org.apache.lucene.index.SegmentInfos;
 import org.apache.lucene.index.Term;
-import org.apache.lucene.index.TermContext;
 import org.apache.lucene.search.CollectionStatistics;
 import org.apache.lucene.search.Explanation;
 import org.apache.lucene.search.IndexSearcher;
@@ -44,14 +37,11 @@ import org.apache.lucene.search.Query;
 import org.apache.lucene.search.TermQuery;
 import org.apache.lucene.search.TermStatistics;
 import org.apache.lucene.search.TopDocs;
-import org.apache.lucene.search.similarities.Similarity.SimWeight;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.Version;
 
-import com.carrotsearch.randomizedtesting.generators.RandomPicks;
-
 /**
  * Tests the {@link SimilarityBase}-based Similarities. Contains unit tests and 
  * integration tests for all Similarities and correctness tests for a select

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bc9c4144/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java b/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java
index 0ea90fc..d3351ab 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java
@@ -334,9 +334,9 @@ public final class TestUtil {
     CheckIndex.testLiveDocs(codecReader, infoStream, true);
     CheckIndex.testFieldInfos(codecReader, infoStream, true);
     CheckIndex.testFieldNorms(codecReader, infoStream, true);
-    CheckIndex.testPostings(codecReader, infoStream, false, true, Version.LUCENE_7_0_0);
+    CheckIndex.testPostings(codecReader, infoStream, false, true);
     CheckIndex.testStoredFields(codecReader, infoStream, true);
-    CheckIndex.testTermVectors(codecReader, infoStream, false, crossCheckTermVectors, true, Version.LUCENE_7_0_0);
+    CheckIndex.testTermVectors(codecReader, infoStream, false, crossCheckTermVectors, true);
     CheckIndex.testDocValues(codecReader, infoStream, true);
     CheckIndex.testPoints(codecReader, infoStream, true);
     

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bc9c4144/solr/core/src/java/org/apache/solr/schema/FieldTypePluginLoader.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/schema/FieldTypePluginLoader.java b/solr/core/src/java/org/apache/solr/schema/FieldTypePluginLoader.java
index f332934..2cddb8f 100644
--- a/solr/core/src/java/org/apache/solr/schema/FieldTypePluginLoader.java
+++ b/solr/core/src/java/org/apache/solr/schema/FieldTypePluginLoader.java
@@ -421,10 +421,10 @@ public final class FieldTypePluginLoader
     Version version = (configuredVersion != null) ?
             Config.parseLuceneVersionString(configuredVersion) : schema.getDefaultLuceneMatchVersion();
 
-    if (!version.onOrAfter(Version.LUCENE_6_0_0)) {
+    if (!version.onOrAfter(Version.LUCENE_7_0_0)) {
       log.warn(pluginClassName + " is using deprecated " + version +
-        " emulation. You should at some point declare and reindex to at least 6.0, because " +
-        "5.x emulation is deprecated and will be removed in 7.0");
+        " emulation. You should at some point declare and reindex to at least 7.0, because " +
+        "6.x emulation is deprecated and will be removed in 8.0");
     }
     return version;
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bc9c4144/solr/core/src/java/org/apache/solr/schema/IndexSchema.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/schema/IndexSchema.java b/solr/core/src/java/org/apache/solr/schema/IndexSchema.java
index 36efbcf..94884a2 100644
--- a/solr/core/src/java/org/apache/solr/schema/IndexSchema.java
+++ b/solr/core/src/java/org/apache/solr/schema/IndexSchema.java
@@ -68,7 +68,6 @@ import org.apache.solr.core.SolrResourceLoader;
 import org.apache.solr.request.LocalSolrQueryRequest;
 import org.apache.solr.response.SchemaXmlWriter;
 import org.apache.solr.response.SolrQueryResponse;
-import org.apache.solr.search.similarities.ClassicSimilarityFactory;
 import org.apache.solr.search.similarities.SchemaSimilarityFactory;
 import org.apache.solr.util.DOMUtil;
 import org.apache.solr.util.plugin.SolrCoreAware;
@@ -475,8 +474,7 @@ public class IndexSchema {
       Node node = (Node) xpath.evaluate(expression, document, XPathConstants.NODE);
       similarityFactory = readSimilarity(loader, node);
       if (similarityFactory == null) {
-        final boolean modernSim = getDefaultLuceneMatchVersion().onOrAfter(Version.LUCENE_6_0_0);
-        final Class simClass = modernSim ? SchemaSimilarityFactory.class : ClassicSimilarityFactory.class;
+        final Class<?> simClass = SchemaSimilarityFactory.class;
         // use the loader to ensure proper SolrCoreAware handling
         similarityFactory = loader.newInstance(simClass.getName(), SimilarityFactory.class);
         similarityFactory.init(new ModifiableSolrParams());

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bc9c4144/solr/core/src/java/org/apache/solr/schema/IndexSchemaFactory.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/schema/IndexSchemaFactory.java b/solr/core/src/java/org/apache/solr/schema/IndexSchemaFactory.java
index 83750c3..fda4868 100644
--- a/solr/core/src/java/org/apache/solr/schema/IndexSchemaFactory.java
+++ b/solr/core/src/java/org/apache/solr/schema/IndexSchemaFactory.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 package org.apache.solr.schema;
-import org.apache.lucene.util.Version;
+
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrException.ErrorCode;
 import org.apache.solr.core.PluginInfo;
@@ -65,12 +65,7 @@ public abstract class IndexSchemaFactory implements NamedListInitializedPlugin {
       factory = config.getResourceLoader().newInstance(info.className, IndexSchemaFactory.class);
       factory.init(info.initArgs);
     } else {
-      if (config.luceneMatchVersion.onOrAfter(Version.LUCENE_6_0_0)) {
-        // ManagedIndexSchemaFactory is SolrCoreAware so we must create using the resource loader
-        factory = config.getResourceLoader().newInstance(ManagedIndexSchemaFactory.class.getName(), IndexSchemaFactory.class);
-      } else {
-        factory = new ClassicIndexSchemaFactory();
-      }
+      factory = config.getResourceLoader().newInstance(ManagedIndexSchemaFactory.class.getName(), IndexSchemaFactory.class);
     }
     IndexSchema schema = factory.create(resourceName, config);
     return schema;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bc9c4144/solr/core/src/java/org/apache/solr/search/similarities/SchemaSimilarityFactory.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/similarities/SchemaSimilarityFactory.java b/solr/core/src/java/org/apache/solr/search/similarities/SchemaSimilarityFactory.java
index 3c942d2..0aea8ed 100644
--- a/solr/core/src/java/org/apache/solr/search/similarities/SchemaSimilarityFactory.java
+++ b/solr/core/src/java/org/apache/solr/search/similarities/SchemaSimilarityFactory.java
@@ -114,9 +114,7 @@ public class SchemaSimilarityFactory extends SimilarityFactory implements SolrCo
       Similarity defaultSim = null;
       if (null == defaultSimFromFieldType) {
         // nothing configured, choose a sensible implicit default...
-        defaultSim = this.core.getSolrConfig().luceneMatchVersion.onOrAfter(Version.LUCENE_6_0_0)
-          ? new BM25Similarity()
-          : new ClassicSimilarity();
+        defaultSim = new BM25Similarity();
       } else {
         FieldType defSimFT = core.getLatestSchema().getFieldTypeByName(defaultSimFromFieldType);
         if (null == defSimFT) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bc9c4144/solr/core/src/test-files/solr/configsets/_default/conf/solrconfig.xml
----------------------------------------------------------------------
diff --git a/solr/core/src/test-files/solr/configsets/_default/conf/solrconfig.xml b/solr/core/src/test-files/solr/configsets/_default/conf/solrconfig.xml
index f75303f..c885f06 100644
--- a/solr/core/src/test-files/solr/configsets/_default/conf/solrconfig.xml
+++ b/solr/core/src/test-files/solr/configsets/_default/conf/solrconfig.xml
@@ -35,7 +35,7 @@
        that you fully re-index after changing this setting as it can
        affect both how text is indexed and queried.
   -->
-  <luceneMatchVersion>7.0.0</luceneMatchVersion>
+  <luceneMatchVersion>8.0.0</luceneMatchVersion>
 
   <!-- <lib/> directives can be used to instruct Solr to load any Jars
        identified and use them to resolve any "plugins" specified in

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bc9c4144/solr/core/src/test/org/apache/solr/search/similarities/TestNonDefinedSimilarityFactory.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/search/similarities/TestNonDefinedSimilarityFactory.java b/solr/core/src/test/org/apache/solr/search/similarities/TestNonDefinedSimilarityFactory.java
index 1560c18..7460652 100644
--- a/solr/core/src/test/org/apache/solr/search/similarities/TestNonDefinedSimilarityFactory.java
+++ b/solr/core/src/test/org/apache/solr/search/similarities/TestNonDefinedSimilarityFactory.java
@@ -16,7 +16,6 @@
  */
 package org.apache.solr.search.similarities;
 
-import org.apache.lucene.search.similarities.ClassicSimilarity;
 import org.apache.lucene.search.similarities.BM25Similarity;
 import org.junit.After;
 
@@ -40,13 +39,4 @@ public class TestNonDefinedSimilarityFactory extends BaseSimilarityTestCase {
     BM25Similarity sim = getSimilarity("text", BM25Similarity.class);
     assertEquals(0.75F, sim.getB(), 0.0F);
   }
-  
-  public void testClassic() throws Exception {
-    // any value below 6.0 should have this behavior
-    System.setProperty("tests.luceneMatchVersion", "5.3");
-    initCore("solrconfig-basic.xml","schema-tiny.xml");
-    ClassicSimilarity sim = getSimilarity("text", ClassicSimilarity.class);
-    assertEquals(true, sim.getDiscountOverlaps());
-    System.clearProperty("tests.luceneMatchVersion");
-  }
 }


[04/23] lucene-solr:jira/solr-10879: Trying to fix failing tests after bumping up version to 8.0

Posted by ab...@apache.org.
Trying to fix failing tests after bumping up version to 8.0


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

Branch: refs/heads/jira/solr-10879
Commit: 93fbfc2e56e88a68715dd5b57b39ee9b801be84f
Parents: 774e3d8
Author: Anshum Gupta <an...@apple.com>
Authored: Fri Jun 30 10:59:05 2017 -0700
Committer: Anshum Gupta <an...@apple.com>
Committed: Fri Jun 30 10:59:05 2017 -0700

----------------------------------------------------------------------
 .../apache/lucene/index/TestSegmentInfos.java   | 14 +++++-----
 .../search/similarities/TestBM25Similarity.java | 26 ------------------
 .../similarities/TestClassicSimilarity.java     | 26 ------------------
 .../search/similarities/TestSimilarityBase.java | 29 --------------------
 4 files changed, 7 insertions(+), 88 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/93fbfc2e/lucene/core/src/test/org/apache/lucene/index/TestSegmentInfos.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestSegmentInfos.java b/lucene/core/src/test/org/apache/lucene/index/TestSegmentInfos.java
index 128601e..e9edf4e 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestSegmentInfos.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestSegmentInfos.java
@@ -32,8 +32,8 @@ public class TestSegmentInfos extends LuceneTestCase {
   public void testIllegalCreatedVersion() {
     IllegalArgumentException e = expectThrows(IllegalArgumentException.class, () -> new SegmentInfos(5));
     assertEquals("indexCreatedVersionMajor must be >= 6, got: 5", e.getMessage());
-    e = expectThrows(IllegalArgumentException.class, () -> new SegmentInfos(8));
-    assertEquals("indexCreatedVersionMajor is in the future: 8", e.getMessage());
+    e = expectThrows(IllegalArgumentException.class, () -> new SegmentInfos(Version.LATEST.major + 1));
+    assertEquals("indexCreatedVersionMajor is in the future: " + (Version.LATEST.major + 1), e.getMessage());
   }
 
   // LUCENE-5954
@@ -56,7 +56,7 @@ public class TestSegmentInfos extends LuceneTestCase {
     Codec codec = Codec.getDefault();
 
     SegmentInfos sis = new SegmentInfos(Version.LATEST.major);
-    SegmentInfo info = new SegmentInfo(dir, Version.LUCENE_7_0_0, Version.LUCENE_7_0_0, "_0", 1, false, Codec.getDefault(), 
+    SegmentInfo info = new SegmentInfo(dir, Version.LUCENE_8_0_0, Version.LUCENE_8_0_0, "_0", 1, false, Codec.getDefault(),
                                        Collections.<String,String>emptyMap(), id, Collections.<String,String>emptyMap(), null);
     info.setFiles(Collections.<String>emptySet());
     codec.segmentInfoFormat().write(dir, info, IOContext.DEFAULT);
@@ -65,7 +65,7 @@ public class TestSegmentInfos extends LuceneTestCase {
     sis.add(commitInfo);
     sis.commit(dir);
     sis = SegmentInfos.readLatestCommit(dir);
-    assertEquals(Version.LUCENE_7_0_0, sis.getMinSegmentLuceneVersion());
+    assertEquals(Version.LUCENE_8_0_0, sis.getMinSegmentLuceneVersion());
     assertEquals(Version.LATEST, sis.getCommitLuceneVersion());
     dir.close();
   }
@@ -78,14 +78,14 @@ public class TestSegmentInfos extends LuceneTestCase {
     Codec codec = Codec.getDefault();
 
     SegmentInfos sis = new SegmentInfos(Version.LATEST.major);
-    SegmentInfo info = new SegmentInfo(dir, Version.LUCENE_7_0_0, Version.LUCENE_7_0_0, "_0", 1, false, Codec.getDefault(), 
+    SegmentInfo info = new SegmentInfo(dir, Version.LUCENE_8_0_0, Version.LUCENE_8_0_0, "_0", 1, false, Codec.getDefault(),
                                        Collections.<String,String>emptyMap(), id, Collections.<String,String>emptyMap(), null);
     info.setFiles(Collections.<String>emptySet());
     codec.segmentInfoFormat().write(dir, info, IOContext.DEFAULT);
     SegmentCommitInfo commitInfo = new SegmentCommitInfo(info, 0, -1, -1, -1);
     sis.add(commitInfo);
 
-    info = new SegmentInfo(dir, Version.LUCENE_7_0_0, Version.LUCENE_7_0_0, "_1", 1, false, Codec.getDefault(), 
+    info = new SegmentInfo(dir, Version.LUCENE_8_0_0, Version.LUCENE_8_0_0, "_1", 1, false, Codec.getDefault(),
                            Collections.<String,String>emptyMap(), id, Collections.<String,String>emptyMap(), null);
     info.setFiles(Collections.<String>emptySet());
     codec.segmentInfoFormat().write(dir, info, IOContext.DEFAULT);
@@ -94,7 +94,7 @@ public class TestSegmentInfos extends LuceneTestCase {
 
     sis.commit(dir);
     sis = SegmentInfos.readLatestCommit(dir);
-    assertEquals(Version.LUCENE_7_0_0, sis.getMinSegmentLuceneVersion());
+    assertEquals(Version.LUCENE_8_0_0, sis.getMinSegmentLuceneVersion());
     assertEquals(Version.LATEST, sis.getCommitLuceneVersion());
     dir.close();
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/93fbfc2e/lucene/core/src/test/org/apache/lucene/search/similarities/TestBM25Similarity.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/search/similarities/TestBM25Similarity.java b/lucene/core/src/test/org/apache/lucene/search/similarities/TestBM25Similarity.java
index e54ce25..7c11d6b 100644
--- a/lucene/core/src/test/org/apache/lucene/search/similarities/TestBM25Similarity.java
+++ b/lucene/core/src/test/org/apache/lucene/search/similarities/TestBM25Similarity.java
@@ -77,32 +77,6 @@ public class TestBM25Similarity extends LuceneTestCase {
     assertTrue(expected.getMessage().contains("illegal b value"));
   }
 
-  public void testLengthEncodingBackwardCompatibility() throws IOException {
-    Similarity similarity = new BM25Similarity();
-    for (int indexCreatedVersionMajor : new int[] { Version.LUCENE_6_0_0.major, Version.LATEST.major}) {
-      for (int length : new int[] {1, 2, 4}) { // these length values are encoded accurately on both cases
-        Directory dir = newDirectory();
-        // set the version on the directory
-        new SegmentInfos(indexCreatedVersionMajor).commit(dir);
-        IndexWriter w = new IndexWriter(dir, newIndexWriterConfig().setSimilarity(similarity));
-        Document doc = new Document();
-        String value = IntStream.range(0, length).mapToObj(i -> "b").collect(Collectors.joining(" "));
-        doc.add(new TextField("foo", value, Store.NO));
-        w.addDocument(doc);
-        IndexReader reader = DirectoryReader.open(w);
-        IndexSearcher searcher = newSearcher(reader);
-        searcher.setSimilarity(similarity);
-        Explanation expl = searcher.explain(new TermQuery(new Term("foo", "b")), 0);
-        Explanation docLen = findExplanation(expl, "fieldLength");
-        assertNotNull(docLen);
-        assertEquals(docLen.toString(), length, (int) docLen.getValue());
-        w.close();
-        reader.close();
-        dir.close();
-      }
-    }
-  }
-
   private static Explanation findExplanation(Explanation expl, String text) {
     if (expl.getDescription().equals(text)) {
       return expl;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/93fbfc2e/lucene/core/src/test/org/apache/lucene/search/similarities/TestClassicSimilarity.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/search/similarities/TestClassicSimilarity.java b/lucene/core/src/test/org/apache/lucene/search/similarities/TestClassicSimilarity.java
index c7f0453..2b36ed2 100644
--- a/lucene/core/src/test/org/apache/lucene/search/similarities/TestClassicSimilarity.java
+++ b/lucene/core/src/test/org/apache/lucene/search/similarities/TestClassicSimilarity.java
@@ -185,32 +185,6 @@ public class TestClassicSimilarity extends LuceneTestCase {
     }
   }
 
-  public void testNormEncodingBackwardCompatibility() throws IOException {
-    Similarity similarity = new ClassicSimilarity();
-    for (int indexCreatedVersionMajor : new int[] { Version.LUCENE_6_0_0.major, Version.LATEST.major}) {
-      for (int length : new int[] {1, 4, 16 }) { // these length values are encoded accurately on both cases
-        Directory dir = newDirectory();
-        // set the version on the directory
-        new SegmentInfos(indexCreatedVersionMajor).commit(dir);
-        IndexWriter w = new IndexWriter(dir, newIndexWriterConfig().setSimilarity(similarity));
-        Document doc = new Document();
-        String value = IntStream.range(0, length).mapToObj(i -> "b").collect(Collectors.joining(" "));
-        doc.add(new TextField("foo", value, Store.NO));
-        w.addDocument(doc);
-        IndexReader reader = DirectoryReader.open(w);
-        IndexSearcher searcher = newSearcher(reader);
-        searcher.setSimilarity(similarity);
-        Explanation expl = searcher.explain(new TermQuery(new Term("foo", "b")), 0);
-        Explanation fieldNorm = findExplanation(expl, "fieldNorm");
-        assertNotNull(fieldNorm);
-        assertEquals(fieldNorm.toString(), 1/Math.sqrt(length), fieldNorm.getValue(), 0f);
-        w.close();
-        reader.close();
-        dir.close();
-      }
-    }
-  }
-
   private static Explanation findExplanation(Explanation expl, String text) {
     if (expl.getDescription().startsWith(text)) {
       return expl;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/93fbfc2e/lucene/core/src/test/org/apache/lucene/search/similarities/TestSimilarityBase.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/search/similarities/TestSimilarityBase.java b/lucene/core/src/test/org/apache/lucene/search/similarities/TestSimilarityBase.java
index 59be73a..c5472fa 100644
--- a/lucene/core/src/test/org/apache/lucene/search/similarities/TestSimilarityBase.java
+++ b/lucene/core/src/test/org/apache/lucene/search/similarities/TestSimilarityBase.java
@@ -609,33 +609,4 @@ public class TestSimilarityBase extends LuceneTestCase {
     actual.setDiscountOverlaps(true);
     assertEquals(expected.computeNorm(state), actual.computeNorm(state));
   }
-
-  public void testLengthEncodingBackwardCompatibility() throws IOException {
-    Similarity similarity = RandomPicks.randomFrom(random(), sims);
-    for (int indexCreatedVersionMajor : new int[] { Version.LUCENE_6_0_0.major, Version.LATEST.major}) {
-      for (int length : new int[] {1, 2, 4}) { // these length values are encoded accurately on both cases
-        Directory dir = newDirectory();
-        // set the version on the directory
-        new SegmentInfos(indexCreatedVersionMajor).commit(dir);
-        IndexWriter w = new IndexWriter(dir, newIndexWriterConfig().setSimilarity(similarity));
-        Document doc = new Document();
-        String value = IntStream.range(0, length).mapToObj(i -> "b").collect(Collectors.joining(" "));
-        doc.add(new TextField("foo", value, Store.NO));
-        w.addDocument(doc);
-        IndexReader reader = DirectoryReader.open(w);
-        IndexSearcher searcher = newSearcher(reader);
-        searcher.setSimilarity(similarity);
-        Term term = new Term("foo", "b");
-        TermContext context = TermContext.build(reader.getContext(), term);
-        SimWeight simWeight = similarity.computeWeight(1f, searcher.collectionStatistics("foo"), searcher.termStatistics(term, context));
-        SimilarityBase.BasicSimScorer simScorer = (SimilarityBase.BasicSimScorer) similarity.simScorer(simWeight, reader.leaves().get(0));
-        float docLength = simScorer.getLengthValue(0);
-        assertEquals(length, (int) docLength);
-        
-        w.close();
-        reader.close();
-        dir.close();
-      }
-    }
-  }
 }


[15/23] lucene-solr:jira/solr-10879: SOLR-10914: RecoveryStrategy's sendPrepRecoveryCmd can get stuck for 5 minutes if leader is unloaded

Posted by ab...@apache.org.
SOLR-10914: RecoveryStrategy's sendPrepRecoveryCmd can get stuck for 5 minutes if leader is unloaded


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

Branch: refs/heads/jira/solr-10879
Commit: 157ff9a4e159158f4ecc474d1874da97577e6190
Parents: b978f37
Author: Shalin Shekhar Mangar <sh...@apache.org>
Authored: Mon Jul 3 19:50:33 2017 +0530
Committer: Shalin Shekhar Mangar <sh...@apache.org>
Committed: Mon Jul 3 19:50:33 2017 +0530

----------------------------------------------------------------------
 solr/CHANGES.txt                                |   2 +
 .../org/apache/solr/cloud/RecoveryStrategy.java |  27 +----
 .../org/apache/solr/util/TestInjection.java     |   2 +-
 solr/core/src/test-files/solr/solr.xml          |   1 +
 .../apache/solr/cloud/TestCloudRecovery.java    |   8 --
 .../org/apache/solr/cloud/TestPrepRecovery.java | 109 +++++++++++++++++++
 6 files changed, 117 insertions(+), 32 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/157ff9a4/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index aadc6db..1bc960d 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -501,6 +501,8 @@ when using one of Exact*StatsCache (Mikhail Khludnev)
 * SOLR-10910: Clean up a few details left over from pluggable transient core and untangling
   CoreDescriptor/CoreContainer references (Erick Erickson)
 
+* SOLR-10914: RecoveryStrategy's sendPrepRecoveryCmd can get stuck for 5 minutes if leader is unloaded. (shalin)
+
 Optimizations
 ----------------------
 * SOLR-10634: JSON Facet API: When a field/terms facet will retrieve all buckets (i.e. limit:-1)

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/157ff9a4/solr/core/src/java/org/apache/solr/cloud/RecoveryStrategy.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/RecoveryStrategy.java b/solr/core/src/java/org/apache/solr/cloud/RecoveryStrategy.java
index 9b0805f..063f794 100644
--- a/solr/core/src/java/org/apache/solr/cloud/RecoveryStrategy.java
+++ b/solr/core/src/java/org/apache/solr/cloud/RecoveryStrategy.java
@@ -19,7 +19,6 @@ package org.apache.solr.cloud;
 import java.io.Closeable;
 import java.io.IOException;
 import java.lang.invoke.MethodHandles;
-import java.net.SocketTimeoutException;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
@@ -811,29 +810,12 @@ public class RecoveryStrategy implements Runnable, Closeable {
       prepCmd.setOnlyIfLeaderActive(true);
     }
 
-    final int maxTries = 30;
-    for (int numTries = 0; numTries < maxTries; numTries++) {
-      try {
-        sendPrepRecoveryCmd(leaderBaseUrl, prepCmd);
-        break;
-      } catch (ExecutionException e) {
-        if (e.getCause() instanceof SolrServerException) {
-          SolrServerException solrException = (SolrServerException) e.getCause();
-          if (solrException.getRootCause() instanceof SocketTimeoutException && numTries < maxTries) {
-            LOG.warn("Socket timeout on send prep recovery cmd, retrying.. ");
-            continue;
-          }
-        }
-        throw  e;
-      }
-    }
-  }
-
-  final private void sendPrepRecoveryCmd(String leaderBaseUrl, WaitForState prepCmd)
-      throws SolrServerException, IOException, InterruptedException, ExecutionException {
+    int conflictWaitMs = zkController.getLeaderConflictResolveWait();
+    // timeout after 5 seconds more than the max timeout (conflictWait + 3 seconds) on the server side
+    int readTimeout = conflictWaitMs + 8000;
     try (HttpSolrClient client = new HttpSolrClient.Builder(leaderBaseUrl).build()) {
       client.setConnectionTimeout(10000);
-      client.setSoTimeout(10000);
+      client.setSoTimeout(readTimeout);
       HttpUriRequestResponse mrr = client.httpUriRequest(prepCmd);
       prevSendPreRecoveryHttpUriRequest = mrr.httpUriRequest;
 
@@ -842,5 +824,4 @@ public class RecoveryStrategy implements Runnable, Closeable {
       mrr.future.get();
     }
   }
-
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/157ff9a4/solr/core/src/java/org/apache/solr/util/TestInjection.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/util/TestInjection.java b/solr/core/src/java/org/apache/solr/util/TestInjection.java
index 5b0d047..d7584da 100644
--- a/solr/core/src/java/org/apache/solr/util/TestInjection.java
+++ b/solr/core/src/java/org/apache/solr/util/TestInjection.java
@@ -329,7 +329,7 @@ public class TestInjection {
       boolean enabled = pair.first();
       int chanceIn100 = pair.second();
       // Prevent for continuous pause forever
-      if (enabled && rand.nextInt(100) >= (100 - chanceIn100) && countPrepRecoveryOpPauseForever.get() < 2) {
+      if (enabled && rand.nextInt(100) >= (100 - chanceIn100) && countPrepRecoveryOpPauseForever.get() < 1) {
         countPrepRecoveryOpPauseForever.incrementAndGet();
         log.info("inject pause forever for prep recovery op");
         try {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/157ff9a4/solr/core/src/test-files/solr/solr.xml
----------------------------------------------------------------------
diff --git a/solr/core/src/test-files/solr/solr.xml b/solr/core/src/test-files/solr/solr.xml
index 526dffa..ae27fe7 100644
--- a/solr/core/src/test-files/solr/solr.xml
+++ b/solr/core/src/test-files/solr/solr.xml
@@ -43,6 +43,7 @@
     <int name="zkClientTimeout">${solr.zkclienttimeout:30000}</int>
     <bool name="genericCoreNodeNames">${genericCoreNodeNames:true}</bool>
     <int name="leaderVoteWait">${leaderVoteWait:10000}</int>
+    <int name="leaderConflictResolveWait">${leaderConflictResolveWait:180000}</int>
     <int name="distribUpdateConnTimeout">${distribUpdateConnTimeout:45000}</int>
     <int name="distribUpdateSoTimeout">${distribUpdateSoTimeout:340000}</int>
     <int name="autoReplicaFailoverWaitAfterExpiration">${autoReplicaFailoverWaitAfterExpiration:10000}</int>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/157ff9a4/solr/core/src/test/org/apache/solr/cloud/TestCloudRecovery.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/TestCloudRecovery.java b/solr/core/src/test/org/apache/solr/cloud/TestCloudRecovery.java
index c7fc0e8..2cf8774 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestCloudRecovery.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestCloudRecovery.java
@@ -43,8 +43,6 @@ import org.apache.solr.core.SolrCore;
 import org.apache.solr.metrics.SolrMetricManager;
 import org.apache.solr.update.DirectUpdateHandler2;
 import org.apache.solr.update.UpdateLog;
-import org.apache.solr.util.TestInjection;
-import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -56,7 +54,6 @@ public class TestCloudRecovery extends SolrCloudTestCase {
 
   @BeforeClass
   public static void setupCluster() throws Exception {
-    TestInjection.prepRecoveryOpPauseForever = "true:30";
     System.setProperty("solr.directoryFactory", "solr.StandardDirectoryFactory");
     System.setProperty("solr.ulog.numRecordsToKeep", "1000");
 
@@ -73,11 +70,6 @@ public class TestCloudRecovery extends SolrCloudTestCase {
         false, true, 30);
   }
 
-  @AfterClass
-  public static void afterClass() {
-    TestInjection.reset();
-  }
-
   @Before
   public void resetCollection() throws IOException, SolrServerException {
     cluster.getSolrClient().deleteByQuery(COLLECTION, "*:*");

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/157ff9a4/solr/core/src/test/org/apache/solr/cloud/TestPrepRecovery.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/TestPrepRecovery.java b/solr/core/src/test/org/apache/solr/cloud/TestPrepRecovery.java
new file mode 100644
index 0000000..a80565b
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/cloud/TestPrepRecovery.java
@@ -0,0 +1,109 @@
+/*
+ * 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.solr.cloud;
+
+import org.apache.solr.client.solrj.embedded.JettySolrRunner;
+import org.apache.solr.client.solrj.impl.CloudSolrClient;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.common.cloud.Replica;
+import org.apache.solr.util.TestInjection;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+/**
+ * Tests for PREPRECOVERY CoreAdmin API
+ */
+public class TestPrepRecovery extends SolrCloudTestCase {
+
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    System.setProperty("solr.directoryFactory", "solr.StandardDirectoryFactory");
+    System.setProperty("solr.ulog.numRecordsToKeep", "1000");
+    // the default is 180s and our waitForState times out in 90s
+    // so we lower this to 10s so that we can still test timeouts
+    System.setProperty("leaderConflictResolveWait", "10000");
+
+    configureCluster(2)
+        .addConfig("config", TEST_PATH().resolve("configsets").resolve("cloud-minimal").resolve("conf"))
+        .withSolrXml(TEST_PATH().resolve("solr.xml"))
+        .configure();
+  }
+
+  public static void tearCluster() throws Exception {
+    System.clearProperty("leaderConflictResolveWait");
+  }
+
+  @Test
+  public void testLeaderUnloaded() throws Exception {
+    CloudSolrClient solrClient = cluster.getSolrClient();
+
+    String collectionName = "testLeaderUnloaded";
+    CollectionAdminRequest.createCollection(collectionName, 1, 2)
+        .process(solrClient);
+
+    waitForState("Expected collection: testLeaderUnloaded to be live with 1 shard and 2 replicas",
+        collectionName, clusterShape(1, 2));
+
+    JettySolrRunner newNode = cluster.startJettySolrRunner();
+    String newNodeName = newNode.getNodeName();
+
+    // add a replica to the new node so that it starts watching the collection
+    CollectionAdminRequest.addReplicaToShard(collectionName, "shard1")
+        .setNode(newNodeName)
+        .process(solrClient);
+
+    // now delete the leader
+    Replica leader = solrClient.getZkStateReader().getLeaderRetry(collectionName, "shard1");
+    CollectionAdminRequest.deleteReplica(collectionName, "shard1", leader.getName())
+        .process(solrClient);
+
+    // add another replica to the new node. When it starts recovering, it will likely have stale state
+    // and ask the erstwhile leader to PREPRECOVERY which will hang for about 30 seconds
+    CollectionAdminRequest.addReplicaToShard(collectionName, "shard1")
+        .setNode(newNodeName)
+        .process(solrClient);
+
+    // in the absence of the fixes made in SOLR-10914, this statement will timeout after 90s
+    waitForState("Expected collection: testLeaderUnloaded to be live with 1 shard and 3 replicas",
+        collectionName, clusterShape(1, 3));
+  }
+
+  public void testLeaderNotResponding() throws Exception {
+    CloudSolrClient solrClient = cluster.getSolrClient();
+
+    String collectionName = "testLeaderNotResponding";
+    CollectionAdminRequest.createCollection(collectionName, 1, 1)
+        .process(solrClient);
+
+    waitForState("Expected collection: testLeaderNotResponding to be live with 1 shard and 1 replicas",
+        collectionName, clusterShape(1, 1));
+
+    TestInjection.prepRecoveryOpPauseForever = "true:100";
+    try {
+      CollectionAdminRequest.addReplicaToShard(collectionName, "shard1")
+          .process(solrClient);
+
+      // in the absence of fixes made in SOLR-9716, prep recovery waits forever and the following statement
+      // times out in 90 seconds
+      waitForState("Expected collection: testLeaderNotResponding to be live with 1 shard and 2 replicas",
+          collectionName, clusterShape(1, 2));
+    } finally {
+      TestInjection.reset();
+    }
+  }
+}