You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by jp...@apache.org on 2016/11/10 14:14:28 UTC

[1/4] lucene-solr:master: LUCENE-7545: Dense norms/doc-values should not consume memory for the IW buffer.

Repository: lucene-solr
Updated Branches:
  refs/heads/master 2902727a1 -> c4c5e868d


LUCENE-7545: Dense norms/doc-values should not consume memory for the IW buffer.


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

Branch: refs/heads/master
Commit: c415bc8d1ddbb6477c83d559cb254a384aceb6df
Parents: 2902727
Author: Adrien Grand <jp...@gmail.com>
Authored: Thu Nov 10 14:04:15 2016 +0100
Committer: Adrien Grand <jp...@gmail.com>
Committed: Thu Nov 10 14:04:15 2016 +0100

----------------------------------------------------------------------
 .../lucene/index/BinaryDocValuesWriter.java     | 15 ++--
 .../apache/lucene/index/DocsWithFieldSet.java   | 62 +++++++++++++++
 .../apache/lucene/index/NormValuesWriter.java   | 15 ++--
 .../lucene/index/NumericDocValuesWriter.java    | 15 ++--
 .../lucene/index/SortedDocValuesWriter.java     | 15 ++--
 .../index/SortedNumericDocValuesWriter.java     | 15 ++--
 .../lucene/index/SortedSetDocValuesWriter.java  | 15 ++--
 .../lucene/index/TestDocsWithFieldSet.java      | 80 ++++++++++++++++++++
 8 files changed, 178 insertions(+), 54 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c415bc8d/lucene/core/src/java/org/apache/lucene/index/BinaryDocValuesWriter.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/BinaryDocValuesWriter.java b/lucene/core/src/java/org/apache/lucene/index/BinaryDocValuesWriter.java
index 9611a03..0344c58 100644
--- a/lucene/core/src/java/org/apache/lucene/index/BinaryDocValuesWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/index/BinaryDocValuesWriter.java
@@ -24,11 +24,9 @@ import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.store.DataInput;
 import org.apache.lucene.store.DataOutput;
 import org.apache.lucene.util.ArrayUtil;
-import org.apache.lucene.util.BitSetIterator;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefBuilder;
 import org.apache.lucene.util.Counter;
-import org.apache.lucene.util.FixedBitSet;
 import org.apache.lucene.util.PagedBytes;
 import org.apache.lucene.util.packed.PackedInts;
 import org.apache.lucene.util.packed.PackedLongValues;
@@ -48,7 +46,7 @@ class BinaryDocValuesWriter extends DocValuesWriter {
 
   private final Counter iwBytesUsed;
   private final PackedLongValues.Builder lengths;
-  private FixedBitSet docsWithField;
+  private DocsWithFieldSet docsWithField;
   private final FieldInfo fieldInfo;
   private long bytesUsed;
   private int lastDocID = -1;
@@ -60,7 +58,7 @@ class BinaryDocValuesWriter extends DocValuesWriter {
     this.bytesOut = bytes.getDataOutput();
     this.lengths = PackedLongValues.deltaPackedBuilder(PackedInts.COMPACT);
     this.iwBytesUsed = iwBytesUsed;
-    this.docsWithField = new FixedBitSet(64);
+    this.docsWithField = new DocsWithFieldSet();
     this.bytesUsed = lengths.ramBytesUsed() + docsWithField.ramBytesUsed();
     iwBytesUsed.addAndGet(bytesUsed);
   }
@@ -84,8 +82,7 @@ class BinaryDocValuesWriter extends DocValuesWriter {
       // Should never happen!
       throw new RuntimeException(ioe);
     }
-    docsWithField = FixedBitSet.ensureCapacity(docsWithField, docID);
-    docsWithField.set(docID);
+    docsWithField.add(docID);
     updateBytesUsed();
 
     lastDocID = docID;
@@ -112,7 +109,7 @@ class BinaryDocValuesWriter extends DocValuesWriter {
                                   if (fieldInfoIn != fieldInfo) {
                                     throw new IllegalArgumentException("wrong fieldInfo");
                                   }
-                                  return new BufferedBinaryDocValues(lengths, maxLength, bytes.getDataInput(), docsWithField);
+                                  return new BufferedBinaryDocValues(lengths, maxLength, bytes.getDataInput(), docsWithField.iterator());
                                 }
                               });
   }
@@ -124,12 +121,12 @@ class BinaryDocValuesWriter extends DocValuesWriter {
     final DocIdSetIterator docsWithField;
     final DataInput bytesIterator;
     
-    BufferedBinaryDocValues(PackedLongValues lengths, int maxLength, DataInput bytesIterator, FixedBitSet docsWithFields) {
+    BufferedBinaryDocValues(PackedLongValues lengths, int maxLength, DataInput bytesIterator, DocIdSetIterator docsWithFields) {
       this.value = new BytesRefBuilder();
       this.value.grow(maxLength);
       this.lengthsIterator = lengths.iterator();
       this.bytesIterator = bytesIterator;
-      this.docsWithField = new BitSetIterator(docsWithFields, lengths.size());
+      this.docsWithField = docsWithFields;
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c415bc8d/lucene/core/src/java/org/apache/lucene/index/DocsWithFieldSet.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/DocsWithFieldSet.java b/lucene/core/src/java/org/apache/lucene/index/DocsWithFieldSet.java
new file mode 100644
index 0000000..6c0d6dd
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/index/DocsWithFieldSet.java
@@ -0,0 +1,62 @@
+/*
+ * 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 org.apache.lucene.search.DocIdSet;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.util.BitSetIterator;
+import org.apache.lucene.util.FixedBitSet;
+import org.apache.lucene.util.RamUsageEstimator;
+
+/** Accumulator for documents that have a value for a field. This is optimized
+ *  for the case that all documents have a value. */
+final class DocsWithFieldSet extends DocIdSet {
+
+  private static long BASE_RAM_BYTES_USED = RamUsageEstimator.shallowSizeOfInstance(DocsWithFieldSet.class);
+
+  private FixedBitSet set;
+  private int cost = 0;
+  private int lastDocId = -1;
+
+  void add(int docID) {
+    if (docID <= lastDocId) {
+      throw new IllegalArgumentException("Out of order doc ids: last=" + lastDocId + ", next=" + docID);
+    }
+    if (set != null) {
+      set = FixedBitSet.ensureCapacity(set, docID);
+      set.set(docID);
+    } else if (docID != cost) {
+      // migrate to a sparse encoding using a bit set
+      set = new FixedBitSet(docID + 1);
+      set.set(0, cost);
+      set.set(docID);
+    }
+    lastDocId = docID;
+    cost++;
+  }
+
+  @Override
+  public long ramBytesUsed() {
+    return BASE_RAM_BYTES_USED + (set == null ? 0 : set.ramBytesUsed());
+  }
+
+  @Override
+  public DocIdSetIterator iterator() {
+    return set != null ? new BitSetIterator(set, cost) : DocIdSetIterator.all(cost);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c415bc8d/lucene/core/src/java/org/apache/lucene/index/NormValuesWriter.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/NormValuesWriter.java b/lucene/core/src/java/org/apache/lucene/index/NormValuesWriter.java
index b0d05e4..4923f3b 100644
--- a/lucene/core/src/java/org/apache/lucene/index/NormValuesWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/index/NormValuesWriter.java
@@ -22,9 +22,7 @@ import java.io.IOException;
 import org.apache.lucene.codecs.NormsConsumer;
 import org.apache.lucene.codecs.NormsProducer;
 import org.apache.lucene.search.DocIdSetIterator;
-import org.apache.lucene.util.BitSetIterator;
 import org.apache.lucene.util.Counter;
-import org.apache.lucene.util.FixedBitSet;
 import org.apache.lucene.util.packed.PackedInts;
 import org.apache.lucene.util.packed.PackedLongValues;
 
@@ -32,7 +30,7 @@ import org.apache.lucene.util.packed.PackedLongValues;
  *  segment flushes. */
 class NormValuesWriter {
 
-  private FixedBitSet docsWithField;
+  private DocsWithFieldSet docsWithField;
   private PackedLongValues.Builder pending;
   private final Counter iwBytesUsed;
   private long bytesUsed;
@@ -40,7 +38,7 @@ class NormValuesWriter {
   private int lastDocID = -1;
 
   public NormValuesWriter(FieldInfo fieldInfo, Counter iwBytesUsed) {
-    docsWithField = new FixedBitSet(64);
+    docsWithField = new DocsWithFieldSet();
     pending = PackedLongValues.deltaPackedBuilder(PackedInts.COMPACT);
     bytesUsed = pending.ramBytesUsed() + docsWithField.ramBytesUsed();
     this.fieldInfo = fieldInfo;
@@ -54,8 +52,7 @@ class NormValuesWriter {
     }
 
     pending.add(value);
-    docsWithField = FixedBitSet.ensureCapacity(docsWithField, docID);
-    docsWithField.set(docID);
+    docsWithField.add(docID);
 
     updateBytesUsed();
 
@@ -82,7 +79,7 @@ class NormValuesWriter {
                                    if (fieldInfo != NormValuesWriter.this.fieldInfo) {
                                      throw new IllegalArgumentException("wrong fieldInfo");
                                    }
-                                   return new BufferedNorms(values, docsWithField);
+                                   return new BufferedNorms(values, docsWithField.iterator());
                                   }
 
                                   @Override
@@ -108,9 +105,9 @@ class NormValuesWriter {
     final DocIdSetIterator docsWithField;
     private long value;
 
-    BufferedNorms(PackedLongValues values, FixedBitSet docsWithFields) {
+    BufferedNorms(PackedLongValues values, DocIdSetIterator docsWithFields) {
       this.iter = values.iterator();
-      this.docsWithField = new BitSetIterator(docsWithFields, values.size());
+      this.docsWithField = docsWithFields;
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c415bc8d/lucene/core/src/java/org/apache/lucene/index/NumericDocValuesWriter.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/NumericDocValuesWriter.java b/lucene/core/src/java/org/apache/lucene/index/NumericDocValuesWriter.java
index 24a7010..d4dd66a 100644
--- a/lucene/core/src/java/org/apache/lucene/index/NumericDocValuesWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/index/NumericDocValuesWriter.java
@@ -21,9 +21,7 @@ import java.io.IOException;
 
 import org.apache.lucene.codecs.DocValuesConsumer;
 import org.apache.lucene.search.DocIdSetIterator;
-import org.apache.lucene.util.BitSetIterator;
 import org.apache.lucene.util.Counter;
-import org.apache.lucene.util.FixedBitSet;
 import org.apache.lucene.util.packed.PackedInts;
 import org.apache.lucene.util.packed.PackedLongValues;
 
@@ -34,13 +32,13 @@ class NumericDocValuesWriter extends DocValuesWriter {
   private PackedLongValues.Builder pending;
   private final Counter iwBytesUsed;
   private long bytesUsed;
-  private FixedBitSet docsWithField;
+  private DocsWithFieldSet docsWithField;
   private final FieldInfo fieldInfo;
   private int lastDocID = -1;
 
   public NumericDocValuesWriter(FieldInfo fieldInfo, Counter iwBytesUsed) {
     pending = PackedLongValues.deltaPackedBuilder(PackedInts.COMPACT);
-    docsWithField = new FixedBitSet(64);
+    docsWithField = new DocsWithFieldSet();
     bytesUsed = pending.ramBytesUsed() + docsWithField.ramBytesUsed();
     this.fieldInfo = fieldInfo;
     this.iwBytesUsed = iwBytesUsed;
@@ -53,8 +51,7 @@ class NumericDocValuesWriter extends DocValuesWriter {
     }
 
     pending.add(value);
-    docsWithField = FixedBitSet.ensureCapacity(docsWithField, docID);
-    docsWithField.set(docID);
+    docsWithField.add(docID);
 
     updateBytesUsed();
 
@@ -83,7 +80,7 @@ class NumericDocValuesWriter extends DocValuesWriter {
                                    if (fieldInfo != NumericDocValuesWriter.this.fieldInfo) {
                                      throw new IllegalArgumentException("wrong fieldInfo");
                                    }
-                                   return new BufferedNumericDocValues(values, docsWithField);
+                                   return new BufferedNumericDocValues(values, docsWithField.iterator());
                                  }
                                });
   }
@@ -94,9 +91,9 @@ class NumericDocValuesWriter extends DocValuesWriter {
     final DocIdSetIterator docsWithField;
     private long value;
 
-    BufferedNumericDocValues(PackedLongValues values, FixedBitSet docsWithFields) {
+    BufferedNumericDocValues(PackedLongValues values, DocIdSetIterator docsWithFields) {
       this.iter = values.iterator();
-      this.docsWithField = new BitSetIterator(docsWithFields, values.size());
+      this.docsWithField = docsWithFields;
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c415bc8d/lucene/core/src/java/org/apache/lucene/index/SortedDocValuesWriter.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/SortedDocValuesWriter.java b/lucene/core/src/java/org/apache/lucene/index/SortedDocValuesWriter.java
index 7e43e49..e439caf 100644
--- a/lucene/core/src/java/org/apache/lucene/index/SortedDocValuesWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/index/SortedDocValuesWriter.java
@@ -22,13 +22,11 @@ import java.io.IOException;
 
 import org.apache.lucene.codecs.DocValuesConsumer;
 import org.apache.lucene.search.DocIdSetIterator;
-import org.apache.lucene.util.BitSetIterator;
 import org.apache.lucene.util.ByteBlockPool;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefHash.DirectBytesStartArray;
 import org.apache.lucene.util.BytesRefHash;
 import org.apache.lucene.util.Counter;
-import org.apache.lucene.util.FixedBitSet;
 import org.apache.lucene.util.packed.PackedInts;
 import org.apache.lucene.util.packed.PackedLongValues;
 
@@ -37,7 +35,7 @@ import org.apache.lucene.util.packed.PackedLongValues;
 class SortedDocValuesWriter extends DocValuesWriter {
   final BytesRefHash hash;
   private PackedLongValues.Builder pending;
-  private FixedBitSet docsWithField;
+  private DocsWithFieldSet docsWithField;
   private final Counter iwBytesUsed;
   private long bytesUsed; // this currently only tracks differences in 'pending'
   private final FieldInfo fieldInfo;
@@ -52,7 +50,7 @@ class SortedDocValuesWriter extends DocValuesWriter {
             BytesRefHash.DEFAULT_CAPACITY,
             new DirectBytesStartArray(BytesRefHash.DEFAULT_CAPACITY, iwBytesUsed));
     pending = PackedLongValues.deltaPackedBuilder(PackedInts.COMPACT);
-    docsWithField = new FixedBitSet(64);
+    docsWithField = new DocsWithFieldSet();
     bytesUsed = pending.ramBytesUsed() + docsWithField.ramBytesUsed();
     iwBytesUsed.addAndGet(bytesUsed);
   }
@@ -69,8 +67,7 @@ class SortedDocValuesWriter extends DocValuesWriter {
     }
 
     addOneValue(value);
-    docsWithField = FixedBitSet.ensureCapacity(docsWithField, docID);
-    docsWithField.set(docID);
+    docsWithField.add(docID);
 
     lastDocID = docID;
   }
@@ -121,7 +118,7 @@ class SortedDocValuesWriter extends DocValuesWriter {
                                   if (fieldInfoIn != fieldInfo) {
                                     throw new IllegalArgumentException("wrong fieldInfo");
                                   }
-                                  return new BufferedSortedDocValues(hash, valueCount, ords, sortedValues, ordMap, docsWithField);
+                                  return new BufferedSortedDocValues(hash, valueCount, ords, sortedValues, ordMap, docsWithField.iterator());
                                 }
                               });
   }
@@ -136,13 +133,13 @@ class SortedDocValuesWriter extends DocValuesWriter {
     final PackedLongValues.Iterator iter;
     final DocIdSetIterator docsWithField;
 
-    public BufferedSortedDocValues(BytesRefHash hash, int valueCount, PackedLongValues docToOrd, int[] sortedValues, int[] ordMap, FixedBitSet docsWithField) {
+    public BufferedSortedDocValues(BytesRefHash hash, int valueCount, PackedLongValues docToOrd, int[] sortedValues, int[] ordMap, DocIdSetIterator docsWithField) {
       this.hash = hash;
       this.valueCount = valueCount;
       this.sortedValues = sortedValues;
       this.iter = docToOrd.iterator();
       this.ordMap = ordMap;
-      this.docsWithField = new BitSetIterator(docsWithField, docToOrd.size());
+      this.docsWithField = docsWithField;
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c415bc8d/lucene/core/src/java/org/apache/lucene/index/SortedNumericDocValuesWriter.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/SortedNumericDocValuesWriter.java b/lucene/core/src/java/org/apache/lucene/index/SortedNumericDocValuesWriter.java
index 3f50623..75236cc 100644
--- a/lucene/core/src/java/org/apache/lucene/index/SortedNumericDocValuesWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/index/SortedNumericDocValuesWriter.java
@@ -23,9 +23,7 @@ import java.util.Arrays;
 import org.apache.lucene.codecs.DocValuesConsumer;
 import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.util.ArrayUtil;
-import org.apache.lucene.util.BitSetIterator;
 import org.apache.lucene.util.Counter;
-import org.apache.lucene.util.FixedBitSet;
 import org.apache.lucene.util.RamUsageEstimator;
 import org.apache.lucene.util.packed.PackedInts;
 import org.apache.lucene.util.packed.PackedLongValues;
@@ -34,7 +32,7 @@ import org.apache.lucene.util.packed.PackedLongValues;
 class SortedNumericDocValuesWriter extends DocValuesWriter {
   private PackedLongValues.Builder pending; // stream of all values
   private PackedLongValues.Builder pendingCounts; // count of values per doc
-  private FixedBitSet docsWithField;
+  private DocsWithFieldSet docsWithField;
   private final Counter iwBytesUsed;
   private long bytesUsed; // this only tracks differences in 'pending' and 'pendingCounts'
   private final FieldInfo fieldInfo;
@@ -47,7 +45,7 @@ class SortedNumericDocValuesWriter extends DocValuesWriter {
     this.iwBytesUsed = iwBytesUsed;
     pending = PackedLongValues.deltaPackedBuilder(PackedInts.COMPACT);
     pendingCounts = PackedLongValues.deltaPackedBuilder(PackedInts.COMPACT);
-    docsWithField = new FixedBitSet(64);
+    docsWithField = new DocsWithFieldSet();
     bytesUsed = pending.ramBytesUsed() + pendingCounts.ramBytesUsed() + docsWithField.ramBytesUsed() + RamUsageEstimator.sizeOf(currentValues);
     iwBytesUsed.addAndGet(bytesUsed);
   }
@@ -76,8 +74,7 @@ class SortedNumericDocValuesWriter extends DocValuesWriter {
     pendingCounts.add(currentUpto);
     currentUpto = 0;
 
-    docsWithField = FixedBitSet.ensureCapacity(docsWithField, currentDoc);
-    docsWithField.set(currentDoc);
+    docsWithField.add(currentDoc);
   }
 
   @Override
@@ -112,7 +109,7 @@ class SortedNumericDocValuesWriter extends DocValuesWriter {
                                          if (fieldInfoIn != fieldInfo) {
                                            throw new IllegalArgumentException("wrong fieldInfo");
                                          }
-                                         return new BufferedSortedNumericDocValues(values, valueCounts, docsWithField);
+                                         return new BufferedSortedNumericDocValues(values, valueCounts, docsWithField.iterator());
                                        }
                                      });
   }
@@ -124,10 +121,10 @@ class SortedNumericDocValuesWriter extends DocValuesWriter {
     private int valueCount;
     private int valueUpto;
 
-    public BufferedSortedNumericDocValues(PackedLongValues values, PackedLongValues valueCounts, FixedBitSet docsWithField) {
+    public BufferedSortedNumericDocValues(PackedLongValues values, PackedLongValues valueCounts, DocIdSetIterator docsWithField) {
       valuesIter = values.iterator();
       valueCountsIter = valueCounts.iterator();
-      this.docsWithField = new BitSetIterator(docsWithField, values.size());
+      this.docsWithField = docsWithField;
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c415bc8d/lucene/core/src/java/org/apache/lucene/index/SortedSetDocValuesWriter.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/SortedSetDocValuesWriter.java b/lucene/core/src/java/org/apache/lucene/index/SortedSetDocValuesWriter.java
index 35157d4..0f4fb5e 100644
--- a/lucene/core/src/java/org/apache/lucene/index/SortedSetDocValuesWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/index/SortedSetDocValuesWriter.java
@@ -24,13 +24,11 @@ import java.util.Arrays;
 import org.apache.lucene.codecs.DocValuesConsumer;
 import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.util.ArrayUtil;
-import org.apache.lucene.util.BitSetIterator;
 import org.apache.lucene.util.ByteBlockPool;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefHash.DirectBytesStartArray;
 import org.apache.lucene.util.BytesRefHash;
 import org.apache.lucene.util.Counter;
-import org.apache.lucene.util.FixedBitSet;
 import org.apache.lucene.util.packed.PackedInts;
 import org.apache.lucene.util.packed.PackedLongValues;
 
@@ -40,7 +38,7 @@ class SortedSetDocValuesWriter extends DocValuesWriter {
   final BytesRefHash hash;
   private PackedLongValues.Builder pending; // stream of all termIDs
   private PackedLongValues.Builder pendingCounts; // termIDs per doc
-  private FixedBitSet docsWithField;
+  private DocsWithFieldSet docsWithField;
   private final Counter iwBytesUsed;
   private long bytesUsed; // this only tracks differences in 'pending' and 'pendingCounts'
   private final FieldInfo fieldInfo;
@@ -59,7 +57,7 @@ class SortedSetDocValuesWriter extends DocValuesWriter {
             new DirectBytesStartArray(BytesRefHash.DEFAULT_CAPACITY, iwBytesUsed));
     pending = PackedLongValues.packedBuilder(PackedInts.COMPACT);
     pendingCounts = PackedLongValues.deltaPackedBuilder(PackedInts.COMPACT);
-    docsWithField = new FixedBitSet(64);
+    docsWithField = new DocsWithFieldSet();
     bytesUsed = pending.ramBytesUsed() + pendingCounts.ramBytesUsed();
     iwBytesUsed.addAndGet(bytesUsed);
   }
@@ -103,8 +101,7 @@ class SortedSetDocValuesWriter extends DocValuesWriter {
     pendingCounts.add(count);
     maxCount = Math.max(maxCount, count);
     currentUpto = 0;
-    docsWithField = FixedBitSet.ensureCapacity(docsWithField, currentDoc);
-    docsWithField.set(currentDoc);
+    docsWithField.add(currentDoc);
   }
 
   @Override
@@ -158,7 +155,7 @@ class SortedSetDocValuesWriter extends DocValuesWriter {
                                      if (fieldInfoIn != fieldInfo) {
                                        throw new IllegalArgumentException("wrong fieldInfo");
                                      }
-                                     return new BufferedSortedSetDocValues(sortedValues, ordMap, hash, ords, ordCounts, maxCount, docsWithField);
+                                     return new BufferedSortedSetDocValues(sortedValues, ordMap, hash, ords, ordCounts, maxCount, docsWithField.iterator());
                                    }
                                  });
   }
@@ -176,14 +173,14 @@ class SortedSetDocValuesWriter extends DocValuesWriter {
     private int ordCount;
     private int ordUpto;
 
-    public BufferedSortedSetDocValues(int[] sortedValues, int[] ordMap, BytesRefHash hash, PackedLongValues ords, PackedLongValues ordCounts, int maxCount, FixedBitSet docsWithField) {
+    public BufferedSortedSetDocValues(int[] sortedValues, int[] ordMap, BytesRefHash hash, PackedLongValues ords, PackedLongValues ordCounts, int maxCount, DocIdSetIterator docsWithField) {
       this.currentDoc = new int[maxCount];
       this.sortedValues = sortedValues;
       this.ordMap = ordMap;
       this.hash = hash;
       this.ordsIter = ords.iterator();
       this.ordCountsIter = ordCounts.iterator();
-      this.docsWithField = new BitSetIterator(docsWithField, ordCounts.size());
+      this.docsWithField = docsWithField;
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c415bc8d/lucene/core/src/test/org/apache/lucene/index/TestDocsWithFieldSet.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestDocsWithFieldSet.java b/lucene/core/src/test/org/apache/lucene/index/TestDocsWithFieldSet.java
new file mode 100644
index 0000000..b719adf
--- /dev/null
+++ b/lucene/core/src/test/org/apache/lucene/index/TestDocsWithFieldSet.java
@@ -0,0 +1,80 @@
+/*
+ * 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 org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.TestUtil;
+
+public class TestDocsWithFieldSet extends LuceneTestCase {
+
+  public void testDense() throws IOException {
+    DocsWithFieldSet set = new DocsWithFieldSet();
+    DocIdSetIterator it = set.iterator();
+    assertEquals(DocIdSetIterator.NO_MORE_DOCS, it.nextDoc());
+
+    set.add(0);
+    it = set.iterator();
+    assertEquals(0, it.nextDoc());
+    assertEquals(DocIdSetIterator.NO_MORE_DOCS, it.nextDoc());
+
+    long ramBytesUsed = set.ramBytesUsed();
+    for (int i = 1; i < 1000; ++i) {
+      set.add(i);
+    }
+    assertEquals(ramBytesUsed, set.ramBytesUsed());
+    it = set.iterator();
+    for (int i = 0; i < 1000; ++i) {
+      assertEquals(i, it.nextDoc());
+    }
+    assertEquals(DocIdSetIterator.NO_MORE_DOCS, it.nextDoc());
+  }
+
+  public void testSparse() throws IOException {
+    DocsWithFieldSet set = new DocsWithFieldSet();
+    int doc = random().nextInt(10000);
+    set.add(doc);
+    DocIdSetIterator it = set.iterator();
+    assertEquals(doc, it.nextDoc());
+    assertEquals(DocIdSetIterator.NO_MORE_DOCS, it.nextDoc());
+    int doc2 = doc + TestUtil.nextInt(random(), 1, 100);
+    set.add(doc2);
+    it = set.iterator();
+    assertEquals(doc, it.nextDoc());
+    assertEquals(doc2, it.nextDoc());
+    assertEquals(DocIdSetIterator.NO_MORE_DOCS, it.nextDoc());
+  }
+
+  public void testDenseThenSparse() throws IOException {
+    int denseCount = random().nextInt(10000);
+    int nextDoc = denseCount + random().nextInt(10000);
+    DocsWithFieldSet set = new DocsWithFieldSet();
+    for (int i = 0; i < denseCount; ++i) {
+      set.add(i);
+    }
+    set.add(nextDoc);
+    DocIdSetIterator it = set.iterator();
+    for (int i = 0; i < denseCount; ++i) {
+      assertEquals(i, it.nextDoc());
+    }
+    assertEquals(nextDoc, it.nextDoc());
+    assertEquals(DocIdSetIterator.NO_MORE_DOCS, it.nextDoc());
+  }
+
+}


[3/4] lucene-solr:master: LUCENE-7531: Removed packing support from FST.

Posted by jp...@apache.org.
LUCENE-7531: Removed packing support from FST.


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

Branch: refs/heads/master
Commit: c4c5e868d2304148c146d52833ac2c80b0541dc3
Parents: 6b9f113
Author: Adrien Grand <jp...@gmail.com>
Authored: Thu Nov 10 14:09:52 2016 +0100
Committer: Adrien Grand <jp...@gmail.com>
Committed: Thu Nov 10 15:01:49 2016 +0100

----------------------------------------------------------------------
 lucene/analysis/kuromoji/ivy.xml                |   2 +-
 .../ja/dict/TokenInfoDictionary$fst.dat         | Bin 1716198 -> 1698563 bytes
 .../analysis/ja/util/ConnectionCostsWriter.java |   1 -
 .../ja/util/TokenInfoDictionaryBuilder.java     |   4 +-
 .../blocktreeords/OrdsBlockTreeTermsWriter.java |   4 +-
 .../codecs/memory/MemoryPostingsFormat.java     |  18 +-
 .../codecs/blocktree/BlockTreeTermsWriter.java  |   4 +-
 .../org/apache/lucene/util/fst/Builder.java     |  28 +-
 .../java/org/apache/lucene/util/fst/FST.java    | 633 ++-----------------
 .../apache/lucene/util/fst/package-info.java    |   1 -
 .../org/apache/lucene/util/fst/Test2BFST.java   |  17 +-
 .../org/apache/lucene/util/fst/TestFSTs.java    |  27 +-
 .../idversion/VersionBlockTreeTermsWriter.java  |   4 +-
 .../suggest/fst/FSTCompletionBuilder.java       |   4 +-
 .../org/apache/lucene/util/fst/FSTTester.java   |  14 +-
 15 files changed, 81 insertions(+), 680 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c4c5e868/lucene/analysis/kuromoji/ivy.xml
----------------------------------------------------------------------
diff --git a/lucene/analysis/kuromoji/ivy.xml b/lucene/analysis/kuromoji/ivy.xml
index 10eba4e..eb08509 100644
--- a/lucene/analysis/kuromoji/ivy.xml
+++ b/lucene/analysis/kuromoji/ivy.xml
@@ -27,7 +27,7 @@
 
   <dependencies>
     <dependency org="mecab" name="mecab-ipadic" rev="${/mecab/mecab-ipadic}" conf="ipadic"> 
-      <artifact name="ipadic" type=".tar.gz" url="http://mecab.googlecode.com/files/mecab-ipadic-2.7.0-20070801.tar.gz"/>
+      <artifact name="ipadic" type=".tar.gz" url="http://jaist.dl.sourceforge.net/project/mecab/mecab-ipadic/2.7.0-20070801/mecab-ipadic-2.7.0-20070801.tar.gz"/>
     </dependency>
     <dependency org="mecab" name="mecab-naist-jdic" rev="${/mecab/mecab-naist-jdic}" conf="naist">
       <artifact name="mecab-naist-jdic" type=".tar.gz" url="http://sourceforge.jp/frs/redir.php?m=iij&amp;f=/naist-jdic/53500/mecab-naist-jdic-0.6.3b-20111013.tar.gz"/>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c4c5e868/lucene/analysis/kuromoji/src/resources/org/apache/lucene/analysis/ja/dict/TokenInfoDictionary$fst.dat
----------------------------------------------------------------------
diff --git a/lucene/analysis/kuromoji/src/resources/org/apache/lucene/analysis/ja/dict/TokenInfoDictionary$fst.dat b/lucene/analysis/kuromoji/src/resources/org/apache/lucene/analysis/ja/dict/TokenInfoDictionary$fst.dat
index 8935809..6cfad72 100644
Binary files a/lucene/analysis/kuromoji/src/resources/org/apache/lucene/analysis/ja/dict/TokenInfoDictionary$fst.dat and b/lucene/analysis/kuromoji/src/resources/org/apache/lucene/analysis/ja/dict/TokenInfoDictionary$fst.dat differ

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c4c5e868/lucene/analysis/kuromoji/src/tools/java/org/apache/lucene/analysis/ja/util/ConnectionCostsWriter.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/kuromoji/src/tools/java/org/apache/lucene/analysis/ja/util/ConnectionCostsWriter.java b/lucene/analysis/kuromoji/src/tools/java/org/apache/lucene/analysis/ja/util/ConnectionCostsWriter.java
index 54382ed..6ad8a68 100644
--- a/lucene/analysis/kuromoji/src/tools/java/org/apache/lucene/analysis/ja/util/ConnectionCostsWriter.java
+++ b/lucene/analysis/kuromoji/src/tools/java/org/apache/lucene/analysis/ja/util/ConnectionCostsWriter.java
@@ -28,7 +28,6 @@ import org.apache.lucene.analysis.ja.dict.ConnectionCosts;
 import org.apache.lucene.codecs.CodecUtil;
 import org.apache.lucene.store.DataOutput;
 import org.apache.lucene.store.OutputStreamDataOutput;
-import org.apache.lucene.util.BitUtil;
 
 public final class ConnectionCostsWriter {
   

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c4c5e868/lucene/analysis/kuromoji/src/tools/java/org/apache/lucene/analysis/ja/util/TokenInfoDictionaryBuilder.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/kuromoji/src/tools/java/org/apache/lucene/analysis/ja/util/TokenInfoDictionaryBuilder.java b/lucene/analysis/kuromoji/src/tools/java/org/apache/lucene/analysis/ja/util/TokenInfoDictionaryBuilder.java
index 61d6f27..1b8abbb 100644
--- a/lucene/analysis/kuromoji/src/tools/java/org/apache/lucene/analysis/ja/util/TokenInfoDictionaryBuilder.java
+++ b/lucene/analysis/kuromoji/src/tools/java/org/apache/lucene/analysis/ja/util/TokenInfoDictionaryBuilder.java
@@ -33,12 +33,10 @@ import java.util.Comparator;
 import java.util.List;
 
 import org.apache.lucene.analysis.ja.util.DictionaryBuilder.DictionaryFormat;
-import org.apache.lucene.util.IntsRef;
 import org.apache.lucene.util.IntsRefBuilder;
 import org.apache.lucene.util.fst.Builder;
 import org.apache.lucene.util.fst.FST;
 import org.apache.lucene.util.fst.PositiveIntOutputs;
-import org.apache.lucene.util.packed.PackedInts;
 
 import com.ibm.icu.text.Normalizer2;
 
@@ -133,7 +131,7 @@ public class TokenInfoDictionaryBuilder {
     System.out.println("  encode...");
 
     PositiveIntOutputs fstOutput = PositiveIntOutputs.getSingleton();
-    Builder<Long> fstBuilder = new Builder<>(FST.INPUT_TYPE.BYTE2, 0, 0, true, true, Integer.MAX_VALUE, fstOutput, true, PackedInts.DEFAULT, true, 15);
+    Builder<Long> fstBuilder = new Builder<>(FST.INPUT_TYPE.BYTE2, 0, 0, true, true, Integer.MAX_VALUE, fstOutput, true, 15);
     IntsRefBuilder scratch = new IntsRefBuilder();
     long ord = -1; // first ord will be 0
     String lastValue = null;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c4c5e868/lucene/codecs/src/java/org/apache/lucene/codecs/blocktreeords/OrdsBlockTreeTermsWriter.java
----------------------------------------------------------------------
diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/blocktreeords/OrdsBlockTreeTermsWriter.java b/lucene/codecs/src/java/org/apache/lucene/codecs/blocktreeords/OrdsBlockTreeTermsWriter.java
index fb682fd..b16bb15 100644
--- a/lucene/codecs/src/java/org/apache/lucene/codecs/blocktreeords/OrdsBlockTreeTermsWriter.java
+++ b/lucene/codecs/src/java/org/apache/lucene/codecs/blocktreeords/OrdsBlockTreeTermsWriter.java
@@ -48,7 +48,6 @@ import org.apache.lucene.util.fst.Builder;
 import org.apache.lucene.util.fst.BytesRefFSTEnum;
 import org.apache.lucene.util.fst.FST;
 import org.apache.lucene.util.fst.Util;
-import org.apache.lucene.util.packed.PackedInts;
 
 /*
   TODO:
@@ -363,8 +362,7 @@ public final class OrdsBlockTreeTermsWriter extends FieldsConsumer {
 
       final Builder<Output> indexBuilder = new Builder<>(FST.INPUT_TYPE.BYTE1,
                                                          0, 0, true, false, Integer.MAX_VALUE,
-                                                         FST_OUTPUTS, false,
-                                                         PackedInts.COMPACT, true, 15);
+                                                         FST_OUTPUTS, true, 15);
       //if (DEBUG) {
       //  System.out.println("  compile index for prefix=" + prefix);
       //}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c4c5e868/lucene/codecs/src/java/org/apache/lucene/codecs/memory/MemoryPostingsFormat.java
----------------------------------------------------------------------
diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/memory/MemoryPostingsFormat.java b/lucene/codecs/src/java/org/apache/lucene/codecs/memory/MemoryPostingsFormat.java
index 1427dec..2f71765 100644
--- a/lucene/codecs/src/java/org/apache/lucene/codecs/memory/MemoryPostingsFormat.java
+++ b/lucene/codecs/src/java/org/apache/lucene/codecs/memory/MemoryPostingsFormat.java
@@ -81,9 +81,6 @@ import org.apache.lucene.util.packed.PackedInts;
 // loads itself in ram?
 public final class MemoryPostingsFormat extends PostingsFormat {
 
-  private final boolean doPackFST;
-  private final float acceptableOverheadRatio;
-
   public MemoryPostingsFormat() {
     this(false, PackedInts.DEFAULT);
   }
@@ -97,13 +94,11 @@ public final class MemoryPostingsFormat extends PostingsFormat {
    */
   public MemoryPostingsFormat(boolean doPackFST, float acceptableOverheadRatio) {
     super("Memory");
-    this.doPackFST = doPackFST;
-    this.acceptableOverheadRatio = acceptableOverheadRatio;
   }
   
   @Override
   public String toString() {
-    return "PostingsFormat(name=" + getName() + " doPackFST= " + doPackFST + ")";
+    return "PostingsFormat(name=" + getName() + ")";
   }
 
   private final static class TermsWriter {
@@ -111,16 +106,12 @@ public final class MemoryPostingsFormat extends PostingsFormat {
     private final FieldInfo field;
     private final Builder<BytesRef> builder;
     private final ByteSequenceOutputs outputs = ByteSequenceOutputs.getSingleton();
-    private final boolean doPackFST;
-    private final float acceptableOverheadRatio;
     private int termCount;
 
-    public TermsWriter(IndexOutput out, FieldInfo field, boolean doPackFST, float acceptableOverheadRatio) {
+    public TermsWriter(IndexOutput out, FieldInfo field) {
       this.out = out;
       this.field = field;
-      this.doPackFST = doPackFST;
-      this.acceptableOverheadRatio = acceptableOverheadRatio;
-      builder = new Builder<>(FST.INPUT_TYPE.BYTE1, 0, 0, true, true, Integer.MAX_VALUE, outputs, doPackFST, acceptableOverheadRatio, true, 15);
+      builder = new Builder<>(FST.INPUT_TYPE.BYTE1, 0, 0, true, true, Integer.MAX_VALUE, outputs, true, 15);
     }
 
     private class PostingsWriter {
@@ -307,8 +298,7 @@ public final class MemoryPostingsFormat extends PostingsFormat {
         TermsEnum termsEnum = terms.iterator();
 
         FieldInfo fieldInfo = state.fieldInfos.fieldInfo(field);
-        TermsWriter termsWriter = new TermsWriter(out, fieldInfo,
-                                                  doPackFST, acceptableOverheadRatio);
+        TermsWriter termsWriter = new TermsWriter(out, fieldInfo);
 
         FixedBitSet docsSeen = new FixedBitSet(state.segmentInfo.maxDoc());
         long sumTotalTermFreq = 0;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c4c5e868/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BlockTreeTermsWriter.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BlockTreeTermsWriter.java b/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BlockTreeTermsWriter.java
index a4a150b..bdacc22 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BlockTreeTermsWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BlockTreeTermsWriter.java
@@ -48,7 +48,6 @@ import org.apache.lucene.util.fst.ByteSequenceOutputs;
 import org.apache.lucene.util.fst.BytesRefFSTEnum;
 import org.apache.lucene.util.fst.FST;
 import org.apache.lucene.util.fst.Util;
-import org.apache.lucene.util.packed.PackedInts;
 
 /*
   TODO:
@@ -456,8 +455,7 @@ public final class BlockTreeTermsWriter extends FieldsConsumer {
       final ByteSequenceOutputs outputs = ByteSequenceOutputs.getSingleton();
       final Builder<BytesRef> indexBuilder = new Builder<>(FST.INPUT_TYPE.BYTE1,
                                                            0, 0, true, false, Integer.MAX_VALUE,
-                                                           outputs, false,
-                                                           PackedInts.COMPACT, true, 15);
+                                                           outputs, true, 15);
       //if (DEBUG) {
       //  System.out.println("  compile index for prefix=" + prefix);
       //}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c4c5e868/lucene/core/src/java/org/apache/lucene/util/fst/Builder.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/util/fst/Builder.java b/lucene/core/src/java/org/apache/lucene/util/fst/Builder.java
index c5ab849..428edd3 100644
--- a/lucene/core/src/java/org/apache/lucene/util/fst/Builder.java
+++ b/lucene/core/src/java/org/apache/lucene/util/fst/Builder.java
@@ -23,7 +23,6 @@ import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.IntsRef;
 import org.apache.lucene.util.IntsRefBuilder;
 import org.apache.lucene.util.fst.FST.INPUT_TYPE; // javadoc
-import org.apache.lucene.util.packed.PackedInts;
 
 // TODO: could we somehow stream an FST to disk while we
 // build it?
@@ -69,10 +68,6 @@ public class Builder<T> {
   private final int shareMaxTailLength;
 
   private final IntsRefBuilder lastInput = new IntsRefBuilder();
-  
-  // for packing
-  private final boolean doPackFST;
-  private final float acceptableOverheadRatio;
 
   // NOTE: cutting this over to ArrayList instead loses ~6%
   // in build performance on 9.8M Wikipedia terms; so we
@@ -99,11 +94,10 @@ public class Builder<T> {
   /**
    * Instantiates an FST/FSA builder without any pruning. A shortcut
    * to {@link #Builder(FST.INPUT_TYPE, int, int, boolean,
-   * boolean, int, Outputs, boolean, float,
-   * boolean, int)} with pruning options turned off.
+   * boolean, int, Outputs, boolean, int)} with pruning options turned off.
    */
   public Builder(FST.INPUT_TYPE inputType, Outputs<T> outputs) {
-    this(inputType, 0, 0, true, true, Integer.MAX_VALUE, outputs, false, PackedInts.COMPACT, true, 15);
+    this(inputType, 0, 0, true, true, Integer.MAX_VALUE, outputs, true, 15);
   }
 
   /**
@@ -143,11 +137,6 @@ public class Builder<T> {
    *    FSA, use {@link NoOutputs#getSingleton()} and {@link NoOutputs#getNoOutput()} as the
    *    singleton output object.
    *
-   * @param doPackFST Pass true to create a packed FST.
-   * 
-   * @param acceptableOverheadRatio How to trade speed for space when building the FST. This option
-   *    is only relevant when doPackFST is true. @see PackedInts#getMutable(int, int, float)
-   *
    * @param allowArrayArcs Pass false to disable the array arc optimization
    *    while building the FST; this will make the resulting
    *    FST smaller but slower to traverse.
@@ -159,16 +148,13 @@ public class Builder<T> {
    */
   public Builder(FST.INPUT_TYPE inputType, int minSuffixCount1, int minSuffixCount2, boolean doShareSuffix,
                  boolean doShareNonSingletonNodes, int shareMaxTailLength, Outputs<T> outputs,
-                 boolean doPackFST, float acceptableOverheadRatio, boolean allowArrayArcs,
-                 int bytesPageBits) {
+                 boolean allowArrayArcs, int bytesPageBits) {
     this.minSuffixCount1 = minSuffixCount1;
     this.minSuffixCount2 = minSuffixCount2;
     this.doShareNonSingletonNodes = doShareNonSingletonNodes;
     this.shareMaxTailLength = shareMaxTailLength;
-    this.doPackFST = doPackFST;
-    this.acceptableOverheadRatio = acceptableOverheadRatio;
     this.allowArrayArcs = allowArrayArcs;
-    fst = new FST<>(inputType, outputs, doPackFST, acceptableOverheadRatio, bytesPageBits);
+    fst = new FST<>(inputType, outputs, bytesPageBits);
     bytes = fst.bytes;
     assert bytes != null;
     if (doShareSuffix) {
@@ -496,11 +482,7 @@ public class Builder<T> {
     //if (DEBUG) System.out.println("  builder.finish root.isFinal=" + root.isFinal + " root.output=" + root.output);
     fst.finish(compileNode(root, lastInput.length()).node);
 
-    if (doPackFST) {
-      return fst.pack(this, 3, Math.max(10, (int) (getNodeCount()/4)), acceptableOverheadRatio);
-    } else {
-      return fst;
-    }
+    return fst;
   }
 
   private void compileAllTargets(UnCompiledNode<T> node, int tailLength) throws IOException {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c4c5e868/lucene/core/src/java/org/apache/lucene/util/fst/FST.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/util/fst/FST.java b/lucene/core/src/java/org/apache/lucene/util/fst/FST.java
index 4a0a3a9..5ea6dab 100644
--- a/lucene/core/src/java/org/apache/lucene/util/fst/FST.java
+++ b/lucene/core/src/java/org/apache/lucene/util/fst/FST.java
@@ -24,13 +24,9 @@ import java.io.InputStream;
 import java.io.OutputStream;
 import java.nio.file.Files;
 import java.nio.file.Path;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
 
 import org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.index.CorruptIndexException;
 import org.apache.lucene.store.ByteArrayDataOutput;
 import org.apache.lucene.store.DataInput;
 import org.apache.lucene.store.DataOutput;
@@ -38,13 +34,9 @@ import org.apache.lucene.store.InputStreamDataInput;
 import org.apache.lucene.store.OutputStreamDataOutput;
 import org.apache.lucene.store.RAMOutputStream;
 import org.apache.lucene.util.Accountable;
-import org.apache.lucene.util.Accountables;
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.Constants;
-import org.apache.lucene.util.PriorityQueue;
 import org.apache.lucene.util.RamUsageEstimator;
-import org.apache.lucene.util.packed.GrowableWriter;
-import org.apache.lucene.util.packed.PackedInts;
 
 // TODO: break this into WritableFST and ReadOnlyFST.. then
 // we can have subclasses of ReadOnlyFST to handle the
@@ -90,14 +82,6 @@ public final class FST<T> implements Accountable {
 
   static final int BIT_ARC_HAS_FINAL_OUTPUT = 1 << 5;
 
-  // Arcs are stored as fixed-size (per entry) array, so
-  // that we can find an arc using binary search.  We do
-  // this when number of arcs is > NUM_ARCS_ARRAY:
-
-  // If set, the target node is delta coded vs current
-  // position:
-  private static final int BIT_TARGET_DELTA = 1 << 6;
-
   // We use this as a marker (because this one flag is
   // illegal by itself ...):
   private static final byte ARCS_AS_FIXED_ARRAY = BIT_ARC_HAS_FINAL_OUTPUT;
@@ -137,7 +121,9 @@ public final class FST<T> implements Accountable {
   /** Don't store arcWithOutputCount anymore */
   private static final int VERSION_NO_NODE_ARC_COUNTS = 5;
 
-  private static final int VERSION_CURRENT = VERSION_NO_NODE_ARC_COUNTS;
+  private static final int VERSION_PACKED_REMOVED = 6;
+
+  private static final int VERSION_CURRENT = VERSION_PACKED_REMOVED;
 
   // Never serialized; just used to represent the virtual
   // final node w/ no arcs:
@@ -168,9 +154,6 @@ public final class FST<T> implements Accountable {
 
   public final Outputs<T> outputs;
 
-  private final boolean packed;
-  private PackedInts.Reader nodeRefToAddress;
-
   private Arc<T> cachedRootArcs[];
 
   /** Represents a single arc. */
@@ -273,18 +256,11 @@ public final class FST<T> implements Accountable {
     return (flags & bit) != 0;
   }
 
-  private GrowableWriter nodeAddress;
-
-  // TODO: we could be smarter here, and prune periodically
-  // as we go; high in-count nodes will "usually" become
-  // clear early on:
-  private GrowableWriter inCounts;
-
   private final int version;
 
   // make a new empty FST, for building; Builder invokes
   // this ctor
-  FST(INPUT_TYPE inputType, Outputs<T> outputs, boolean willPackFST, float acceptableOverheadRatio, int bytesPageBits) {
+  FST(INPUT_TYPE inputType, Outputs<T> outputs, int bytesPageBits) {
     this.inputType = inputType;
     this.outputs = outputs;
     version = VERSION_CURRENT;
@@ -293,17 +269,8 @@ public final class FST<T> implements Accountable {
     // pad: ensure no node gets address 0 which is reserved to mean
     // the stop state w/ no arcs
     bytes.writeByte((byte) 0);
-    if (willPackFST) {
-      nodeAddress = new GrowableWriter(15, 8, acceptableOverheadRatio);
-      inCounts = new GrowableWriter(1, 8, acceptableOverheadRatio);
-    } else {
-      nodeAddress = null;
-      inCounts = null;
-    }
 
     emptyOutput = null;
-    packed = false;
-    nodeRefToAddress = null;
   }
 
   public static final int DEFAULT_MAX_BLOCK_BITS = Constants.JRE_IS_64BIT ? 30 : 28;
@@ -324,8 +291,12 @@ public final class FST<T> implements Accountable {
 
     // NOTE: only reads most recent format; we don't have
     // back-compat promise for FSTs (they are experimental):
-    version = CodecUtil.checkHeader(in, FILE_FORMAT_NAME, VERSION_PACKED, VERSION_NO_NODE_ARC_COUNTS);
-    packed = in.readByte() == 1;
+    version = CodecUtil.checkHeader(in, FILE_FORMAT_NAME, VERSION_PACKED, VERSION_CURRENT);
+    if (version < VERSION_PACKED_REMOVED) {
+      if (in.readByte() == 1) {
+        throw new CorruptIndexException("Cannot read packed FSTs anymore", in);
+      }
+    }
     if (in.readByte() == 1) {
       // accepts empty string
       // 1 KB blocks:
@@ -334,17 +305,12 @@ public final class FST<T> implements Accountable {
       emptyBytes.copyBytes(in, numBytes);
 
       // De-serialize empty-string output:
-      BytesReader reader;
-      if (packed) {
-        reader = emptyBytes.getForwardReader();
-      } else {
-        reader = emptyBytes.getReverseReader();
-        // NoOutputs uses 0 bytes when writing its output,
-        // so we have to check here else BytesStore gets
-        // angry:
-        if (numBytes > 0) {
-          reader.setPosition(numBytes-1);
-        }
+      BytesReader reader = emptyBytes.getReverseReader();
+      // NoOutputs uses 0 bytes when writing its output,
+      // so we have to check here else BytesStore gets
+      // angry:
+      if (numBytes > 0) {
+        reader.setPosition(numBytes-1);
       }
       emptyOutput = outputs.readFinalOutput(reader);
     } else {
@@ -364,11 +330,6 @@ public final class FST<T> implements Accountable {
     default:
       throw new IllegalStateException("invalid input type " + t);
     }
-    if (packed) {
-      nodeRefToAddress = PackedInts.getReader(in);
-    } else {
-      nodeRefToAddress = null;
-    }
     startNode = in.readVLong();
     if (version < VERSION_NO_NODE_ARC_COUNTS) {
       in.readVLong();
@@ -424,31 +385,13 @@ public final class FST<T> implements Accountable {
     } else {
       size += bytes.ramBytesUsed();
     }
-    if (packed) {
-      size += nodeRefToAddress.ramBytesUsed();
-    } else if (nodeAddress != null) {
-      size += nodeAddress.ramBytesUsed();
-      size += inCounts.ramBytesUsed();
-    }
     size += cachedArcsBytesUsed;
     return size;
   }
 
   @Override
-  public Collection<Accountable> getChildResources() {
-    List<Accountable> resources = new ArrayList<>();
-    if (packed) {
-      resources.add(Accountables.namedAccountable("node ref to address", nodeRefToAddress));
-    } else if (nodeAddress != null) {
-      resources.add(Accountables.namedAccountable("node addresses", nodeAddress));
-      resources.add(Accountables.namedAccountable("in counts", inCounts));
-    }
-    return resources;
-  }
-
-  @Override
   public String toString() {
-    return getClass().getSimpleName() + "(input=" + inputType + ",output=" + outputs + ",packed=" + packed;
+    return getClass().getSimpleName() + "(input=" + inputType + ",output=" + outputs;
   }
 
   void finish(long newStartNode) throws IOException {
@@ -463,16 +406,6 @@ public final class FST<T> implements Accountable {
     bytes.finish();
     cacheRootArcs();
   }
-
-  private long getNodeAddress(long node) {
-    if (nodeAddress != null) {
-      // Deref
-      return nodeAddress.get((int) node);
-    } else {
-      // Straight
-      return node;
-    }
-  }
   
   // Optionally caches first 128 labels
   @SuppressWarnings({"rawtypes","unchecked"})
@@ -527,18 +460,7 @@ public final class FST<T> implements Accountable {
     if (startNode == -1) {
       throw new IllegalStateException("call finish first");
     }
-    if (nodeAddress != null) {
-      throw new IllegalStateException("cannot save an FST pre-packed FST; it must first be packed");
-    }
-    if (packed && !(nodeRefToAddress instanceof PackedInts.Mutable)) {
-      throw new IllegalStateException("cannot save a FST which has been loaded from disk ");
-    }
     CodecUtil.writeHeader(out, FILE_FORMAT_NAME, VERSION_CURRENT);
-    if (packed) {
-      out.writeByte((byte) 1);
-    } else {
-      out.writeByte((byte) 0);
-    }
     // TODO: really we should encode this as an arc, arriving
     // to the root node, instead of special casing here:
     if (emptyOutput != null) {
@@ -552,16 +474,14 @@ public final class FST<T> implements Accountable {
       byte[] emptyOutputBytes = new byte[(int) ros.getFilePointer()];
       ros.writeTo(emptyOutputBytes, 0);
 
-      if (!packed) {
-        // reverse
-        final int stopAt = emptyOutputBytes.length/2;
-        int upto = 0;
-        while(upto < stopAt) {
-          final byte b = emptyOutputBytes[upto];
-          emptyOutputBytes[upto] = emptyOutputBytes[emptyOutputBytes.length-upto-1];
-          emptyOutputBytes[emptyOutputBytes.length-upto-1] = b;
-          upto++;
-        }
+      // reverse
+      final int stopAt = emptyOutputBytes.length/2;
+      int upto = 0;
+      while(upto < stopAt) {
+        final byte b = emptyOutputBytes[upto];
+        emptyOutputBytes[upto] = emptyOutputBytes[emptyOutputBytes.length-upto-1];
+        emptyOutputBytes[emptyOutputBytes.length-upto-1] = b;
+        upto++;
       }
       out.writeVInt(emptyOutputBytes.length);
       out.writeBytes(emptyOutputBytes, 0, emptyOutputBytes.length);
@@ -577,9 +497,6 @@ public final class FST<T> implements Accountable {
       t = 2;
     }
     out.writeByte(t);
-    if (packed) {
-      ((PackedInts.Mutable) nodeRefToAddress).save(out);
-    }
     out.writeVLong(startNode);
     if (bytes != null) {
       long numBytes = bytes.getPosition();
@@ -705,8 +622,6 @@ public final class FST<T> implements Accountable {
 
       if (!targetHasArcs) {
         flags += BIT_STOP_NODE;
-      } else if (inCounts != null) {
-        inCounts.set((int) target.node, inCounts.get((int) target.node) + 1);
       }
 
       if (arc.output != NO_OUTPUT) {
@@ -810,30 +725,8 @@ public final class FST<T> implements Accountable {
 
     builder.bytes.reverse(startAddress, thisNodeAddress);
 
-    // PackedInts uses int as the index, so we cannot handle
-    // > 2.1B nodes when packing:
-    if (nodeAddress != null && builder.nodeCount == Integer.MAX_VALUE) {
-      throw new IllegalStateException("cannot create a packed FST with more than 2.1 billion nodes");
-    }
-
     builder.nodeCount++;
-    final long node;
-    if (nodeAddress != null) {
-
-      // Nodes are addressed by 1+ord:
-      if ((int) builder.nodeCount == nodeAddress.size()) {
-        nodeAddress = nodeAddress.resize(ArrayUtil.oversize(nodeAddress.size() + 1, nodeAddress.getBitsPerValue()));
-        inCounts = inCounts.resize(ArrayUtil.oversize(inCounts.size() + 1, inCounts.getBitsPerValue()));
-      }
-      nodeAddress.set((int) builder.nodeCount, thisNodeAddress);
-      // System.out.println("  write nodeAddress[" + nodeCount + "] = " + endAddress);
-      node = builder.nodeCount;
-    } else {
-      node = thisNodeAddress;
-    }
-
-    //System.out.println("  ret node=" + node + " address=" + thisNodeAddress + " nodeAddress=" + nodeAddress);
-    return node;
+    return thisNodeAddress;
   }
 
   /** Fills virtual 'start' arc, ie, an empty incoming arc to
@@ -876,13 +769,13 @@ public final class FST<T> implements Accountable {
       arc.flags = BIT_LAST_ARC;
       return arc;
     } else {
-      in.setPosition(getNodeAddress(follow.target));
+      in.setPosition(follow.target);
       arc.node = follow.target;
       final byte b = in.readByte();
       if (b == ARCS_AS_FIXED_ARRAY) {
         // array: jump straight to end
         arc.numArcs = in.readVInt();
-        if (packed || version >= VERSION_VINT_TARGET) {
+        if (version >= VERSION_VINT_TARGET) {
           arc.bytesPerArc = in.readVInt();
         } else {
           arc.bytesPerArc = in.readInt();
@@ -906,8 +799,6 @@ public final class FST<T> implements Accountable {
           }
           if (arc.flag(BIT_STOP_NODE)) {
           } else if (arc.flag(BIT_TARGET_NEXT)) {
-          } else if (packed) {
-            in.readVLong();
           } else {
             readUnpackedNodeTarget(in);
           }
@@ -964,7 +855,7 @@ public final class FST<T> implements Accountable {
   }
 
   public Arc<T> readFirstRealTargetArc(long node, Arc<T> arc, final BytesReader in) throws IOException {
-    final long address = getNodeAddress(node);
+    final long address = node;
     in.setPosition(address);
     //System.out.println("  readFirstRealTargtArc address="
     //+ address);
@@ -975,7 +866,7 @@ public final class FST<T> implements Accountable {
       //System.out.println("  fixedArray");
       // this is first arc in a fixed-array
       arc.numArcs = in.readVInt();
-      if (packed || version >= VERSION_VINT_TARGET) {
+      if (version >= VERSION_VINT_TARGET) {
         arc.bytesPerArc = in.readVInt();
       } else {
         arc.bytesPerArc = in.readInt();
@@ -1002,7 +893,7 @@ public final class FST<T> implements Accountable {
     if (!targetHasArcs(follow)) {
       return false;
     } else {
-      in.setPosition(getNodeAddress(follow.target));
+      in.setPosition(follow.target);
       return in.readByte() == ARCS_AS_FIXED_ARRAY;
     }
   }
@@ -1029,7 +920,7 @@ public final class FST<T> implements Accountable {
       //System.out.println("    nextArc fake " +
       //arc.nextArc);
       
-      long pos = getNodeAddress(arc.nextArc);
+      long pos = arc.nextArc;
       in.setPosition(pos);
 
       final byte b = in.readByte();
@@ -1038,7 +929,7 @@ public final class FST<T> implements Accountable {
         in.readVInt();
 
         // Skip bytesPerArc:
-        if (packed || version >= VERSION_VINT_TARGET) {
+        if (version >= VERSION_VINT_TARGET) {
           in.readVInt();
         } else {
           in.readInt();
@@ -1107,41 +998,18 @@ public final class FST<T> implements Accountable {
       arc.nextArc = in.getPosition();
       // TODO: would be nice to make this lazy -- maybe
       // caller doesn't need the target and is scanning arcs...
-      if (nodeAddress == null) {
-        if (!arc.flag(BIT_LAST_ARC)) {
-          if (arc.bytesPerArc == 0) {
-            // must scan
-            seekToNextNode(in);
-          } else {
-            in.setPosition(arc.posArcsStart);
-            in.skipBytes(arc.bytesPerArc * arc.numArcs);
-          }
-        }
-        arc.target = in.getPosition();
-      } else {
-        arc.target = arc.node - 1;
-        assert arc.target > 0;
-      }
-    } else {
-      if (packed) {
-        final long pos = in.getPosition();
-        final long code = in.readVLong();
-        if (arc.flag(BIT_TARGET_DELTA)) {
-          // Address is delta-coded from current address:
-          arc.target = pos + code;
-          //System.out.println("    delta pos=" + pos + " delta=" + code + " target=" + arc.target);
-        } else if (code < nodeRefToAddress.size()) {
-          // Deref
-          arc.target = nodeRefToAddress.get((int) code);
-          //System.out.println("    deref code=" + code + " target=" + arc.target);
+      if (!arc.flag(BIT_LAST_ARC)) {
+        if (arc.bytesPerArc == 0) {
+          // must scan
+          seekToNextNode(in);
         } else {
-          // Absolute
-          arc.target = code;
-          //System.out.println("    abs code=" + code);
+          in.setPosition(arc.posArcsStart);
+          in.skipBytes(arc.bytesPerArc * arc.numArcs);
         }
-      } else {
-        arc.target = readUnpackedNodeTarget(in);
       }
+      arc.target = in.getPosition();
+    } else {
+      arc.target = readUnpackedNodeTarget(in);
       arc.nextArc = in.getPosition();
     }
     return arc;
@@ -1228,7 +1096,7 @@ public final class FST<T> implements Accountable {
       return null;
     }
 
-    in.setPosition(getNodeAddress(follow.target));
+    in.setPosition(follow.target);
 
     arc.node = follow.target;
 
@@ -1237,7 +1105,7 @@ public final class FST<T> implements Accountable {
     if (in.readByte() == ARCS_AS_FIXED_ARRAY) {
       // Arcs are full array; do binary search:
       arc.numArcs = in.readVInt();
-      if (packed || version >= VERSION_VINT_TARGET) {
+      if (version >= VERSION_VINT_TARGET) {
         arc.bytesPerArc = in.readVInt();
       } else {
         arc.bytesPerArc = in.readInt();
@@ -1303,11 +1171,7 @@ public final class FST<T> implements Accountable {
       }
 
       if (!flag(flags, BIT_STOP_NODE) && !flag(flags, BIT_TARGET_NEXT)) {
-        if (packed) {
-          in.readVLong();
-        } else {
-          readUnpackedNodeTarget(in);
-        }
+        readUnpackedNodeTarget(in);
       }
 
       if (flag(flags, BIT_LAST_ARC)) {
@@ -1340,18 +1204,10 @@ public final class FST<T> implements Accountable {
   /** Returns a {@link BytesReader} for this FST, positioned at
    *  position 0. */
   public BytesReader getBytesReader() {
-    if (packed) {
-      if (bytesArray != null) {
-        return new ForwardBytesReader(bytesArray);
-      } else {
-        return bytes.getForwardReader();
-      }
+    if (bytesArray != null) {
+      return new ReverseBytesReader(bytesArray);
     } else {
-      if (bytesArray != null) {
-        return new ReverseBytesReader(bytesArray);
-      } else {
-        return bytes.getReverseReader();
-      }
+      return bytes.getReverseReader();
     }
   }
 
@@ -1476,395 +1332,4 @@ public final class FST<T> implements Accountable {
   }
  */
 
-  // Creates a packed FST
-  private FST(INPUT_TYPE inputType, Outputs<T> outputs, int bytesPageBits) {
-    version = VERSION_CURRENT;
-    packed = true;
-    this.inputType = inputType;
-    bytesArray = null;
-    bytes = new BytesStore(bytesPageBits);
-    this.outputs = outputs;
-  }
-
-  /** Expert: creates an FST by packing this one.  This
-   *  process requires substantial additional RAM (currently
-   *  up to ~8 bytes per node depending on
-   *  <code>acceptableOverheadRatio</code>), but then should
-   *  produce a smaller FST.
-   *
-   *  <p>The implementation of this method uses ideas from
-   *  <a target="_blank" href="http://www.cs.put.poznan.pl/dweiss/site/publications/download/fsacomp.pdf">Smaller Representation of Finite State Automata</a>,
-   *  which describes techniques to reduce the size of a FST.
-   *  However, this is not a strict implementation of the
-   *  algorithms described in this paper.
-   */
-  FST<T> pack(Builder<T> builder, int minInCountDeref, int maxDerefNodes, float acceptableOverheadRatio) throws IOException {
-
-    // NOTE: maxDerefNodes is intentionally int: we cannot
-    // support > 2.1B deref nodes
-
-    // TODO: other things to try
-    //   - renumber the nodes to get more next / better locality?
-    //   - allow multiple input labels on an arc, so
-    //     singular chain of inputs can take one arc (on
-    //     wikipedia terms this could save another ~6%)
-    //   - in the ord case, the output '1' is presumably
-    //     very common (after NO_OUTPUT)... maybe use a bit
-    //     for it..?
-    //   - use spare bits in flags.... for top few labels /
-    //     outputs / targets
-
-    if (nodeAddress == null) {
-      throw new IllegalArgumentException("this FST was not built with willPackFST=true");
-    }
-
-    T NO_OUTPUT = outputs.getNoOutput();
-
-    Arc<T> arc = new Arc<>();
-
-    final BytesReader r = getBytesReader();
-
-    final int topN = Math.min(maxDerefNodes, inCounts.size());
-
-    // Find top nodes with highest number of incoming arcs:
-    NodeQueue q = new NodeQueue(topN);
-
-    // TODO: we could use more RAM efficient selection algo here...
-    NodeAndInCount bottom = null;
-    for(int node=0; node<inCounts.size(); node++) {
-      if (inCounts.get(node) >= minInCountDeref) {
-        if (bottom == null) {
-          q.add(new NodeAndInCount(node, (int) inCounts.get(node)));
-          if (q.size() == topN) {
-            bottom = q.top();
-          }
-        } else if (inCounts.get(node) > bottom.count) {
-          q.insertWithOverflow(new NodeAndInCount(node, (int) inCounts.get(node)));
-        }
-      }
-    }
-
-    // Free up RAM:
-    inCounts = null;
-
-    final Map<Integer,Integer> topNodeMap = new HashMap<>();
-    for(int downTo=q.size()-1;downTo>=0;downTo--) {
-      NodeAndInCount n = q.pop();
-      topNodeMap.put(n.node, downTo);
-      //System.out.println("map node=" + n.node + " inCount=" + n.count + " to newID=" + downTo);
-    }
-
-    // +1 because node ords start at 1 (0 is reserved as stop node):
-    final GrowableWriter newNodeAddress = new GrowableWriter(
-                                                             PackedInts.bitsRequired(builder.bytes.getPosition()), (int) (1 + builder.nodeCount), acceptableOverheadRatio);
-
-    // Fill initial coarse guess:
-    for(int node=1;node<=builder.nodeCount;node++) {
-      newNodeAddress.set(node, 1 + builder.bytes.getPosition() - nodeAddress.get(node));
-    }
-
-    int absCount;
-    int deltaCount;
-    int topCount;
-    int nextCount;
-
-    FST<T> fst;
-
-    // Iterate until we converge:
-    while(true) {
-
-      //System.out.println("\nITER");
-      boolean changed = false;
-
-      // for assert:
-      boolean negDelta = false;
-
-      fst = new FST<>(inputType, outputs, builder.bytes.getBlockBits());
-      
-      final BytesStore writer = fst.bytes;
-
-      // Skip 0 byte since 0 is reserved target:
-      writer.writeByte((byte) 0);
-
-      absCount = deltaCount = topCount = nextCount = 0;
-
-      int changedCount = 0;
-
-      long addressError = 0;
-
-      //int totWasted = 0;
-
-      // Since we re-reverse the bytes, we now write the
-      // nodes backwards, so that BIT_TARGET_NEXT is
-      // unchanged:
-      for(int node=(int) builder.nodeCount;node>=1;node--) {
-        final long address = writer.getPosition();
-
-        //System.out.println("  node: " + node + " address=" + address);
-        if (address != newNodeAddress.get(node)) {
-          addressError = address - newNodeAddress.get(node);
-          //System.out.println("    change: " + (address - newNodeAddress[node]));
-          changed = true;
-          newNodeAddress.set(node, address);
-          changedCount++;
-        }
-
-        int nodeArcCount = 0;
-        int bytesPerArc = 0;
-
-        boolean retry = false;
-
-        // for assert:
-        boolean anyNegDelta = false;
-
-        // Retry loop: possibly iterate more than once, if
-        // this is an array'd node and bytesPerArc changes:
-        writeNode:
-        while(true) { // retry writing this node
-
-          //System.out.println("  cycle: retry");
-          readFirstRealTargetArc(node, arc, r);
-
-          final boolean useArcArray = arc.bytesPerArc != 0;
-          if (useArcArray) {
-            // Write false first arc:
-            if (bytesPerArc == 0) {
-              bytesPerArc = arc.bytesPerArc;
-            }
-            writer.writeByte(ARCS_AS_FIXED_ARRAY);
-            writer.writeVInt(arc.numArcs);
-            writer.writeVInt(bytesPerArc);
-            //System.out.println("node " + node + ": " + arc.numArcs + " arcs");
-          }
-
-          int maxBytesPerArc = 0;
-          //int wasted = 0;
-          while(true) {  // iterate over all arcs for this node
-            //System.out.println("    cycle next arc");
-
-            final long arcStartPos = writer.getPosition();
-            nodeArcCount++;
-
-            byte flags = 0;
-
-            if (arc.isLast()) {
-              flags += BIT_LAST_ARC;
-            }
-            /*
-            if (!useArcArray && nodeUpto < nodes.length-1 && arc.target == nodes[nodeUpto+1]) {
-              flags += BIT_TARGET_NEXT;
-            }
-            */
-            if (!useArcArray && node != 1 && arc.target == node-1) {
-              flags += BIT_TARGET_NEXT;
-              if (!retry) {
-                nextCount++;
-              }
-            }
-            if (arc.isFinal()) {
-              flags += BIT_FINAL_ARC;
-              if (arc.nextFinalOutput != NO_OUTPUT) {
-                flags += BIT_ARC_HAS_FINAL_OUTPUT;
-              }
-            } else {
-              assert arc.nextFinalOutput == NO_OUTPUT;
-            }
-            if (!targetHasArcs(arc)) {
-              flags += BIT_STOP_NODE;
-            }
-
-            if (arc.output != NO_OUTPUT) {
-              flags += BIT_ARC_HAS_OUTPUT;
-            }
-
-            final long absPtr;
-            final boolean doWriteTarget = targetHasArcs(arc) && (flags & BIT_TARGET_NEXT) == 0;
-            if (doWriteTarget) {
-
-              final Integer ptr = topNodeMap.get(arc.target);
-              if (ptr != null) {
-                absPtr = ptr;
-              } else {
-                absPtr = topNodeMap.size() + newNodeAddress.get((int) arc.target) + addressError;
-              }
-
-              long delta = newNodeAddress.get((int) arc.target) + addressError - writer.getPosition() - 2;
-              if (delta < 0) {
-                //System.out.println("neg: " + delta);
-                anyNegDelta = true;
-                delta = 0;
-              }
-
-              if (delta < absPtr) {
-                flags |= BIT_TARGET_DELTA;
-              }
-            } else {
-              absPtr = 0;
-            }
-
-            assert flags != ARCS_AS_FIXED_ARRAY;
-            writer.writeByte(flags);
-
-            fst.writeLabel(writer, arc.label);
-
-            if (arc.output != NO_OUTPUT) {
-              outputs.write(arc.output, writer);
-            }
-            if (arc.nextFinalOutput != NO_OUTPUT) {
-              outputs.writeFinalOutput(arc.nextFinalOutput, writer);
-            }
-
-            if (doWriteTarget) {
-
-              long delta = newNodeAddress.get((int) arc.target) + addressError - writer.getPosition();
-              if (delta < 0) {
-                anyNegDelta = true;
-                //System.out.println("neg: " + delta);
-                delta = 0;
-              }
-
-              if (flag(flags, BIT_TARGET_DELTA)) {
-                //System.out.println("        delta");
-                writer.writeVLong(delta);
-                if (!retry) {
-                  deltaCount++;
-                }
-              } else {
-                /*
-                if (ptr != null) {
-                  System.out.println("        deref");
-                } else {
-                  System.out.println("        abs");
-                }
-                */
-                writer.writeVLong(absPtr);
-                if (!retry) {
-                  if (absPtr >= topNodeMap.size()) {
-                    absCount++;
-                  } else {
-                    topCount++;
-                  }
-                }
-              }
-            }
-
-            if (useArcArray) {
-              final int arcBytes = (int) (writer.getPosition() - arcStartPos);
-              //System.out.println("  " + arcBytes + " bytes");
-              maxBytesPerArc = Math.max(maxBytesPerArc, arcBytes);
-              // NOTE: this may in fact go "backwards", if
-              // somehow (rarely, possibly never) we use
-              // more bytesPerArc in this rewrite than the
-              // incoming FST did... but in this case we
-              // will retry (below) so it's OK to ovewrite
-              // bytes:
-              //wasted += bytesPerArc - arcBytes;
-              writer.skipBytes((int) (arcStartPos + bytesPerArc - writer.getPosition()));
-            }
-
-            if (arc.isLast()) {
-              break;
-            }
-
-            readNextRealArc(arc, r);
-          }
-
-          if (useArcArray) {
-            if (maxBytesPerArc == bytesPerArc || (retry && maxBytesPerArc <= bytesPerArc)) {
-              // converged
-              //System.out.println("  bba=" + bytesPerArc + " wasted=" + wasted);
-              //totWasted += wasted;
-              break;
-            }
-          } else {
-            break;
-          }
-
-          //System.out.println("  retry this node maxBytesPerArc=" + maxBytesPerArc + " vs " + bytesPerArc);
-
-          // Retry:
-          bytesPerArc = maxBytesPerArc;
-          writer.truncate(address);
-          nodeArcCount = 0;
-          retry = true;
-          anyNegDelta = false;
-        }
-
-        negDelta |= anyNegDelta;
-      }
-
-      if (!changed) {
-        // We don't renumber the nodes (just reverse their
-        // order) so nodes should only point forward to
-        // other nodes because we only produce acyclic FSTs
-        // w/ nodes only pointing "forwards":
-        assert !negDelta;
-        //System.out.println("TOT wasted=" + totWasted);
-        // Converged!
-        break;
-      }
-    }
-
-    long maxAddress = 0;
-    for (long key : topNodeMap.keySet()) {
-      maxAddress = Math.max(maxAddress, newNodeAddress.get((int) key));
-    }
-
-    PackedInts.Mutable nodeRefToAddressIn = PackedInts.getMutable(topNodeMap.size(),
-        PackedInts.bitsRequired(maxAddress), acceptableOverheadRatio);
-    for(Map.Entry<Integer,Integer> ent : topNodeMap.entrySet()) {
-      nodeRefToAddressIn.set(ent.getValue(), newNodeAddress.get(ent.getKey()));
-    }
-    fst.nodeRefToAddress = nodeRefToAddressIn;
-    
-    fst.startNode = newNodeAddress.get((int) startNode);
-    //System.out.println("new startNode=" + fst.startNode + " old startNode=" + startNode);
-
-    if (emptyOutput != null) {
-      fst.setEmptyOutput(emptyOutput);
-    }
-
-    fst.bytes.finish();
-    fst.cacheRootArcs();
-
-    //final int size = fst.sizeInBytes();
-    //System.out.println("nextCount=" + nextCount + " topCount=" + topCount + " deltaCount=" + deltaCount + " absCount=" + absCount);
-
-    return fst;
-  }
-
-  private static class NodeAndInCount implements Comparable<NodeAndInCount> {
-    final int node;
-    final int count;
-
-    public NodeAndInCount(int node, int count) {
-      this.node = node;
-      this.count = count;
-    }
-    
-    @Override
-    public int compareTo(NodeAndInCount other) {
-      if (count > other.count) {
-        return 1;
-      } else if (count < other.count) {
-        return -1;
-      } else {
-        // Tie-break: smaller node compares as greater than
-        return other.node - node;
-      }
-    }
-  }
-
-  private static class NodeQueue extends PriorityQueue<NodeAndInCount> {
-    public NodeQueue(int topN) {
-      super(topN, false);
-    }
-
-    @Override
-    public boolean lessThan(NodeAndInCount a, NodeAndInCount b) {
-      final int cmp = a.compareTo(b);
-      assert cmp != 0;
-      return cmp < 0;
-    }
-  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c4c5e868/lucene/core/src/java/org/apache/lucene/util/fst/package-info.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/util/fst/package-info.java b/lucene/core/src/java/org/apache/lucene/util/fst/package-info.java
index 41426f9..d984586 100644
--- a/lucene/core/src/java/org/apache/lucene/util/fst/package-info.java
+++ b/lucene/core/src/java/org/apache/lucene/util/fst/package-info.java
@@ -24,7 +24,6 @@
  *    <li>Fast and low memory overhead construction of the minimal FST 
  *        (but inputs must be provided in sorted order)</li>
  *    <li>Low object overhead and quick deserialization (byte[] representation)</li>
- *    <li>Optional two-pass compression: {@link org.apache.lucene.util.fst.FST#pack FST.pack()}</li>
  *    <li>{@link org.apache.lucene.util.fst.Util#getByOutput Lookup-by-output} when the 
  *        outputs are in sorted order (e.g., ordinals or file pointers)</li>
  *    <li>Pluggable {@link org.apache.lucene.util.fst.Outputs Outputs} representation</li>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c4c5e868/lucene/core/src/test/org/apache/lucene/util/fst/Test2BFST.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/util/fst/Test2BFST.java b/lucene/core/src/test/org/apache/lucene/util/fst/Test2BFST.java
index bdec65c..a02bf8a 100644
--- a/lucene/core/src/test/org/apache/lucene/util/fst/Test2BFST.java
+++ b/lucene/core/src/test/org/apache/lucene/util/fst/Test2BFST.java
@@ -29,7 +29,6 @@ import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.IntsRef;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.TimeUnits;
-import org.apache.lucene.util.packed.PackedInts;
 import org.junit.Ignore;
 
 import com.carrotsearch.randomizedtesting.annotations.TimeoutSuite;
@@ -47,16 +46,14 @@ public class Test2BFST extends LuceneTestCase {
 
     Directory dir = new MMapDirectory(createTempDir("2BFST"));
 
-    for(int doPackIter=0;doPackIter<2;doPackIter++) {
-      boolean doPack = doPackIter == 1;
-
+    for(int iter=0;iter<1;iter++) {
       // Build FST w/ NoOutputs and stop when nodeCount > 2.2B
-      if (!doPack) {
+      {
         System.out.println("\nTEST: 3B nodes; doPack=false output=NO_OUTPUTS");
         Outputs<Object> outputs = NoOutputs.getSingleton();
         Object NO_OUTPUT = outputs.getNoOutput();
         final Builder<Object> b = new Builder<>(FST.INPUT_TYPE.BYTE1, 0, 0, true, true, Integer.MAX_VALUE, outputs,
-                                                doPack, PackedInts.COMPACT, true, 15);
+                                                true, 15);
 
         int count = 0;
         Random r = new Random(seed);
@@ -135,10 +132,10 @@ public class Test2BFST extends LuceneTestCase {
       // Build FST w/ ByteSequenceOutputs and stop when FST
       // size = 3GB
       {
-        System.out.println("\nTEST: 3 GB size; doPack=" + doPack + " outputs=bytes");
+        System.out.println("\nTEST: 3 GB size; outputs=bytes");
         Outputs<BytesRef> outputs = ByteSequenceOutputs.getSingleton();
         final Builder<BytesRef> b = new Builder<>(FST.INPUT_TYPE.BYTE1, 0, 0, true, true, Integer.MAX_VALUE, outputs,
-                                                  doPack, PackedInts.COMPACT, true, 15);
+                                                  true, 15);
 
         byte[] outputBytes = new byte[20];
         BytesRef output = new BytesRef(outputBytes);
@@ -212,10 +209,10 @@ public class Test2BFST extends LuceneTestCase {
       // Build FST w/ PositiveIntOutputs and stop when FST
       // size = 3GB
       {
-        System.out.println("\nTEST: 3 GB size; doPack=" + doPack + " outputs=long");
+        System.out.println("\nTEST: 3 GB size; outputs=long");
         Outputs<Long> outputs = PositiveIntOutputs.getSingleton();
         final Builder<Long> b = new Builder<>(FST.INPUT_TYPE.BYTE1, 0, 0, true, true, Integer.MAX_VALUE, outputs,
-                                              doPack, PackedInts.COMPACT, true, 15);
+                                              true, 15);
 
         long output = 1;
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c4c5e868/lucene/core/src/test/org/apache/lucene/util/fst/TestFSTs.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/util/fst/TestFSTs.java b/lucene/core/src/test/org/apache/lucene/util/fst/TestFSTs.java
index 39b3282..6b218cf 100644
--- a/lucene/core/src/test/org/apache/lucene/util/fst/TestFSTs.java
+++ b/lucene/core/src/test/org/apache/lucene/util/fst/TestFSTs.java
@@ -76,7 +76,6 @@ import org.apache.lucene.util.fst.FST.Arc;
 import org.apache.lucene.util.fst.FST.BytesReader;
 import org.apache.lucene.util.fst.PairOutputs.Pair;
 import org.apache.lucene.util.fst.Util.Result;
-import org.apache.lucene.util.packed.PackedInts;
 
 import static org.apache.lucene.util.fst.FSTTester.getRandomString;
 import static org.apache.lucene.util.fst.FSTTester.simpleRandomString;
@@ -328,9 +327,7 @@ public class TestFSTs extends LuceneTestCase {
     writer.close();
     final PositiveIntOutputs outputs = PositiveIntOutputs.getSingleton();
 
-    final boolean doRewrite = random().nextBoolean();
-
-    Builder<Long> builder = new Builder<>(FST.INPUT_TYPE.BYTE1, 0, 0, true, true, Integer.MAX_VALUE, outputs, doRewrite, PackedInts.DEFAULT, true, 15);
+    Builder<Long> builder = new Builder<>(FST.INPUT_TYPE.BYTE1, 0, 0, true, true, Integer.MAX_VALUE, outputs, true, 15);
 
     boolean storeOrd = random().nextBoolean();
     if (VERBOSE) {
@@ -464,16 +461,14 @@ public class TestFSTs extends LuceneTestCase {
     private int inputMode;
     private final Outputs<T> outputs;
     private final Builder<T> builder;
-    private final boolean doPack;
 
-    public VisitTerms(Path dirOut, Path wordsFileIn, int inputMode, int prune, Outputs<T> outputs, boolean doPack, boolean noArcArrays) {
+    public VisitTerms(Path dirOut, Path wordsFileIn, int inputMode, int prune, Outputs<T> outputs, boolean noArcArrays) {
       this.dirOut = dirOut;
       this.wordsFileIn = wordsFileIn;
       this.inputMode = inputMode;
       this.outputs = outputs;
-      this.doPack = doPack;
 
-      builder = new Builder<>(inputMode == 0 ? FST.INPUT_TYPE.BYTE1 : FST.INPUT_TYPE.BYTE4, 0, prune, prune == 0, true, Integer.MAX_VALUE, outputs, doPack, PackedInts.DEFAULT, !noArcArrays, 15);
+      builder = new Builder<>(inputMode == 0 ? FST.INPUT_TYPE.BYTE1 : FST.INPUT_TYPE.BYTE4, 0, prune, prune == 0, true, Integer.MAX_VALUE, outputs, !noArcArrays, 15);
     }
 
     protected abstract T getOutput(IntsRef input, int ord) throws IOException;
@@ -622,7 +617,6 @@ public class TestFSTs extends LuceneTestCase {
     boolean storeOrds = false;
     boolean storeDocFreqs = false;
     boolean verify = true;
-    boolean doPack = false;
     boolean noArcArrays = false;
     Path wordsFileIn = null;
     Path dirOut = null;
@@ -647,8 +641,6 @@ public class TestFSTs extends LuceneTestCase {
         storeOrds = true;
       } else if (args[idx].equals("-noverify")) {
         verify = false;
-      } else if (args[idx].equals("-pack")) {
-        doPack = true;
       } else if (args[idx].startsWith("-")) {
         System.err.println("Unrecognized option: " + args[idx]);
         System.exit(-1);
@@ -677,7 +669,7 @@ public class TestFSTs extends LuceneTestCase {
       final PositiveIntOutputs o1 = PositiveIntOutputs.getSingleton();
       final PositiveIntOutputs o2 = PositiveIntOutputs.getSingleton();
       final PairOutputs<Long,Long> outputs = new PairOutputs<>(o1, o2);
-      new VisitTerms<PairOutputs.Pair<Long,Long>>(dirOut, wordsFileIn, inputMode, prune, outputs, doPack, noArcArrays) {
+      new VisitTerms<PairOutputs.Pair<Long,Long>>(dirOut, wordsFileIn, inputMode, prune, outputs, noArcArrays) {
         Random rand;
         @Override
         public PairOutputs.Pair<Long,Long> getOutput(IntsRef input, int ord) {
@@ -691,7 +683,7 @@ public class TestFSTs extends LuceneTestCase {
     } else if (storeOrds) {
       // Store only ords
       final PositiveIntOutputs outputs = PositiveIntOutputs.getSingleton();
-      new VisitTerms<Long>(dirOut, wordsFileIn, inputMode, prune, outputs, doPack, noArcArrays) {
+      new VisitTerms<Long>(dirOut, wordsFileIn, inputMode, prune, outputs, noArcArrays) {
         @Override
         public Long getOutput(IntsRef input, int ord) {
           return (long) ord;
@@ -700,7 +692,7 @@ public class TestFSTs extends LuceneTestCase {
     } else if (storeDocFreqs) {
       // Store only docFreq
       final PositiveIntOutputs outputs = PositiveIntOutputs.getSingleton();
-      new VisitTerms<Long>(dirOut, wordsFileIn, inputMode, prune, outputs, doPack, noArcArrays) {
+      new VisitTerms<Long>(dirOut, wordsFileIn, inputMode, prune, outputs, noArcArrays) {
         Random rand;
         @Override
         public Long getOutput(IntsRef input, int ord) {
@@ -714,7 +706,7 @@ public class TestFSTs extends LuceneTestCase {
       // Store nothing
       final NoOutputs outputs = NoOutputs.getSingleton();
       final Object NO_OUTPUT = outputs.getNoOutput();
-      new VisitTerms<Object>(dirOut, wordsFileIn, inputMode, prune, outputs, doPack, noArcArrays) {
+      new VisitTerms<Object>(dirOut, wordsFileIn, inputMode, prune, outputs, noArcArrays) {
         @Override
         public Object getOutput(IntsRef input, int ord) {
           return NO_OUTPUT;
@@ -1118,7 +1110,7 @@ public class TestFSTs extends LuceneTestCase {
   public void testFinalOutputOnEndState() throws Exception {
     final PositiveIntOutputs outputs = PositiveIntOutputs.getSingleton();
 
-    final Builder<Long> builder = new Builder<>(FST.INPUT_TYPE.BYTE4, 2, 0, true, true, Integer.MAX_VALUE, outputs, random().nextBoolean(), PackedInts.DEFAULT, true, 15);
+    final Builder<Long> builder = new Builder<>(FST.INPUT_TYPE.BYTE4, 2, 0, true, true, Integer.MAX_VALUE, outputs, true, 15);
     builder.add(Util.toUTF32("stat", new IntsRefBuilder()), 17L);
     builder.add(Util.toUTF32("station", new IntsRefBuilder()), 10L);
     final FST<Long> fst = builder.finish();
@@ -1132,8 +1124,7 @@ public class TestFSTs extends LuceneTestCase {
 
   public void testInternalFinalState() throws Exception {
     final PositiveIntOutputs outputs = PositiveIntOutputs.getSingleton();
-    final boolean willRewrite = random().nextBoolean();
-    final Builder<Long> builder = new Builder<>(FST.INPUT_TYPE.BYTE1, 0, 0, true, true, Integer.MAX_VALUE, outputs, willRewrite, PackedInts.DEFAULT, true, 15);
+    final Builder<Long> builder = new Builder<>(FST.INPUT_TYPE.BYTE1, 0, 0, true, true, Integer.MAX_VALUE, outputs, true, 15);
     builder.add(Util.toIntsRef(new BytesRef("stat"), new IntsRefBuilder()), outputs.getNoOutput());
     builder.add(Util.toIntsRef(new BytesRef("station"), new IntsRefBuilder()), outputs.getNoOutput());
     final FST<Long> fst = builder.finish();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c4c5e868/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/VersionBlockTreeTermsWriter.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/VersionBlockTreeTermsWriter.java b/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/VersionBlockTreeTermsWriter.java
index b49ea79..d83b915 100644
--- a/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/VersionBlockTreeTermsWriter.java
+++ b/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/VersionBlockTreeTermsWriter.java
@@ -50,7 +50,6 @@ import org.apache.lucene.util.fst.PairOutputs.Pair;
 import org.apache.lucene.util.fst.PairOutputs;
 import org.apache.lucene.util.fst.PositiveIntOutputs;
 import org.apache.lucene.util.fst.Util;
-import org.apache.lucene.util.packed.PackedInts;
 
 /*
   TODO:
@@ -354,8 +353,7 @@ public final class VersionBlockTreeTermsWriter extends FieldsConsumer {
 
       final Builder<Pair<BytesRef,Long>> indexBuilder = new Builder<>(FST.INPUT_TYPE.BYTE1,
                                                                       0, 0, true, false, Integer.MAX_VALUE,
-                                                                      FST_OUTPUTS, false,
-                                                                      PackedInts.COMPACT, true, 15);
+                                                                      FST_OUTPUTS, true, 15);
       //if (DEBUG) {
       //  System.out.println("  compile index for prefix=" + prefix);
       //}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c4c5e868/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/FSTCompletionBuilder.java
----------------------------------------------------------------------
diff --git a/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/FSTCompletionBuilder.java b/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/FSTCompletionBuilder.java
index 3706724..3d20412 100644
--- a/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/FSTCompletionBuilder.java
+++ b/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/FSTCompletionBuilder.java
@@ -26,7 +26,6 @@ import org.apache.lucene.util.BytesRefBuilder;
 import org.apache.lucene.util.BytesRefIterator;
 import org.apache.lucene.util.IntsRefBuilder;
 import org.apache.lucene.util.fst.*;
-import org.apache.lucene.util.packed.PackedInts;
 
 /**
  * Finite state automata based implementation of "autocomplete" functionality.
@@ -237,8 +236,7 @@ public class FSTCompletionBuilder {
     final Object empty = outputs.getNoOutput();
     final Builder<Object> builder = new Builder<>(
         FST.INPUT_TYPE.BYTE1, 0, 0, true, true, 
-        shareMaxTailLength, outputs, false, 
-        PackedInts.DEFAULT, true, 15);
+        shareMaxTailLength, outputs, true, 15);
     
     BytesRefBuilder scratch = new BytesRefBuilder();
     BytesRef entry;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c4c5e868/lucene/test-framework/src/java/org/apache/lucene/util/fst/FSTTester.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/java/org/apache/lucene/util/fst/FSTTester.java b/lucene/test-framework/src/java/org/apache/lucene/util/fst/FSTTester.java
index 11b1325..8e6a4ea 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/util/fst/FSTTester.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/util/fst/FSTTester.java
@@ -40,7 +40,6 @@ import org.apache.lucene.util.IntsRefBuilder;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.TestUtil;
 import org.apache.lucene.util.UnicodeUtil;
-import org.apache.lucene.util.packed.PackedInts;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
@@ -273,25 +272,14 @@ public class FSTTester<T> {
       System.out.println("\nTEST: prune1=" + prune1 + " prune2=" + prune2);
     }
 
-    final boolean willRewrite = random.nextBoolean();
-
     final Builder<T> builder = new Builder<>(inputMode == 0 ? FST.INPUT_TYPE.BYTE1 : FST.INPUT_TYPE.BYTE4,
                                               prune1, prune2,
                                               prune1==0 && prune2==0,
                                               allowRandomSuffixSharing ? random.nextBoolean() : true,
                                               allowRandomSuffixSharing ? TestUtil.nextInt(random, 1, 10) : Integer.MAX_VALUE,
                                               outputs,
-                                              willRewrite,
-                                              PackedInts.DEFAULT,
                                               true,
                                               15);
-    if (LuceneTestCase.VERBOSE) {
-      if (willRewrite) {
-        System.out.println("TEST: packed FST");
-      } else {
-        System.out.println("TEST: non-packed FST");
-      }
-    }
 
     for(InputOutput<T> pair : pairs) {
       if (pair.output instanceof List) {
@@ -306,7 +294,7 @@ public class FSTTester<T> {
     }
     FST<T> fst = builder.finish();
 
-    if (random.nextBoolean() && fst != null && !willRewrite) {
+    if (random.nextBoolean() && fst != null) {
       IOContext context = LuceneTestCase.newIOContext(random);
       IndexOutput out = dir.createOutput("fst.bin", context);
       fst.save(out);


[2/4] lucene-solr:master: LUCENE-5867: Add a BooleanSimilarity.

Posted by jp...@apache.org.
LUCENE-5867: Add a BooleanSimilarity.


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

Branch: refs/heads/master
Commit: 3e15233b23197122e40a851edab7b7257ce63f02
Parents: c415bc8
Author: Adrien Grand <jp...@gmail.com>
Authored: Thu Nov 10 14:05:13 2016 +0100
Committer: Adrien Grand <jp...@gmail.com>
Committed: Thu Nov 10 15:01:49 2016 +0100

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |   5 +-
 .../search/similarities/BooleanSimilarity.java  |  95 +++++++++++++++
 .../similarities/TestBooleanSimilarity.java     | 117 +++++++++++++++++++
 .../search/similarities/RandomSimilarity.java   |   1 +
 .../util/TestRuleSetupAndRestoreClassEnv.java   |   3 +-
 5 files changed, 218 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3e15233b/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 191c103..97e6939 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -55,7 +55,10 @@ Other
 * LUCENE-7360: Remove Explanation.toHtml() (Alan Woodward)
 
 ======================= Lucene 6.4.0 =======================
-(No Changes)
+
+New features
+
+* LUCENE-5867: Added BooleanSimilarity. (Robert Muir, Adrien Grand)
 
 Improvements
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3e15233b/lucene/core/src/java/org/apache/lucene/search/similarities/BooleanSimilarity.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/similarities/BooleanSimilarity.java b/lucene/core/src/java/org/apache/lucene/search/similarities/BooleanSimilarity.java
new file mode 100644
index 0000000..a7b7614
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/search/similarities/BooleanSimilarity.java
@@ -0,0 +1,95 @@
+/*
+ * 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.similarities;
+
+import java.io.IOException;
+
+import org.apache.lucene.index.FieldInvertState;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.search.CollectionStatistics;
+import org.apache.lucene.search.Explanation;
+import org.apache.lucene.search.TermStatistics;
+import org.apache.lucene.util.BytesRef;
+
+/**
+ * Simple similarity that gives terms a score that is equal to their query
+ * boost. This similarity is typically used with disabled norms since neither
+ * document statistics nor index statistics are used for scoring. That said,
+ * if norms are enabled, they will be computed the same way as
+ * {@link SimilarityBase} and {@link BM25Similarity} with
+ * {@link SimilarityBase#setDiscountOverlaps(boolean) discounted overlaps}
+ * so that the {@link Similarity} can be changed after the index has been
+ * created.
+ */
+public class BooleanSimilarity extends Similarity {
+
+  private static final Similarity BM25_SIM = new BM25Similarity();
+
+  /** Sole constructor */
+  public BooleanSimilarity() {}
+
+  @Override
+  public long computeNorm(FieldInvertState state) {
+    return BM25_SIM.computeNorm(state);
+  }
+
+  @Override
+  public SimWeight computeWeight(float boost, CollectionStatistics collectionStats, TermStatistics... termStats) {
+    return new BooleanWeight(boost);
+  }
+
+  private static class BooleanWeight extends SimWeight {
+    final float boost;
+
+    BooleanWeight(float boost) {
+      this.boost = boost;
+    }
+  }
+
+  @Override
+  public SimScorer simScorer(SimWeight weight, LeafReaderContext context) throws IOException {
+    final float boost = ((BooleanWeight) weight).boost;
+
+    return new SimScorer() {
+
+      @Override
+      public float score(int doc, float freq) throws IOException {
+        return boost;
+      }
+
+      @Override
+      public Explanation explain(int doc, Explanation freq) throws IOException {
+        Explanation queryBoostExpl = Explanation.match(boost, "query boost");
+        return Explanation.match(
+            queryBoostExpl.getValue(),
+            "score(" + getClass().getSimpleName() + ", doc=" + doc + "), computed from:",
+            queryBoostExpl);
+      }
+
+      @Override
+      public float computeSlopFactor(int distance) {
+        return 1f;
+      }
+
+      @Override
+      public float computePayloadFactor(int doc, int start, int end, BytesRef payload) {
+        return 1f;
+      }
+    };
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3e15233b/lucene/core/src/test/org/apache/lucene/search/similarities/TestBooleanSimilarity.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/search/similarities/TestBooleanSimilarity.java b/lucene/core/src/test/org/apache/lucene/search/similarities/TestBooleanSimilarity.java
new file mode 100644
index 0000000..15b1448
--- /dev/null
+++ b/lucene/core/src/test/org/apache/lucene/search/similarities/TestBooleanSimilarity.java
@@ -0,0 +1,117 @@
+/*
+ * 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.similarities;
+
+import java.io.IOException;
+
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field.Store;
+import org.apache.lucene.document.StringField;
+import org.apache.lucene.document.TextField;
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.FieldInvertState;
+import org.apache.lucene.index.RandomIndexWriter;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.BoostQuery;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.PhraseQuery;
+import org.apache.lucene.search.TermQuery;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.TestUtil;
+
+public class TestBooleanSimilarity extends LuceneTestCase {
+
+  public void testTermScoreIsEqualToBoost() throws IOException {
+    Directory dir = newDirectory();
+    RandomIndexWriter w = new RandomIndexWriter(random(), dir,
+        newIndexWriterConfig());
+    Document doc = new Document();
+    doc.add(new StringField("foo", "bar", Store.NO));
+    doc.add(new StringField("foo", "baz", Store.NO));
+    w.addDocument(doc);
+    doc = new Document();
+    doc.add(new StringField("foo", "bar", Store.NO));
+    doc.add(new StringField("foo", "bar", Store.NO));
+    w.addDocument(doc);
+
+    DirectoryReader reader = w.getReader();
+    w.close();
+    IndexSearcher searcher = newSearcher(reader);
+    searcher.setSimilarity(new BooleanSimilarity());
+    TopDocs topDocs = searcher.search(new TermQuery(new Term("foo", "bar")), 2);
+    assertEquals(2, topDocs.totalHits);
+    assertEquals(1f, topDocs.scoreDocs[0].score, 0f);
+    assertEquals(1f, topDocs.scoreDocs[1].score, 0f);
+
+    topDocs = searcher.search(new TermQuery(new Term("foo", "baz")), 1);
+    assertEquals(1, topDocs.totalHits);
+    assertEquals(1f, topDocs.scoreDocs[0].score, 0f);
+
+    topDocs = searcher.search(new BoostQuery(new TermQuery(new Term("foo", "baz")), 3f), 1);
+    assertEquals(1, topDocs.totalHits);
+    assertEquals(3f, topDocs.scoreDocs[0].score, 0f);
+
+    reader.close();
+    dir.close();
+  }
+
+  public void testPhraseScoreIsEqualToBoost() throws IOException {
+    Directory dir = newDirectory();
+    RandomIndexWriter w = new RandomIndexWriter(random(), dir,
+        newIndexWriterConfig().setSimilarity(new BooleanSimilarity()));
+    Document doc = new Document();
+    doc.add(new TextField("foo", "bar baz quux", Store.NO));
+    w.addDocument(doc);
+
+    DirectoryReader reader = w.getReader();
+    w.close();
+    IndexSearcher searcher = newSearcher(reader);
+    searcher.setSimilarity(new BooleanSimilarity());
+
+    PhraseQuery query = new PhraseQuery(2, "foo", "bar", "quux");
+
+    TopDocs topDocs = searcher.search(query, 2);
+    assertEquals(1, topDocs.totalHits);
+    assertEquals(1f, topDocs.scoreDocs[0].score, 0f);
+
+    topDocs = searcher.search(new BoostQuery(query, 7), 2);
+    assertEquals(1, topDocs.totalHits);
+    assertEquals(7f, topDocs.scoreDocs[0].score, 0f);
+
+    reader.close();
+    dir.close();
+  }
+
+  public void testSameNormsAsBM25() {
+    BooleanSimilarity sim1 = new BooleanSimilarity();
+    BM25Similarity sim2 = new BM25Similarity();
+    sim2.setDiscountOverlaps(true);
+    for (int iter = 0; iter < 100; ++iter) {
+      final int length = TestUtil.nextInt(random(), 1, 100);
+      final int position = random().nextInt(length);
+      final int numOverlaps = random().nextInt(50);
+      final float boost = random().nextFloat() * 10;
+      FieldInvertState state = new FieldInvertState("foo", position, length, numOverlaps, 100, boost);
+      assertEquals(
+          sim2.computeNorm(state),
+          sim1.computeNorm(state),
+          0f);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3e15233b/lucene/test-framework/src/java/org/apache/lucene/search/similarities/RandomSimilarity.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/java/org/apache/lucene/search/similarities/RandomSimilarity.java b/lucene/test-framework/src/java/org/apache/lucene/search/similarities/RandomSimilarity.java
index 136d7e5..43b6c3c 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/search/similarities/RandomSimilarity.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/search/similarities/RandomSimilarity.java
@@ -91,6 +91,7 @@ public class RandomSimilarity extends PerFieldSimilarityWrapper {
     allSims = new ArrayList<>();
     allSims.add(new ClassicSimilarity());
     allSims.add(new BM25Similarity());
+    allSims.add(new BooleanSimilarity());
     for (BasicModel basicModel : BASIC_MODELS) {
       for (AfterEffect afterEffect : AFTER_EFFECTS) {
         for (Normalization normalization : NORMALIZATIONS) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3e15233b/lucene/test-framework/src/java/org/apache/lucene/util/TestRuleSetupAndRestoreClassEnv.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/java/org/apache/lucene/util/TestRuleSetupAndRestoreClassEnv.java b/lucene/test-framework/src/java/org/apache/lucene/util/TestRuleSetupAndRestoreClassEnv.java
index 0e4facc..96b4353 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/util/TestRuleSetupAndRestoreClassEnv.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/util/TestRuleSetupAndRestoreClassEnv.java
@@ -36,7 +36,6 @@ import org.apache.lucene.codecs.lucene70.Lucene70Codec;
 import org.apache.lucene.codecs.mockrandom.MockRandomPostingsFormat;
 import org.apache.lucene.codecs.simpletext.SimpleTextCodec;
 import org.apache.lucene.index.RandomCodec;
-import org.apache.lucene.search.similarities.ClassicSimilarity;
 import org.apache.lucene.search.similarities.RandomSimilarity;
 import org.apache.lucene.search.similarities.Similarity;
 import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
@@ -213,7 +212,7 @@ final class TestRuleSetupAndRestoreClassEnv extends AbstractBeforeAfterRule {
     TimeZone randomTimeZone = randomTimeZone(random());
     timeZone = testTimeZone.equals("random") ? randomTimeZone : TimeZone.getTimeZone(testTimeZone);
     TimeZone.setDefault(timeZone);
-    similarity = random().nextBoolean() ? new ClassicSimilarity() : new RandomSimilarity(random());
+    similarity = new RandomSimilarity(random());
 
     // Check codec restrictions once at class level.
     try {


[4/4] lucene-solr:master: LUCENE-7524: More detailed explanation of idf.

Posted by jp...@apache.org.
LUCENE-7524: More detailed explanation of idf.


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

Branch: refs/heads/master
Commit: 6b9f11311a2481743d8b2b42a875a11162a5e215
Parents: 3e15233
Author: Adrien Grand <jp...@gmail.com>
Authored: Thu Nov 10 14:08:26 2016 +0100
Committer: Adrien Grand <jp...@gmail.com>
Committed: Thu Nov 10 15:01:49 2016 +0100

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |  3 +++
 .../search/similarities/BM25Similarity.java     | 20 ++++++++++----------
 .../search/similarities/ClassicSimilarity.java  | 13 +++++++++++++
 .../search/similarities/TFIDFSimilarity.java    | 12 +++++-------
 .../org/apache/lucene/search/CheckHits.java     | 10 ++++++----
 5 files changed, 37 insertions(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6b9f1131/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 97e6939..5b9fee7 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -77,6 +77,9 @@ Improvements
   now throws IllegalArgumentException instead of a cryptic exception
   that closes your IndexWriter (Steve Chen via Mike McCandless)
 
+* LUCENE-7524: Added more detailed explanation of how IDF is computed in
+  ClassicSimilarity and BM25Similarity. (Adrien Grand)
+
 ======================= Lucene 6.3.0 =======================
 
 API Changes

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6b9f1131/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 ff390b3..6763118 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
@@ -175,7 +175,9 @@ public class BM25Similarity extends Similarity {
     final long df = termStats.docFreq();
     final long docCount = collectionStats.docCount() == -1 ? collectionStats.maxDoc() : collectionStats.docCount();
     final float idf = idf(df, docCount);
-    return Explanation.match(idf, "idf(docFreq=" + df + ", docCount=" + docCount + ")");
+    return Explanation.match(idf, "idf, computed as log(1 + (docCount - docFreq + 0.5) / (docFreq + 0.5)) from:",
+        Explanation.match(df, "docFreq"),
+        Explanation.match(docCount, "docCount"));
   }
 
   /**
@@ -192,16 +194,14 @@ public class BM25Similarity extends Similarity {
    *         for each term.
    */
   public Explanation idfExplain(CollectionStatistics collectionStats, TermStatistics termStats[]) {
-    final long docCount = collectionStats.docCount() == -1 ? collectionStats.maxDoc() : collectionStats.docCount();
-    float idf = 0.0f;
+    double idf = 0d; // sum into a double before casting into a float
     List<Explanation> details = new ArrayList<>();
     for (final TermStatistics stat : termStats ) {
-      final long df = stat.docFreq();
-      final float termIdf = idf(df, docCount);
-      details.add(Explanation.match(termIdf, "idf(docFreq=" + df + ", docCount=" + docCount + ")"));
-      idf += termIdf;
+      Explanation idfExplain = idfExplain(collectionStats, stat);
+      details.add(idfExplain);
+      idf += idfExplain.getValue();
     }
-    return Explanation.match(idf, "idf(), sum of:", details);
+    return Explanation.match((float) idf, "idf(), sum of:", details);
   }
 
   @Override
@@ -303,7 +303,7 @@ public class BM25Similarity extends Similarity {
       subs.add(Explanation.match(0, "parameter b (norms omitted for field)"));
       return Explanation.match(
           (freq.getValue() * (k1 + 1)) / (freq.getValue() + k1),
-          "tfNorm, computed from:", subs);
+          "tfNorm, computed as (freq * (k1 + 1)) / (freq + k1) from:", subs);
     } else {
       byte norm;
       if (norms.advanceExact(doc)) {
@@ -317,7 +317,7 @@ public class BM25Similarity extends Similarity {
       subs.add(Explanation.match(doclen, "fieldLength"));
       return Explanation.match(
           (freq.getValue() * (k1 + 1)) / (freq.getValue() + k1 * (1 - b + b * doclen/stats.avgdl)),
-          "tfNorm, computed from:", subs);
+          "tfNorm, computed as (freq * (k1 + 1)) / (freq + k1 * (1 - b + b * fieldLength / avgFieldLength)) from:", subs);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6b9f1131/lucene/core/src/java/org/apache/lucene/search/similarities/ClassicSimilarity.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/similarities/ClassicSimilarity.java b/lucene/core/src/java/org/apache/lucene/search/similarities/ClassicSimilarity.java
index fae85a0..5a1e237 100644
--- a/lucene/core/src/java/org/apache/lucene/search/similarities/ClassicSimilarity.java
+++ b/lucene/core/src/java/org/apache/lucene/search/similarities/ClassicSimilarity.java
@@ -18,6 +18,9 @@ package org.apache.lucene.search.similarities;
 
 
 import org.apache.lucene.index.FieldInvertState;
+import org.apache.lucene.search.CollectionStatistics;
+import org.apache.lucene.search.Explanation;
+import org.apache.lucene.search.TermStatistics;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.SmallFloat;
 
@@ -121,6 +124,16 @@ public class ClassicSimilarity extends TFIDFSimilarity {
     return 1;
   }
 
+  @Override
+  public Explanation idfExplain(CollectionStatistics collectionStats, TermStatistics termStats) {
+    final long df = termStats.docFreq();
+    final long docCount = collectionStats.docCount() == -1 ? collectionStats.maxDoc() : collectionStats.docCount();
+    final float idf = idf(df, docCount);
+    return Explanation.match(idf, "idf, computed as log((docCount+1)/(docFreq+1)) + 1 from:",
+        Explanation.match(df, "docFreq"),
+        Explanation.match(docCount, "docCount"));
+  }
+
   /** Implemented as <code>log((docCount+1)/(docFreq+1)) + 1</code>. */
   @Override
   public float idf(long docFreq, long docCount) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6b9f1131/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 cd8acd6..12ab1a2 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
@@ -484,16 +484,14 @@ public abstract class TFIDFSimilarity extends Similarity {
    *         for each term.
    */
   public Explanation idfExplain(CollectionStatistics collectionStats, TermStatistics termStats[]) {
-    final long docCount = collectionStats.docCount() == -1 ? collectionStats.maxDoc() : collectionStats.docCount();
-    float idf = 0.0f;
+    double idf = 0d; // sum into a double before casting into a float
     List<Explanation> subs = new ArrayList<>();
     for (final TermStatistics stat : termStats ) {
-      final long df = stat.docFreq();
-      final float termIdf = idf(df, docCount);
-      subs.add(Explanation.match(termIdf, "idf(docFreq=" + df + ", docCount=" + docCount + ")"));
-      idf += termIdf;
+      Explanation idfExplain = idfExplain(collectionStats, stat);
+      subs.add(idfExplain);
+      idf += idfExplain.getValue();
     }
-    return Explanation.match(idf, "idf(), sum of:", subs);
+    return Explanation.match((float) idf, "idf(), sum of:", subs);
   }
 
   /** Computes a score factor based on a term's document frequency (the number

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6b9f1131/lucene/test-framework/src/java/org/apache/lucene/search/CheckHits.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/java/org/apache/lucene/search/CheckHits.java b/lucene/test-framework/src/java/org/apache/lucene/search/CheckHits.java
index 9a14b5d..dee7d84 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/search/CheckHits.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/search/CheckHits.java
@@ -368,8 +368,9 @@ public class CheckHits {
         boolean productOf = descr.endsWith("product of:");
         boolean sumOf = descr.endsWith("sum of:");
         boolean maxOf = descr.endsWith("max of:");
+        boolean computedOf = descr.matches(".*, computed as .* from:");
         boolean maxTimesOthers = false;
-        if (!(productOf || sumOf || maxOf)) {
+        if (!(productOf || sumOf || maxOf || computedOf)) {
           // maybe 'max plus x times others'
           int k1 = descr.indexOf("max plus ");
           if (k1>=0) {
@@ -387,9 +388,9 @@ public class CheckHits {
         // TODO: this is a TERRIBLE assertion!!!!
         Assert.assertTrue(
             q+": multi valued explanation description=\""+descr
-            +"\" must be 'max of plus x times others' or end with 'product of'"
+            +"\" must be 'max of plus x times others', 'computed as x from:' or end with 'product of'"
             +" or 'sum of:' or 'max of:' - "+expl,
-            productOf || sumOf || maxOf || maxTimesOthers);
+            productOf || sumOf || maxOf || computedOf || maxTimesOthers);
         float sum = 0;
         float product = 1;
         float max = 0;
@@ -410,7 +411,8 @@ public class CheckHits {
         } else if (maxTimesOthers) {
           combined = max + x * (sum - max);
         } else {
-            Assert.assertTrue("should never get here!",false);
+          Assert.assertTrue("should never get here!", computedOf);
+          combined = value;
         }
         Assert.assertEquals(q+": actual subDetails combined=="+combined+
             " != value="+value+" Explanation: "+expl,