You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ro...@apache.org on 2017/01/09 15:32:02 UTC

[1/2] lucene-solr:master: LUCENE-7624: Move TermsQuery into core as TermInSetQuery

Repository: lucene-solr
Updated Branches:
  refs/heads/branch_6x 31b8a4428 -> 8511f9e69
  refs/heads/master 5e9f9279c -> 22940f5c4


LUCENE-7624: Move TermsQuery into core as TermInSetQuery


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

Branch: refs/heads/master
Commit: 22940f5c49297b606d710c6775309d67ff064f2f
Parents: 5e9f927
Author: Alan Woodward <ro...@apache.org>
Authored: Mon Jan 9 14:01:33 2017 +0000
Committer: Alan Woodward <ro...@apache.org>
Committed: Mon Jan 9 14:25:05 2017 +0000

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |   3 +
 .../apache/lucene/search/TermInSetQuery.java    | 369 +++++++++++++++++++
 .../search/UsageTrackingQueryCachingPolicy.java |   5 +-
 .../lucene/search/TermInSetQueryTest.java       | 328 +++++++++++++++++
 .../apache/lucene/facet/MultiFacetQuery.java    |  12 +-
 .../org/apache/lucene/queries/TermsQuery.java   | 332 +----------------
 .../apache/lucene/queries/TermsQueryTest.java   | 339 -----------------
 .../prefix/TermQueryPrefixTreeStrategy.java     |  10 +-
 .../spatial/prefix/NumberRangeFacetsTest.java   |   6 +-
 .../solr/handler/component/ExpandComponent.java |  25 +-
 .../java/org/apache/solr/schema/FieldType.java  |   4 +-
 .../apache/solr/search/TermsQParserPlugin.java  |   4 +-
 .../org/apache/solr/search/join/GraphQuery.java |   4 +-
 .../apache/solr/search/TestSolrQueryParser.java |  12 +-
 14 files changed, 748 insertions(+), 705 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/22940f5c/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 4bbf9ee..109a534 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -85,6 +85,9 @@ API Changes
 * LUCENE-7611: DocumentValueSourceDictionary now takes a LongValuesSource
   as a parameter, and the ValueSource equivalent is deprecated (Alan Woodward)
 
+* LUCENE-7624: TermsQuery has been renamed as TermInSetQuery and moved to core.
+  (Alan Woodward)
+
 New features
 
 * LUCENE-5867: Added BooleanSimilarity. (Robert Muir, Adrien Grand)

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/22940f5c/lucene/core/src/java/org/apache/lucene/search/TermInSetQuery.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/TermInSetQuery.java b/lucene/core/src/java/org/apache/lucene/search/TermInSetQuery.java
new file mode 100644
index 0000000..e1a1575
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/search/TermInSetQuery.java
@@ -0,0 +1,369 @@
+/*
+ * 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;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Objects;
+import java.util.Set;
+import java.util.SortedSet;
+
+import org.apache.lucene.index.Fields;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.LeafReader;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.PostingsEnum;
+import org.apache.lucene.index.PrefixCodedTerms;
+import org.apache.lucene.index.PrefixCodedTerms.TermIterator;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.index.TermContext;
+import org.apache.lucene.index.TermState;
+import org.apache.lucene.index.Terms;
+import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.search.BooleanClause.Occur;
+import org.apache.lucene.util.Accountable;
+import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefBuilder;
+import org.apache.lucene.util.DocIdSetBuilder;
+import org.apache.lucene.util.RamUsageEstimator;
+
+/**
+ * Specialization for a disjunction over many terms that behaves like a
+ * {@link ConstantScoreQuery} over a {@link BooleanQuery} containing only
+ * {@link org.apache.lucene.search.BooleanClause.Occur#SHOULD} clauses.
+ * <p>For instance in the following example, both @{code q1} and {@code q2}
+ * would yield the same scores:
+ * <pre class="prettyprint">
+ * Query q1 = new TermInSetQuery(new Term("field", "foo"), new Term("field", "bar"));
+ *
+ * BooleanQuery bq = new BooleanQuery();
+ * bq.add(new TermQuery(new Term("field", "foo")), Occur.SHOULD);
+ * bq.add(new TermQuery(new Term("field", "bar")), Occur.SHOULD);
+ * Query q2 = new ConstantScoreQuery(bq);
+ * </pre>
+ * <p>When there are few terms, this query executes like a regular disjunction.
+ * However, when there are many terms, instead of merging iterators on the fly,
+ * it will populate a bit set with matching docs and return a {@link Scorer}
+ * over this bit set.
+ * <p>NOTE: This query produces scores that are equal to its boost
+ */
+public class TermInSetQuery extends Query implements Accountable {
+
+  private static final long BASE_RAM_BYTES_USED = RamUsageEstimator.shallowSizeOfInstance(TermInSetQuery.class);
+  // Same threshold as MultiTermQueryConstantScoreWrapper
+  static final int BOOLEAN_REWRITE_TERM_COUNT_THRESHOLD = 16;
+
+  private final boolean singleField; // whether all terms are from the same field
+  private final PrefixCodedTerms termData;
+  private final int termDataHashCode; // cached hashcode of termData
+
+  /**
+   * Creates a new {@link TermInSetQuery} from the given collection. It
+   * can contain duplicate terms and multiple fields.
+   */
+  public TermInSetQuery(Collection<Term> terms) {
+    Term[] sortedTerms = terms.toArray(new Term[terms.size()]);
+    // already sorted if we are a SortedSet with natural order
+    boolean sorted = terms instanceof SortedSet && ((SortedSet<Term>)terms).comparator() == null;
+    if (!sorted) {
+      ArrayUtil.timSort(sortedTerms);
+    }
+    PrefixCodedTerms.Builder builder = new PrefixCodedTerms.Builder();
+    Set<String> fields = new HashSet<>();
+    Term previous = null;
+    for (Term term : sortedTerms) {
+      if (term.equals(previous) == false) {
+        fields.add(term.field());
+        builder.add(term);
+      }
+      previous = term;
+    }
+    singleField = fields.size() == 1;
+    termData = builder.finish();
+    termDataHashCode = termData.hashCode();
+  }
+
+  /**
+   * Creates a new {@link TermInSetQuery} from the given collection for
+   * a single field. It can contain duplicate terms.
+   */
+  public TermInSetQuery(String field, Collection<BytesRef> terms) {
+    BytesRef[] sortedTerms = terms.toArray(new BytesRef[terms.size()]);
+    // already sorted if we are a SortedSet with natural order
+    boolean sorted = terms instanceof SortedSet && ((SortedSet<BytesRef>)terms).comparator() == null;
+    if (!sorted) {
+      ArrayUtil.timSort(sortedTerms);
+    }
+    PrefixCodedTerms.Builder builder = new PrefixCodedTerms.Builder();
+    BytesRefBuilder previous = null;
+    for (BytesRef term : sortedTerms) {
+      if (previous == null) {
+        previous = new BytesRefBuilder();
+      } else if (previous.get().equals(term)) {
+        continue; // deduplicate
+      }
+      builder.add(field, term);
+      previous.copyBytes(term);
+    }
+    singleField = true;
+    termData = builder.finish();
+    termDataHashCode = termData.hashCode();
+  }
+
+  /**
+   * Creates a new {@link TermInSetQuery} from the given {@link BytesRef} array for
+   * a single field.
+   */
+  public TermInSetQuery(String field, BytesRef...terms) {
+    this(field, Arrays.asList(terms));
+  }
+
+  /**
+   * Creates a new {@link TermInSetQuery} from the given array. The array can
+   * contain duplicate terms and multiple fields.
+   */
+  public TermInSetQuery(final Term... terms) {
+    this(Arrays.asList(terms));
+  }
+
+  @Override
+  public Query rewrite(IndexReader reader) throws IOException {
+    final int threshold = Math.min(BOOLEAN_REWRITE_TERM_COUNT_THRESHOLD, BooleanQuery.getMaxClauseCount());
+    if (termData.size() <= threshold) {
+      BooleanQuery.Builder bq = new BooleanQuery.Builder();
+      TermIterator iterator = termData.iterator();
+      for (BytesRef term = iterator.next(); term != null; term = iterator.next()) {
+        bq.add(new TermQuery(new Term(iterator.field(), BytesRef.deepCopyOf(term))), Occur.SHOULD);
+      }
+      return new ConstantScoreQuery(bq.build());
+    }
+    return super.rewrite(reader);
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    return sameClassAs(other) &&
+        equalsTo(getClass().cast(other));
+  }
+
+  private boolean equalsTo(TermInSetQuery other) {
+    // termData might be heavy to compare so check the hash code first
+    return termDataHashCode == other.termDataHashCode &&
+        termData.equals(other.termData);
+  }
+
+  @Override
+  public int hashCode() {
+    return 31 * classHash() + termDataHashCode;
+  }
+
+  /** Returns the terms wrapped in a PrefixCodedTerms. */
+  public PrefixCodedTerms getTermData() {
+    return termData;
+  }
+
+  @Override
+  public String toString(String defaultField) {
+    StringBuilder builder = new StringBuilder();
+    boolean first = true;
+    TermIterator iterator = termData.iterator();
+    for (BytesRef term = iterator.next(); term != null; term = iterator.next()) {
+      if (!first) {
+        builder.append(' ');
+      }
+      first = false;
+      builder.append(new Term(iterator.field(), term).toString());
+    }
+
+    return builder.toString();
+  }
+
+  @Override
+  public long ramBytesUsed() {
+    return BASE_RAM_BYTES_USED + termData.ramBytesUsed();
+  }
+
+  @Override
+  public Collection<Accountable> getChildResources() {
+    return Collections.emptyList();
+  }
+
+  private static class TermAndState {
+    final String field;
+    final TermsEnum termsEnum;
+    final BytesRef term;
+    final TermState state;
+    final int docFreq;
+    final long totalTermFreq;
+
+    TermAndState(String field, TermsEnum termsEnum) throws IOException {
+      this.field = field;
+      this.termsEnum = termsEnum;
+      this.term = BytesRef.deepCopyOf(termsEnum.term());
+      this.state = termsEnum.termState();
+      this.docFreq = termsEnum.docFreq();
+      this.totalTermFreq = termsEnum.totalTermFreq();
+    }
+  }
+
+  private static class WeightOrDocIdSet {
+    final Weight weight;
+    final DocIdSet set;
+
+    WeightOrDocIdSet(Weight weight) {
+      this.weight = Objects.requireNonNull(weight);
+      this.set = null;
+    }
+
+    WeightOrDocIdSet(DocIdSet bitset) {
+      this.set = bitset;
+      this.weight = null;
+    }
+  }
+
+  @Override
+  public Weight createWeight(IndexSearcher searcher, boolean needsScores, float boost) throws IOException {
+    return new ConstantScoreWeight(this, boost) {
+
+      @Override
+      public void extractTerms(Set<Term> terms) {
+        // no-op
+        // This query is for abuse cases when the number of terms is too high to
+        // run efficiently as a BooleanQuery. So likewise we hide its terms in
+        // order to protect highlighters
+      }
+
+      /**
+       * On the given leaf context, try to either rewrite to a disjunction if
+       * there are few matching terms, or build a bitset containing matching docs.
+       */
+      private WeightOrDocIdSet rewrite(LeafReaderContext context) throws IOException {
+        final LeafReader reader = context.reader();
+
+        // We will first try to collect up to 'threshold' terms into 'matchingTerms'
+        // if there are two many terms, we will fall back to building the 'builder'
+        final int threshold = Math.min(BOOLEAN_REWRITE_TERM_COUNT_THRESHOLD, BooleanQuery.getMaxClauseCount());
+        assert termData.size() > threshold : "Query should have been rewritten";
+        List<TermAndState> matchingTerms = new ArrayList<>(threshold);
+        DocIdSetBuilder builder = null;
+
+        final Fields fields = reader.fields();
+        String lastField = null;
+        Terms terms = null;
+        TermsEnum termsEnum = null;
+        PostingsEnum docs = null;
+        TermIterator iterator = termData.iterator();
+        for (BytesRef term = iterator.next(); term != null; term = iterator.next()) {
+          String field = iterator.field();
+          // comparing references is fine here
+          if (field != lastField) {
+            terms = fields.terms(field);
+            if (terms == null) {
+              termsEnum = null;
+            } else {
+              termsEnum = terms.iterator();
+            }
+            lastField = field;
+          }
+          if (termsEnum != null && termsEnum.seekExact(term)) {
+            if (matchingTerms == null) {
+              docs = termsEnum.postings(docs, PostingsEnum.NONE);
+              builder.add(docs);
+            } else if (matchingTerms.size() < threshold) {
+              matchingTerms.add(new TermAndState(field, termsEnum));
+            } else {
+              assert matchingTerms.size() == threshold;
+              if (singleField) {
+                // common case: all terms are in the same field
+                // use an optimized builder that leverages terms stats to be more efficient
+                builder = new DocIdSetBuilder(reader.maxDoc(), terms);
+              } else {
+                // corner case: different fields
+                // don't make assumptions about the docs we will get
+                builder = new DocIdSetBuilder(reader.maxDoc());
+              }
+              docs = termsEnum.postings(docs, PostingsEnum.NONE);
+              builder.add(docs);
+              for (TermAndState t : matchingTerms) {
+                t.termsEnum.seekExact(t.term, t.state);
+                docs = t.termsEnum.postings(docs, PostingsEnum.NONE);
+                builder.add(docs);
+              }
+              matchingTerms = null;
+            }
+          }
+        }
+        if (matchingTerms != null) {
+          assert builder == null;
+          BooleanQuery.Builder bq = new BooleanQuery.Builder();
+          for (TermAndState t : matchingTerms) {
+            final TermContext termContext = new TermContext(searcher.getTopReaderContext());
+            termContext.register(t.state, context.ord, t.docFreq, t.totalTermFreq);
+            bq.add(new TermQuery(new Term(t.field, t.term), termContext), Occur.SHOULD);
+          }
+          Query q = new ConstantScoreQuery(bq.build());
+          final Weight weight = searcher.rewrite(q).createWeight(searcher, needsScores, score());
+          return new WeightOrDocIdSet(weight);
+        } else {
+          assert builder != null;
+          return new WeightOrDocIdSet(builder.build());
+        }
+      }
+
+      private Scorer scorer(DocIdSet set) throws IOException {
+        if (set == null) {
+          return null;
+        }
+        final DocIdSetIterator disi = set.iterator();
+        if (disi == null) {
+          return null;
+        }
+        return new ConstantScoreScorer(this, score(), disi);
+      }
+
+      @Override
+      public BulkScorer bulkScorer(LeafReaderContext context) throws IOException {
+        final WeightOrDocIdSet weightOrBitSet = rewrite(context);
+        if (weightOrBitSet.weight != null) {
+          return weightOrBitSet.weight.bulkScorer(context);
+        } else {
+          final Scorer scorer = scorer(weightOrBitSet.set);
+          if (scorer == null) {
+            return null;
+          }
+          return new DefaultBulkScorer(scorer);
+        }
+      }
+
+      @Override
+      public Scorer scorer(LeafReaderContext context) throws IOException {
+        final WeightOrDocIdSet weightOrBitSet = rewrite(context);
+        if (weightOrBitSet.weight != null) {
+          return weightOrBitSet.weight.scorer(context);
+        } else {
+          return scorer(weightOrBitSet.set);
+        }
+      }
+    };
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/22940f5c/lucene/core/src/java/org/apache/lucene/search/UsageTrackingQueryCachingPolicy.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/UsageTrackingQueryCachingPolicy.java b/lucene/core/src/java/org/apache/lucene/search/UsageTrackingQueryCachingPolicy.java
index ab68eeb..035947f 100644
--- a/lucene/core/src/java/org/apache/lucene/search/UsageTrackingQueryCachingPolicy.java
+++ b/lucene/core/src/java/org/apache/lucene/search/UsageTrackingQueryCachingPolicy.java
@@ -50,9 +50,8 @@ public final class UsageTrackingQueryCachingPolicy implements QueryCachingPolicy
     // DocIdSet in the first place
     return query instanceof MultiTermQuery ||
         query instanceof MultiTermQueryConstantScoreWrapper ||
-        isPointQuery(query) ||
-        // can't refer to TermsQuery directly as it is in another module
-        "TermsQuery".equals(query.getClass().getSimpleName());
+        query instanceof TermInSetQuery ||
+        isPointQuery(query);
   }
 
   static boolean isCheap(Query query) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/22940f5c/lucene/core/src/test/org/apache/lucene/search/TermInSetQueryTest.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/search/TermInSetQueryTest.java b/lucene/core/src/test/org/apache/lucene/search/TermInSetQueryTest.java
new file mode 100644
index 0000000..e694d97
--- /dev/null
+++ b/lucene/core/src/test/org/apache/lucene/search/TermInSetQueryTest.java
@@ -0,0 +1,328 @@
+/*
+ * 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;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import com.carrotsearch.randomizedtesting.generators.RandomPicks;
+import com.carrotsearch.randomizedtesting.generators.RandomStrings;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field.Store;
+import org.apache.lucene.document.StringField;
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.Fields;
+import org.apache.lucene.index.FilterDirectoryReader;
+import org.apache.lucene.index.FilterLeafReader;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.LeafReader;
+import org.apache.lucene.index.RandomIndexWriter;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.index.Terms;
+import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.search.BooleanClause.Occur;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.RamUsageTester;
+import org.apache.lucene.util.TestUtil;
+
+public class TermInSetQueryTest extends LuceneTestCase {
+
+  public void testDuel() throws IOException {
+    final int iters = atLeast(2);
+    for (int iter = 0; iter < iters; ++iter) {
+      final List<Term> allTerms = new ArrayList<>();
+      final int numTerms = TestUtil.nextInt(random(), 1, 1 << TestUtil.nextInt(random(), 1, 10));
+      for (int i = 0; i < numTerms; ++i) {
+        final String field = usually() ? "f" : "g";
+        final String value = TestUtil.randomAnalysisString(random(), 10, true);
+        allTerms.add(new Term(field, value));
+      }
+      Directory dir = newDirectory();
+      RandomIndexWriter iw = new RandomIndexWriter(random(), dir);
+      final int numDocs = atLeast(100);
+      for (int i = 0; i < numDocs; ++i) {
+        Document doc = new Document();
+        final Term term = allTerms.get(random().nextInt(allTerms.size()));
+        doc.add(new StringField(term.field(), term.text(), Store.NO));
+        iw.addDocument(doc);
+      }
+      if (numTerms > 1 && random().nextBoolean()) {
+        iw.deleteDocuments(new TermQuery(allTerms.get(0)));
+      }
+      iw.commit();
+      final IndexReader reader = iw.getReader();
+      final IndexSearcher searcher = newSearcher(reader);
+      iw.close();
+
+      if (reader.numDocs() == 0) {
+        // may occasionally happen if all documents got the same term
+        IOUtils.close(reader, dir);
+        continue;
+      }
+
+      for (int i = 0; i < 100; ++i) {
+        final float boost = random().nextFloat() * 10;
+        final int numQueryTerms = TestUtil.nextInt(random(), 1, 1 << TestUtil.nextInt(random(), 1, 8));
+        List<Term> queryTerms = new ArrayList<>();
+        for (int j = 0; j < numQueryTerms; ++j) {
+          queryTerms.add(allTerms.get(random().nextInt(allTerms.size())));
+        }
+        final BooleanQuery.Builder bq = new BooleanQuery.Builder();
+        for (Term t : queryTerms) {
+          bq.add(new TermQuery(t), Occur.SHOULD);
+        }
+        final Query q1 = new ConstantScoreQuery(bq.build());
+        final Query q2 = new TermInSetQuery(queryTerms);
+        assertSameMatches(searcher, new BoostQuery(q1, boost), new BoostQuery(q2, boost), true);
+      }
+
+      reader.close();
+      dir.close();
+    }
+  }
+
+  private void assertSameMatches(IndexSearcher searcher, Query q1, Query q2, boolean scores) throws IOException {
+    final int maxDoc = searcher.getIndexReader().maxDoc();
+    final TopDocs td1 = searcher.search(q1, maxDoc, scores ? Sort.RELEVANCE : Sort.INDEXORDER);
+    final TopDocs td2 = searcher.search(q2, maxDoc, scores ? Sort.RELEVANCE : Sort.INDEXORDER);
+    assertEquals(td1.totalHits, td2.totalHits);
+    for (int i = 0; i < td1.scoreDocs.length; ++i) {
+      assertEquals(td1.scoreDocs[i].doc, td2.scoreDocs[i].doc);
+      if (scores) {
+        assertEquals(td1.scoreDocs[i].score, td2.scoreDocs[i].score, 10e-7);
+      }
+    }
+  }
+
+  private TermInSetQuery termsQuery(boolean singleField, Term...terms) {
+    return termsQuery(singleField, Arrays.asList(terms));
+  }
+
+  private TermInSetQuery termsQuery(boolean singleField, Collection<Term> termList) {
+    if (!singleField) {
+      return new TermInSetQuery(new ArrayList<>(termList));
+    }
+    final TermInSetQuery filter;
+    List<BytesRef> bytes = new ArrayList<>();
+    String field = null;
+    for (Term term : termList) {
+        bytes.add(term.bytes());
+        if (field != null) {
+          assertEquals(term.field(), field);
+        }
+        field = term.field();
+    }
+    assertNotNull(field);
+    filter = new TermInSetQuery(field, bytes);
+    return filter;
+  }
+
+  public void testHashCodeAndEquals() {
+    int num = atLeast(100);
+    final boolean singleField = random().nextBoolean();
+    List<Term> terms = new ArrayList<>();
+    Set<Term> uniqueTerms = new HashSet<>();
+    for (int i = 0; i < num; i++) {
+      String field = "field" + (singleField ? "1" : random().nextInt(100));
+      String string = TestUtil.randomRealisticUnicodeString(random());
+      terms.add(new Term(field, string));
+      uniqueTerms.add(new Term(field, string));
+      TermInSetQuery left = termsQuery(singleField ? random().nextBoolean() : false, uniqueTerms);
+      Collections.shuffle(terms, random());
+      TermInSetQuery right = termsQuery(singleField ? random().nextBoolean() : false, terms);
+      assertEquals(right, left);
+      assertEquals(right.hashCode(), left.hashCode());
+      if (uniqueTerms.size() > 1) {
+        List<Term> asList = new ArrayList<>(uniqueTerms);
+        asList.remove(0);
+        TermInSetQuery notEqual = termsQuery(singleField ? random().nextBoolean() : false, asList);
+        assertFalse(left.equals(notEqual));
+        assertFalse(right.equals(notEqual));
+      }
+    }
+
+    TermInSetQuery tq1 = new TermInSetQuery(new Term("thing", "apple"));
+    TermInSetQuery tq2 = new TermInSetQuery(new Term("thing", "orange"));
+    assertFalse(tq1.hashCode() == tq2.hashCode());
+
+    // different fields with the same term should have differing hashcodes
+    tq1 = new TermInSetQuery(new Term("thing1", "apple"));
+    tq2 = new TermInSetQuery(new Term("thing2", "apple"));
+    assertFalse(tq1.hashCode() == tq2.hashCode());
+  }
+
+  public void testSingleFieldEquals() {
+    // Two terms with the same hash code
+    assertEquals("AaAaBB".hashCode(), "BBBBBB".hashCode());
+    TermInSetQuery left = termsQuery(true, new Term("id", "AaAaAa"), new Term("id", "AaAaBB"));
+    TermInSetQuery right = termsQuery(true, new Term("id", "AaAaAa"), new Term("id", "BBBBBB"));
+    assertFalse(left.equals(right));
+  }
+
+  public void testToString() {
+    TermInSetQuery termsQuery = new TermInSetQuery(new Term("field1", "a"),
+                                              new Term("field1", "b"),
+                                              new Term("field1", "c"));
+    assertEquals("field1:a field1:b field1:c", termsQuery.toString());
+  }
+
+  public void testDedup() {
+    Query query1 = new TermInSetQuery(new Term("foo", "bar"));
+    Query query2 = new TermInSetQuery(new Term("foo", "bar"), new Term("foo", "bar"));
+    QueryUtils.checkEqual(query1, query2);
+  }
+
+  public void testOrderDoesNotMatter() {
+    // order of terms if different
+    Query query1 = new TermInSetQuery(new Term("foo", "bar"), new Term("foo", "baz"));
+    Query query2 = new TermInSetQuery(new Term("foo", "baz"), new Term("foo", "bar"));
+    QueryUtils.checkEqual(query1, query2);
+
+    // order of fields is different
+    query1 = new TermInSetQuery(new Term("foo", "bar"), new Term("bar", "bar"));
+    query2 = new TermInSetQuery(new Term("bar", "bar"), new Term("foo", "bar"));
+    QueryUtils.checkEqual(query1, query2);
+  }
+
+  public void testRamBytesUsed() {
+    List<Term> terms = new ArrayList<>();
+    final int numTerms = 1000 + random().nextInt(1000);
+    for (int i = 0; i < numTerms; ++i) {
+      terms.add(new Term("f", RandomStrings.randomUnicodeOfLength(random(), 10)));
+    }
+    TermInSetQuery query = new TermInSetQuery(terms);
+    final long actualRamBytesUsed = RamUsageTester.sizeOf(query);
+    final long expectedRamBytesUsed = query.ramBytesUsed();
+    // error margin within 5%
+    assertEquals(actualRamBytesUsed, expectedRamBytesUsed, actualRamBytesUsed / 20);
+  }
+
+  private static class TermsCountingDirectoryReaderWrapper extends FilterDirectoryReader {
+
+    private final AtomicInteger counter;
+    
+    public TermsCountingDirectoryReaderWrapper(DirectoryReader in, AtomicInteger counter) throws IOException {
+      super(in, new TermsCountingSubReaderWrapper(counter));
+      this.counter = counter;
+    }
+
+    private static class TermsCountingSubReaderWrapper extends SubReaderWrapper {
+      private final AtomicInteger counter;
+
+      public TermsCountingSubReaderWrapper(AtomicInteger counter) {
+        this.counter = counter;
+      }
+
+      @Override
+      public LeafReader wrap(LeafReader reader) {
+        return new TermsCountingLeafReaderWrapper(reader, counter);
+      }
+    }
+
+    private static class TermsCountingLeafReaderWrapper extends FilterLeafReader {
+
+      private final AtomicInteger counter;
+
+      public TermsCountingLeafReaderWrapper(LeafReader in, AtomicInteger counter) {
+        super(in);
+        this.counter = counter;
+      }
+
+      @Override
+      public Fields fields() throws IOException {
+        return new FilterFields(in.fields()) {
+          @Override
+          public Terms terms(String field) throws IOException {
+            final Terms in = this.in.terms(field);
+            if (in == null) {
+              return null;
+            }
+            return new FilterTerms(in) {
+              @Override
+              public TermsEnum iterator() throws IOException {
+                counter.incrementAndGet();
+                return super.iterator();
+              }
+            };
+          }
+        };
+      }
+      
+    }
+
+    @Override
+    protected DirectoryReader doWrapDirectoryReader(DirectoryReader in) throws IOException {
+      return new TermsCountingDirectoryReaderWrapper(in, counter);
+    }
+
+  }
+
+  public void testPullOneTermsEnumPerField() throws Exception {
+    Directory dir = newDirectory();
+    RandomIndexWriter w = new RandomIndexWriter(random(), dir);
+    Document doc = new Document();
+    doc.add(new StringField("foo", "1", Store.NO));
+    doc.add(new StringField("bar", "2", Store.NO));
+    doc.add(new StringField("baz", "3", Store.NO));
+    w.addDocument(doc);
+    DirectoryReader reader = w.getReader();
+    w.close();
+    final AtomicInteger counter = new AtomicInteger();
+    DirectoryReader wrapped = new TermsCountingDirectoryReaderWrapper(reader, counter);
+
+    final List<Term> terms = new ArrayList<>();
+    final Set<String> fields = new HashSet<>();
+    // enough terms to avoid the rewrite
+    final int numTerms = TestUtil.nextInt(random(), TermInSetQuery.BOOLEAN_REWRITE_TERM_COUNT_THRESHOLD + 1, 100);
+    for (int i = 0; i < numTerms; ++i) {
+      final String field = RandomPicks.randomFrom(random(), new String[] {"foo", "bar", "baz"});
+      final BytesRef term = new BytesRef(RandomStrings.randomUnicodeOfCodepointLength(random(), 10));
+      fields.add(field);
+      terms.add(new Term(field, term));
+    }
+
+    new IndexSearcher(wrapped).count(new TermInSetQuery(terms));
+    assertEquals(fields.size(), counter.get());
+    wrapped.close();
+    dir.close();
+  }
+  
+  public void testBinaryToString() {
+    TermInSetQuery query = new TermInSetQuery(new Term("field", new BytesRef(new byte[] { (byte) 0xff, (byte) 0xfe })));
+    assertEquals("field:[ff fe]", query.toString());
+  }
+
+  public void testIsConsideredCostlyByQueryCache() throws IOException {
+    TermInSetQuery query = new TermInSetQuery(new Term("foo", "bar"), new Term("foo", "baz"));
+    UsageTrackingQueryCachingPolicy policy = new UsageTrackingQueryCachingPolicy();
+    assertFalse(policy.shouldCache(query));
+    policy.onUse(query);
+    policy.onUse(query);
+    // cached after two uses
+    assertTrue(policy.shouldCache(query));
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/22940f5c/lucene/facet/src/java/org/apache/lucene/facet/MultiFacetQuery.java
----------------------------------------------------------------------
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/MultiFacetQuery.java b/lucene/facet/src/java/org/apache/lucene/facet/MultiFacetQuery.java
index dd212c6..a010709 100644
--- a/lucene/facet/src/java/org/apache/lucene/facet/MultiFacetQuery.java
+++ b/lucene/facet/src/java/org/apache/lucene/facet/MultiFacetQuery.java
@@ -16,13 +16,13 @@
  */
 package org.apache.lucene.facet;
 
-import org.apache.lucene.index.Term;
-import org.apache.lucene.queries.TermsQuery;
-import org.apache.lucene.search.Query;
-
 import java.util.ArrayList;
 import java.util.Collection;
 
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.TermInSetQuery;
+
 /**
  * A multi-terms {@link Query} over a {@link FacetField}.
  * <p>
@@ -30,9 +30,9 @@ import java.util.Collection;
  * especially in cases where you don't intend to use {@link DrillSideways}
  *
  * @lucene.experimental
- * @see org.apache.lucene.queries.TermsQuery
+ * @see org.apache.lucene.search.TermInSetQuery
  */
-public class MultiFacetQuery extends TermsQuery {
+public class MultiFacetQuery extends TermInSetQuery {
 
   /**
    * Creates a new {@code MultiFacetQuery} filtering the query on the given dimension.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/22940f5c/lucene/queries/src/java/org/apache/lucene/queries/TermsQuery.java
----------------------------------------------------------------------
diff --git a/lucene/queries/src/java/org/apache/lucene/queries/TermsQuery.java b/lucene/queries/src/java/org/apache/lucene/queries/TermsQuery.java
index 7b7f094..5effa83 100644
--- a/lucene/queries/src/java/org/apache/lucene/queries/TermsQuery.java
+++ b/lucene/queries/src/java/org/apache/lucene/queries/TermsQuery.java
@@ -16,130 +16,33 @@
  */
 package org.apache.lucene.queries;
 
-import java.io.IOException;
-import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Objects;
-import java.util.Set;
-import java.util.SortedSet;
 
-import org.apache.lucene.index.Fields;
-import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.index.LeafReader;
-import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.index.PostingsEnum;
-import org.apache.lucene.index.PrefixCodedTerms;
-import org.apache.lucene.index.PrefixCodedTerms.TermIterator;
 import org.apache.lucene.index.Term;
-import org.apache.lucene.index.TermContext;
-import org.apache.lucene.index.TermState;
-import org.apache.lucene.index.Terms;
-import org.apache.lucene.index.TermsEnum;
-import org.apache.lucene.search.BooleanClause.Occur;
-import org.apache.lucene.search.BooleanQuery;
-import org.apache.lucene.search.BulkScorer;
-import org.apache.lucene.search.ConstantScoreQuery;
-import org.apache.lucene.search.ConstantScoreScorer;
-import org.apache.lucene.search.ConstantScoreWeight;
-import org.apache.lucene.search.DocIdSet;
-import org.apache.lucene.search.DocIdSetIterator;
-import org.apache.lucene.search.IndexSearcher;
-import org.apache.lucene.search.Query;
-import org.apache.lucene.search.Scorer;
-import org.apache.lucene.search.TermQuery;
-import org.apache.lucene.search.Weight;
-import org.apache.lucene.util.Accountable;
-import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.search.TermInSetQuery;
 import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.DocIdSetBuilder;
-import org.apache.lucene.util.BytesRefBuilder;
-import org.apache.lucene.util.RamUsageEstimator;
 
 /**
- * Specialization for a disjunction over many terms that behaves like a
- * {@link ConstantScoreQuery} over a {@link BooleanQuery} containing only
- * {@link org.apache.lucene.search.BooleanClause.Occur#SHOULD} clauses.
- * <p>For instance in the following example, both @{code q1} and {@code q2}
- * would yield the same scores:
- * <pre class="prettyprint">
- * Query q1 = new TermsQuery(new Term("field", "foo"), new Term("field", "bar"));
- *
- * BooleanQuery bq = new BooleanQuery();
- * bq.add(new TermQuery(new Term("field", "foo")), Occur.SHOULD);
- * bq.add(new TermQuery(new Term("field", "bar")), Occur.SHOULD);
- * Query q2 = new ConstantScoreQuery(bq);
- * </pre>
- * <p>When there are few terms, this query executes like a regular disjunction.
- * However, when there are many terms, instead of merging iterators on the fly,
- * it will populate a bit set with matching docs and return a {@link Scorer}
- * over this bit set.
- * <p>NOTE: This query produces scores that are equal to its boost
+ * @deprecated Use {@link org.apache.lucene.search.TermInSetQuery}
  */
-public class TermsQuery extends Query implements Accountable {
-
-  private static final long BASE_RAM_BYTES_USED = RamUsageEstimator.shallowSizeOfInstance(TermsQuery.class);
-  // Same threshold as MultiTermQueryConstantScoreWrapper
-  static final int BOOLEAN_REWRITE_TERM_COUNT_THRESHOLD = 16;
-
-  private final boolean singleField; // whether all terms are from the same field
-  private final PrefixCodedTerms termData;
-  private final int termDataHashCode; // cached hashcode of termData
+@Deprecated
+public class TermsQuery extends TermInSetQuery {
 
   /**
    * Creates a new {@link TermsQuery} from the given collection. It
    * can contain duplicate terms and multiple fields.
    */
   public TermsQuery(Collection<Term> terms) {
-    Term[] sortedTerms = terms.toArray(new Term[terms.size()]);
-    // already sorted if we are a SortedSet with natural order
-    boolean sorted = terms instanceof SortedSet && ((SortedSet<Term>)terms).comparator() == null;
-    if (!sorted) {
-      ArrayUtil.timSort(sortedTerms);
-    }
-    PrefixCodedTerms.Builder builder = new PrefixCodedTerms.Builder();
-    Set<String> fields = new HashSet<>();
-    Term previous = null;
-    for (Term term : sortedTerms) {
-      if (term.equals(previous) == false) {
-        fields.add(term.field());
-        builder.add(term);
-      }
-      previous = term;
-    }
-    singleField = fields.size() == 1;
-    termData = builder.finish();
-    termDataHashCode = termData.hashCode();
+    super(terms);
   }
-  
+
   /**
    * Creates a new {@link TermsQuery} from the given collection for
    * a single field. It can contain duplicate terms.
    */
   public TermsQuery(String field, Collection<BytesRef> terms) {
-    BytesRef[] sortedTerms = terms.toArray(new BytesRef[terms.size()]);
-    // already sorted if we are a SortedSet with natural order
-    boolean sorted = terms instanceof SortedSet && ((SortedSet<BytesRef>)terms).comparator() == null;
-    if (!sorted) {
-      ArrayUtil.timSort(sortedTerms);
-    }
-    PrefixCodedTerms.Builder builder = new PrefixCodedTerms.Builder();
-    BytesRefBuilder previous = null;
-    for (BytesRef term : sortedTerms) {
-      if (previous == null) {
-        previous = new BytesRefBuilder();
-      } else if (previous.get().equals(term)) {
-        continue; // deduplicate
-      }
-      builder.add(field, term);
-      previous.copyBytes(term);
-    }
-    singleField = true;
-    termData = builder.finish();
-    termDataHashCode = termData.hashCode();
+    super(field, terms);
   }
 
   /**
@@ -147,7 +50,7 @@ public class TermsQuery extends Query implements Accountable {
    * a single field.
    */
   public TermsQuery(String field, BytesRef...terms) {
-   this(field, Arrays.asList(terms));
+    this(field, Arrays.asList(terms));
   }
 
   /**
@@ -158,224 +61,5 @@ public class TermsQuery extends Query implements Accountable {
     this(Arrays.asList(terms));
   }
 
-  @Override
-  public Query rewrite(IndexReader reader) throws IOException {
-    final int threshold = Math.min(BOOLEAN_REWRITE_TERM_COUNT_THRESHOLD, BooleanQuery.getMaxClauseCount());
-    if (termData.size() <= threshold) {
-      BooleanQuery.Builder bq = new BooleanQuery.Builder();
-      TermIterator iterator = termData.iterator();
-      for (BytesRef term = iterator.next(); term != null; term = iterator.next()) {
-        bq.add(new TermQuery(new Term(iterator.field(), BytesRef.deepCopyOf(term))), Occur.SHOULD);
-      }
-      return new ConstantScoreQuery(bq.build());
-    }
-    return super.rewrite(reader);
-  }
-
-  @Override
-  public boolean equals(Object other) {
-    return sameClassAs(other) &&
-           equalsTo(getClass().cast(other));
-  }
-
-  private boolean equalsTo(TermsQuery other) {
-    // termData might be heavy to compare so check the hash code first
-    return termDataHashCode == other.termDataHashCode && 
-           termData.equals(other.termData);
-  }
-
-  @Override
-  public int hashCode() {
-    return 31 * classHash() + termDataHashCode;
-  }
-
-  /** Returns the terms wrapped in a PrefixCodedTerms. */
-  public PrefixCodedTerms getTermData() {
-    return termData;
-  }
-
-  @Override
-  public String toString(String defaultField) {
-    StringBuilder builder = new StringBuilder();
-    boolean first = true;
-    TermIterator iterator = termData.iterator();
-    for (BytesRef term = iterator.next(); term != null; term = iterator.next()) {
-      if (!first) {
-        builder.append(' ');
-      }
-      first = false;
-      builder.append(new Term(iterator.field(), term).toString());
-    }
-
-    return builder.toString();
-  }
-
-  @Override
-  public long ramBytesUsed() {
-    return BASE_RAM_BYTES_USED + termData.ramBytesUsed();
-  }
 
-  @Override
-  public Collection<Accountable> getChildResources() {
-    return Collections.emptyList();
-  }
-
-  private static class TermAndState {
-    final String field;
-    final TermsEnum termsEnum;
-    final BytesRef term;
-    final TermState state;
-    final int docFreq;
-    final long totalTermFreq;
-
-    TermAndState(String field, TermsEnum termsEnum) throws IOException {
-      this.field = field;
-      this.termsEnum = termsEnum;
-      this.term = BytesRef.deepCopyOf(termsEnum.term());
-      this.state = termsEnum.termState();
-      this.docFreq = termsEnum.docFreq();
-      this.totalTermFreq = termsEnum.totalTermFreq();
-    }
-  }
-
-  private static class WeightOrDocIdSet {
-    final Weight weight;
-    final DocIdSet set;
-
-    WeightOrDocIdSet(Weight weight) {
-      this.weight = Objects.requireNonNull(weight);
-      this.set = null;
-    }
-
-    WeightOrDocIdSet(DocIdSet bitset) {
-      this.set = bitset;
-      this.weight = null;
-    }
-  }
-
-  @Override
-  public Weight createWeight(IndexSearcher searcher, boolean needsScores, float boost) throws IOException {
-    return new ConstantScoreWeight(this, boost) {
-
-      @Override
-      public void extractTerms(Set<Term> terms) {
-        // no-op
-        // This query is for abuse cases when the number of terms is too high to
-        // run efficiently as a BooleanQuery. So likewise we hide its terms in
-        // order to protect highlighters
-      }
-
-      /**
-       * On the given leaf context, try to either rewrite to a disjunction if
-       * there are few matching terms, or build a bitset containing matching docs.
-       */
-      private WeightOrDocIdSet rewrite(LeafReaderContext context) throws IOException {
-        final LeafReader reader = context.reader();
-
-        // We will first try to collect up to 'threshold' terms into 'matchingTerms'
-        // if there are two many terms, we will fall back to building the 'builder'
-        final int threshold = Math.min(BOOLEAN_REWRITE_TERM_COUNT_THRESHOLD, BooleanQuery.getMaxClauseCount());
-        assert termData.size() > threshold : "Query should have been rewritten";
-        List<TermAndState> matchingTerms = new ArrayList<>(threshold);
-        DocIdSetBuilder builder = null;
-
-        final Fields fields = reader.fields();
-        String lastField = null;
-        Terms terms = null;
-        TermsEnum termsEnum = null;
-        PostingsEnum docs = null;
-        TermIterator iterator = termData.iterator();
-        for (BytesRef term = iterator.next(); term != null; term = iterator.next()) {
-          String field = iterator.field();
-          // comparing references is fine here
-          if (field != lastField) {
-            terms = fields.terms(field);
-            if (terms == null) {
-              termsEnum = null;
-            } else {
-              termsEnum = terms.iterator();
-            }
-            lastField = field;
-          }
-          if (termsEnum != null && termsEnum.seekExact(term)) {
-            if (matchingTerms == null) {
-              docs = termsEnum.postings(docs, PostingsEnum.NONE);
-              builder.add(docs);
-            } else if (matchingTerms.size() < threshold) {
-              matchingTerms.add(new TermAndState(field, termsEnum));
-            } else {
-              assert matchingTerms.size() == threshold;
-              if (singleField) {
-                // common case: all terms are in the same field
-                // use an optimized builder that leverages terms stats to be more efficient
-                builder = new DocIdSetBuilder(reader.maxDoc(), terms);
-              } else {
-                // corner case: different fields
-                // don't make assumptions about the docs we will get
-                builder = new DocIdSetBuilder(reader.maxDoc());
-              }
-              docs = termsEnum.postings(docs, PostingsEnum.NONE);
-              builder.add(docs);
-              for (TermAndState t : matchingTerms) {
-                t.termsEnum.seekExact(t.term, t.state);
-                docs = t.termsEnum.postings(docs, PostingsEnum.NONE);
-                builder.add(docs);
-              }
-              matchingTerms = null;
-            }
-          }
-        }
-        if (matchingTerms != null) {
-          assert builder == null;
-          BooleanQuery.Builder bq = new BooleanQuery.Builder();
-          for (TermAndState t : matchingTerms) {
-            final TermContext termContext = new TermContext(searcher.getTopReaderContext());
-            termContext.register(t.state, context.ord, t.docFreq, t.totalTermFreq);
-            bq.add(new TermQuery(new Term(t.field, t.term), termContext), Occur.SHOULD);
-          }
-          Query q = new ConstantScoreQuery(bq.build());
-          final Weight weight = searcher.rewrite(q).createWeight(searcher, needsScores, score());
-          return new WeightOrDocIdSet(weight);
-        } else {
-          assert builder != null;
-          return new WeightOrDocIdSet(builder.build());
-        }
-      }
-
-      private Scorer scorer(DocIdSet set) throws IOException {
-        if (set == null) {
-          return null;
-        }
-        final DocIdSetIterator disi = set.iterator();
-        if (disi == null) {
-          return null;
-        }
-        return new ConstantScoreScorer(this, score(), disi);
-      }
-
-      @Override
-      public BulkScorer bulkScorer(LeafReaderContext context) throws IOException {
-        final WeightOrDocIdSet weightOrBitSet = rewrite(context);
-        if (weightOrBitSet.weight != null) {
-          return weightOrBitSet.weight.bulkScorer(context);
-        } else {
-          final Scorer scorer = scorer(weightOrBitSet.set);
-          if (scorer == null) {
-            return null;
-          }
-          return new DefaultBulkScorer(scorer);
-        }
-      }
-
-      @Override
-      public Scorer scorer(LeafReaderContext context) throws IOException {
-        final WeightOrDocIdSet weightOrBitSet = rewrite(context);
-        if (weightOrBitSet.weight != null) {
-          return weightOrBitSet.weight.scorer(context);
-        } else {
-          return scorer(weightOrBitSet.set);
-        }
-      }
-    };
-  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/22940f5c/lucene/queries/src/test/org/apache/lucene/queries/TermsQueryTest.java
----------------------------------------------------------------------
diff --git a/lucene/queries/src/test/org/apache/lucene/queries/TermsQueryTest.java b/lucene/queries/src/test/org/apache/lucene/queries/TermsQueryTest.java
deleted file mode 100644
index f8b10ef..0000000
--- a/lucene/queries/src/test/org/apache/lucene/queries/TermsQueryTest.java
+++ /dev/null
@@ -1,339 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.lucene.queries;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-import java.util.concurrent.atomic.AtomicInteger;
-
-import org.apache.lucene.document.Document;
-import org.apache.lucene.document.Field.Store;
-import org.apache.lucene.document.StringField;
-import org.apache.lucene.index.DirectoryReader;
-import org.apache.lucene.index.Fields;
-import org.apache.lucene.index.FilterDirectoryReader;
-import org.apache.lucene.index.FilterLeafReader;
-import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.index.LeafReader;
-import org.apache.lucene.index.RandomIndexWriter;
-import org.apache.lucene.index.Term;
-import org.apache.lucene.index.Terms;
-import org.apache.lucene.index.TermsEnum;
-import org.apache.lucene.search.BooleanClause.Occur;
-import org.apache.lucene.search.BooleanQuery;
-import org.apache.lucene.search.BoostQuery;
-import org.apache.lucene.search.ConstantScoreQuery;
-import org.apache.lucene.search.IndexSearcher;
-import org.apache.lucene.search.Query;
-import org.apache.lucene.search.QueryUtils;
-import org.apache.lucene.search.Sort;
-import org.apache.lucene.search.TermQuery;
-import org.apache.lucene.search.TopDocs;
-import org.apache.lucene.search.UsageTrackingQueryCachingPolicy;
-import org.apache.lucene.store.Directory;
-import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.IOUtils;
-import org.apache.lucene.util.LuceneTestCase;
-import org.apache.lucene.util.RamUsageTester;
-import org.apache.lucene.util.TestUtil;
-
-import com.carrotsearch.randomizedtesting.generators.RandomPicks;
-import com.carrotsearch.randomizedtesting.generators.RandomStrings;
-
-public class TermsQueryTest extends LuceneTestCase {
-
-  public void testDuel() throws IOException {
-    final int iters = atLeast(2);
-    for (int iter = 0; iter < iters; ++iter) {
-      final List<Term> allTerms = new ArrayList<>();
-      final int numTerms = TestUtil.nextInt(random(), 1, 1 << TestUtil.nextInt(random(), 1, 10));
-      for (int i = 0; i < numTerms; ++i) {
-        final String field = usually() ? "f" : "g";
-        final String value = TestUtil.randomAnalysisString(random(), 10, true);
-        allTerms.add(new Term(field, value));
-      }
-      Directory dir = newDirectory();
-      RandomIndexWriter iw = new RandomIndexWriter(random(), dir);
-      final int numDocs = atLeast(100);
-      for (int i = 0; i < numDocs; ++i) {
-        Document doc = new Document();
-        final Term term = allTerms.get(random().nextInt(allTerms.size()));
-        doc.add(new StringField(term.field(), term.text(), Store.NO));
-        iw.addDocument(doc);
-      }
-      if (numTerms > 1 && random().nextBoolean()) {
-        iw.deleteDocuments(new TermQuery(allTerms.get(0)));
-      }
-      iw.commit();
-      final IndexReader reader = iw.getReader();
-      final IndexSearcher searcher = newSearcher(reader);
-      iw.close();
-
-      if (reader.numDocs() == 0) {
-        // may occasionally happen if all documents got the same term
-        IOUtils.close(reader, dir);
-        continue;
-      }
-
-      for (int i = 0; i < 100; ++i) {
-        final float boost = random().nextFloat() * 10;
-        final int numQueryTerms = TestUtil.nextInt(random(), 1, 1 << TestUtil.nextInt(random(), 1, 8));
-        List<Term> queryTerms = new ArrayList<>();
-        for (int j = 0; j < numQueryTerms; ++j) {
-          queryTerms.add(allTerms.get(random().nextInt(allTerms.size())));
-        }
-        final BooleanQuery.Builder bq = new BooleanQuery.Builder();
-        for (Term t : queryTerms) {
-          bq.add(new TermQuery(t), Occur.SHOULD);
-        }
-        final Query q1 = new ConstantScoreQuery(bq.build());
-        final Query q2 = new TermsQuery(queryTerms);
-        assertSameMatches(searcher, new BoostQuery(q1, boost), new BoostQuery(q2, boost), true);
-      }
-
-      reader.close();
-      dir.close();
-    }
-  }
-
-  private void assertSameMatches(IndexSearcher searcher, Query q1, Query q2, boolean scores) throws IOException {
-    final int maxDoc = searcher.getIndexReader().maxDoc();
-    final TopDocs td1 = searcher.search(q1, maxDoc, scores ? Sort.RELEVANCE : Sort.INDEXORDER);
-    final TopDocs td2 = searcher.search(q2, maxDoc, scores ? Sort.RELEVANCE : Sort.INDEXORDER);
-    assertEquals(td1.totalHits, td2.totalHits);
-    for (int i = 0; i < td1.scoreDocs.length; ++i) {
-      assertEquals(td1.scoreDocs[i].doc, td2.scoreDocs[i].doc);
-      if (scores) {
-        assertEquals(td1.scoreDocs[i].score, td2.scoreDocs[i].score, 10e-7);
-      }
-    }
-  }
-
-  private TermsQuery termsQuery(boolean singleField, Term...terms) {
-    return termsQuery(singleField, Arrays.asList(terms));
-  }
-
-  private TermsQuery termsQuery(boolean singleField, Collection<Term> termList) {
-    if (!singleField) {
-      return new TermsQuery(new ArrayList<>(termList));
-    }
-    final TermsQuery filter;
-    List<BytesRef> bytes = new ArrayList<>();
-    String field = null;
-    for (Term term : termList) {
-        bytes.add(term.bytes());
-        if (field != null) {
-          assertEquals(term.field(), field);
-        }
-        field = term.field();
-    }
-    assertNotNull(field);
-    filter = new TermsQuery(field, bytes);
-    return filter;
-  }
-
-  public void testHashCodeAndEquals() {
-    int num = atLeast(100);
-    final boolean singleField = random().nextBoolean();
-    List<Term> terms = new ArrayList<>();
-    Set<Term> uniqueTerms = new HashSet<>();
-    for (int i = 0; i < num; i++) {
-      String field = "field" + (singleField ? "1" : random().nextInt(100));
-      String string = TestUtil.randomRealisticUnicodeString(random());
-      terms.add(new Term(field, string));
-      uniqueTerms.add(new Term(field, string));
-      TermsQuery left = termsQuery(singleField ? random().nextBoolean() : false, uniqueTerms);
-      Collections.shuffle(terms, random());
-      TermsQuery right = termsQuery(singleField ? random().nextBoolean() : false, terms);
-      assertEquals(right, left);
-      assertEquals(right.hashCode(), left.hashCode());
-      if (uniqueTerms.size() > 1) {
-        List<Term> asList = new ArrayList<>(uniqueTerms);
-        asList.remove(0);
-        TermsQuery notEqual = termsQuery(singleField ? random().nextBoolean() : false, asList);
-        assertFalse(left.equals(notEqual));
-        assertFalse(right.equals(notEqual));
-      }
-    }
-
-    TermsQuery tq1 = new TermsQuery(new Term("thing", "apple"));
-    TermsQuery tq2 = new TermsQuery(new Term("thing", "orange"));
-    assertFalse(tq1.hashCode() == tq2.hashCode());
-
-    // different fields with the same term should have differing hashcodes
-    tq1 = new TermsQuery(new Term("thing1", "apple"));
-    tq2 = new TermsQuery(new Term("thing2", "apple"));
-    assertFalse(tq1.hashCode() == tq2.hashCode());
-  }
-
-  public void testSingleFieldEquals() {
-    // Two terms with the same hash code
-    assertEquals("AaAaBB".hashCode(), "BBBBBB".hashCode());
-    TermsQuery left = termsQuery(true, new Term("id", "AaAaAa"), new Term("id", "AaAaBB"));
-    TermsQuery right = termsQuery(true, new Term("id", "AaAaAa"), new Term("id", "BBBBBB"));
-    assertFalse(left.equals(right));
-  }
-
-  public void testToString() {
-    TermsQuery termsQuery = new TermsQuery(new Term("field1", "a"),
-                                              new Term("field1", "b"),
-                                              new Term("field1", "c"));
-    assertEquals("field1:a field1:b field1:c", termsQuery.toString());
-  }
-
-  public void testDedup() {
-    Query query1 = new TermsQuery(new Term("foo", "bar"));
-    Query query2 = new TermsQuery(new Term("foo", "bar"), new Term("foo", "bar"));
-    QueryUtils.checkEqual(query1, query2);
-  }
-
-  public void testOrderDoesNotMatter() {
-    // order of terms if different
-    Query query1 = new TermsQuery(new Term("foo", "bar"), new Term("foo", "baz"));
-    Query query2 = new TermsQuery(new Term("foo", "baz"), new Term("foo", "bar"));
-    QueryUtils.checkEqual(query1, query2);
-
-    // order of fields is different
-    query1 = new TermsQuery(new Term("foo", "bar"), new Term("bar", "bar"));
-    query2 = new TermsQuery(new Term("bar", "bar"), new Term("foo", "bar"));
-    QueryUtils.checkEqual(query1, query2);
-  }
-
-  public void testRamBytesUsed() {
-    List<Term> terms = new ArrayList<>();
-    final int numTerms = 1000 + random().nextInt(1000);
-    for (int i = 0; i < numTerms; ++i) {
-      terms.add(new Term("f", RandomStrings.randomUnicodeOfLength(random(), 10)));
-    }
-    TermsQuery query = new TermsQuery(terms);
-    final long actualRamBytesUsed = RamUsageTester.sizeOf(query);
-    final long expectedRamBytesUsed = query.ramBytesUsed();
-    // error margin within 5%
-    assertEquals(actualRamBytesUsed, expectedRamBytesUsed, actualRamBytesUsed / 20);
-  }
-
-  private static class TermsCountingDirectoryReaderWrapper extends FilterDirectoryReader {
-
-    private final AtomicInteger counter;
-    
-    public TermsCountingDirectoryReaderWrapper(DirectoryReader in, AtomicInteger counter) throws IOException {
-      super(in, new TermsCountingSubReaderWrapper(counter));
-      this.counter = counter;
-    }
-
-    private static class TermsCountingSubReaderWrapper extends SubReaderWrapper {
-      private final AtomicInteger counter;
-
-      public TermsCountingSubReaderWrapper(AtomicInteger counter) {
-        this.counter = counter;
-      }
-
-      @Override
-      public LeafReader wrap(LeafReader reader) {
-        return new TermsCountingLeafReaderWrapper(reader, counter);
-      }
-    }
-
-    private static class TermsCountingLeafReaderWrapper extends FilterLeafReader {
-
-      private final AtomicInteger counter;
-
-      public TermsCountingLeafReaderWrapper(LeafReader in, AtomicInteger counter) {
-        super(in);
-        this.counter = counter;
-      }
-
-      @Override
-      public Fields fields() throws IOException {
-        return new FilterFields(in.fields()) {
-          @Override
-          public Terms terms(String field) throws IOException {
-            final Terms in = this.in.terms(field);
-            if (in == null) {
-              return null;
-            }
-            return new FilterTerms(in) {
-              @Override
-              public TermsEnum iterator() throws IOException {
-                counter.incrementAndGet();
-                return super.iterator();
-              }
-            };
-          }
-        };
-      }
-      
-    }
-
-    @Override
-    protected DirectoryReader doWrapDirectoryReader(DirectoryReader in) throws IOException {
-      return new TermsCountingDirectoryReaderWrapper(in, counter);
-    }
-
-  }
-
-  public void testPullOneTermsEnumPerField() throws Exception {
-    Directory dir = newDirectory();
-    RandomIndexWriter w = new RandomIndexWriter(random(), dir);
-    Document doc = new Document();
-    doc.add(new StringField("foo", "1", Store.NO));
-    doc.add(new StringField("bar", "2", Store.NO));
-    doc.add(new StringField("baz", "3", Store.NO));
-    w.addDocument(doc);
-    DirectoryReader reader = w.getReader();
-    w.close();
-    final AtomicInteger counter = new AtomicInteger();
-    DirectoryReader wrapped = new TermsCountingDirectoryReaderWrapper(reader, counter);
-
-    final List<Term> terms = new ArrayList<>();
-    final Set<String> fields = new HashSet<>();
-    // enough terms to avoid the rewrite
-    final int numTerms = TestUtil.nextInt(random(), TermsQuery.BOOLEAN_REWRITE_TERM_COUNT_THRESHOLD + 1, 100);
-    for (int i = 0; i < numTerms; ++i) {
-      final String field = RandomPicks.randomFrom(random(), new String[] {"foo", "bar", "baz"});
-      final BytesRef term = new BytesRef(RandomStrings.randomUnicodeOfCodepointLength(random(), 10));
-      fields.add(field);
-      terms.add(new Term(field, term));
-    }
-
-    new IndexSearcher(wrapped).count(new TermsQuery(terms));
-    assertEquals(fields.size(), counter.get());
-    wrapped.close();
-    dir.close();
-  }
-  
-  public void testBinaryToString() {
-    TermsQuery query = new TermsQuery(new Term("field", new BytesRef(new byte[] { (byte) 0xff, (byte) 0xfe })));
-    assertEquals("field:[ff fe]", query.toString());
-  }
-
-  public void testIsConsideredCostlyByQueryCache() throws IOException {
-    TermsQuery query = new TermsQuery(new Term("foo", "bar"), new Term("foo", "baz"));
-    UsageTrackingQueryCachingPolicy policy = new UsageTrackingQueryCachingPolicy();
-    assertFalse(policy.shouldCache(query));
-    policy.onUse(query);
-    policy.onUse(query);
-    // cached after two uses
-    assertTrue(policy.shouldCache(query));
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/22940f5c/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/TermQueryPrefixTreeStrategy.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/TermQueryPrefixTreeStrategy.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/TermQueryPrefixTreeStrategy.java
index 0273466..4e37f5d 100644
--- a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/TermQueryPrefixTreeStrategy.java
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/TermQueryPrefixTreeStrategy.java
@@ -19,10 +19,8 @@ package org.apache.lucene.spatial.prefix;
 import java.util.ArrayList;
 import java.util.List;
 
-import org.locationtech.spatial4j.shape.Point;
-import org.locationtech.spatial4j.shape.Shape;
-import org.apache.lucene.queries.TermsQuery;
 import org.apache.lucene.search.Query;
+import org.apache.lucene.search.TermInSetQuery;
 import org.apache.lucene.spatial.prefix.tree.Cell;
 import org.apache.lucene.spatial.prefix.tree.CellIterator;
 import org.apache.lucene.spatial.prefix.tree.SpatialPrefixTree;
@@ -31,10 +29,12 @@ import org.apache.lucene.spatial.query.SpatialOperation;
 import org.apache.lucene.spatial.query.UnsupportedSpatialOperation;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefBuilder;
+import org.locationtech.spatial4j.shape.Point;
+import org.locationtech.spatial4j.shape.Shape;
 
 /**
  * A basic implementation of {@link PrefixTreeStrategy} using a large
- * {@link TermsQuery} of all the cells from
+ * {@link TermInSetQuery} of all the cells from
  * {@link SpatialPrefixTree#getTreeCellIterator(org.locationtech.spatial4j.shape.Shape, int)}.
  * It only supports the search of indexed Point shapes.
  * <p>
@@ -105,7 +105,7 @@ public class TermQueryPrefixTreeStrategy extends PrefixTreeStrategy {
     }
     //unfortunately TermsQuery will needlessly sort & dedupe
     //TODO an automatonQuery might be faster?
-    return new TermsQuery(getFieldName(), terms);
+    return new TermInSetQuery(getFieldName(), terms);
   }
 
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/22940f5c/lucene/spatial-extras/src/test/org/apache/lucene/spatial/prefix/NumberRangeFacetsTest.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/test/org/apache/lucene/spatial/prefix/NumberRangeFacetsTest.java b/lucene/spatial-extras/src/test/org/apache/lucene/spatial/prefix/NumberRangeFacetsTest.java
index 514c18e..bb26a2e 100644
--- a/lucene/spatial-extras/src/test/org/apache/lucene/spatial/prefix/NumberRangeFacetsTest.java
+++ b/lucene/spatial-extras/src/test/org/apache/lucene/spatial/prefix/NumberRangeFacetsTest.java
@@ -23,12 +23,11 @@ import java.util.Collections;
 import java.util.List;
 
 import com.carrotsearch.randomizedtesting.annotations.Repeat;
-import org.locationtech.spatial4j.shape.Shape;
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.Term;
-import org.apache.lucene.queries.TermsQuery;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.SimpleCollector;
+import org.apache.lucene.search.TermInSetQuery;
 import org.apache.lucene.spatial.StrategyTestCase;
 import org.apache.lucene.spatial.prefix.NumberRangePrefixTreeStrategy.Facets;
 import org.apache.lucene.spatial.prefix.tree.Cell;
@@ -40,6 +39,7 @@ import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.FixedBitSet;
 import org.junit.Before;
 import org.junit.Test;
+import org.locationtech.spatial4j.shape.Shape;
 
 import static com.carrotsearch.randomizedtesting.RandomizedTest.randomInt;
 import static com.carrotsearch.randomizedtesting.RandomizedTest.randomIntBetween;
@@ -132,7 +132,7 @@ public class NumberRangeFacetsTest extends StrategyTestCase {
             terms.add(new Term("id", acceptDocId.toString()));
           }
 
-          topAcceptDocs = searchForDocBits(new TermsQuery(terms));
+          topAcceptDocs = searchForDocBits(new TermInSetQuery(terms));
         }
       }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/22940f5c/solr/core/src/java/org/apache/solr/handler/component/ExpandComponent.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/component/ExpandComponent.java b/solr/core/src/java/org/apache/solr/handler/component/ExpandComponent.java
index 366c4a9..a5cbee2 100644
--- a/solr/core/src/java/org/apache/solr/handler/component/ExpandComponent.java
+++ b/solr/core/src/java/org/apache/solr/handler/component/ExpandComponent.java
@@ -26,6 +26,15 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 
+import com.carrotsearch.hppc.IntHashSet;
+import com.carrotsearch.hppc.IntObjectHashMap;
+import com.carrotsearch.hppc.LongHashSet;
+import com.carrotsearch.hppc.LongObjectHashMap;
+import com.carrotsearch.hppc.LongObjectMap;
+import com.carrotsearch.hppc.cursors.IntObjectCursor;
+import com.carrotsearch.hppc.cursors.LongCursor;
+import com.carrotsearch.hppc.cursors.LongObjectCursor;
+import com.carrotsearch.hppc.cursors.ObjectCursor;
 import org.apache.lucene.index.DocValues;
 import org.apache.lucene.index.DocValuesType;
 import org.apache.lucene.index.FieldInfo;
@@ -36,7 +45,6 @@ import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.MultiDocValues;
 import org.apache.lucene.index.NumericDocValues;
 import org.apache.lucene.index.SortedDocValues;
-import org.apache.lucene.queries.TermsQuery;
 import org.apache.lucene.search.BooleanClause.Occur;
 import org.apache.lucene.search.BooleanQuery;
 import org.apache.lucene.search.Collector;
@@ -46,6 +54,7 @@ import org.apache.lucene.search.Query;
 import org.apache.lucene.search.ScoreDoc;
 import org.apache.lucene.search.Scorer;
 import org.apache.lucene.search.Sort;
+import org.apache.lucene.search.TermInSetQuery;
 import org.apache.lucene.search.TopDocs;
 import org.apache.lucene.search.TopDocsCollector;
 import org.apache.lucene.search.TopFieldCollector;
@@ -83,16 +92,6 @@ import org.apache.solr.uninverting.UninvertingReader;
 import org.apache.solr.util.plugin.PluginInfoInitialized;
 import org.apache.solr.util.plugin.SolrCoreAware;
 
-import com.carrotsearch.hppc.IntHashSet;
-import com.carrotsearch.hppc.IntObjectHashMap;
-import com.carrotsearch.hppc.LongHashSet;
-import com.carrotsearch.hppc.LongObjectHashMap;
-import com.carrotsearch.hppc.LongObjectMap;
-import com.carrotsearch.hppc.cursors.IntObjectCursor;
-import com.carrotsearch.hppc.cursors.LongCursor;
-import com.carrotsearch.hppc.cursors.LongObjectCursor;
-import com.carrotsearch.hppc.cursors.ObjectCursor;
-
 /**
  * The ExpandComponent is designed to work with the CollapsingPostFilter.
  * The CollapsingPostFilter collapses a result set on a field.
@@ -720,7 +719,7 @@ public class ExpandComponent extends SearchComponent implements PluginInfoInitia
       bytesRefs[++index] = term.toBytesRef();
     }
 
-    return new SolrConstantScoreQuery(new QueryWrapperFilter(new TermsQuery(fname, bytesRefs)));
+    return new SolrConstantScoreQuery(new QueryWrapperFilter(new TermInSetQuery(fname, bytesRefs)));
   }
 
   private Query getGroupQuery(String fname,
@@ -733,7 +732,7 @@ public class ExpandComponent extends SearchComponent implements PluginInfoInitia
       IntObjectCursor<BytesRef> cursor = it.next();
       bytesRefs[++index] = cursor.value;
     }
-    return new SolrConstantScoreQuery(new QueryWrapperFilter(new TermsQuery(fname, bytesRefs)));
+    return new SolrConstantScoreQuery(new QueryWrapperFilter(new TermInSetQuery(fname, bytesRefs)));
   }
 
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/22940f5c/solr/core/src/java/org/apache/solr/schema/FieldType.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/schema/FieldType.java b/solr/core/src/java/org/apache/solr/schema/FieldType.java
index b67f88d..a5c898a 100644
--- a/solr/core/src/java/org/apache/solr/schema/FieldType.java
+++ b/solr/core/src/java/org/apache/solr/schema/FieldType.java
@@ -39,7 +39,6 @@ import org.apache.lucene.document.Field;
 import org.apache.lucene.index.IndexableField;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.legacy.LegacyNumericType;
-import org.apache.lucene.queries.TermsQuery;
 import org.apache.lucene.queries.function.ValueSource;
 import org.apache.lucene.search.BooleanClause;
 import org.apache.lucene.search.BooleanQuery;
@@ -51,6 +50,7 @@ import org.apache.lucene.search.Query;
 import org.apache.lucene.search.SortField;
 import org.apache.lucene.search.SortedNumericSelector;
 import org.apache.lucene.search.SortedSetSelector;
+import org.apache.lucene.search.TermInSetQuery;
 import org.apache.lucene.search.TermQuery;
 import org.apache.lucene.search.similarities.Similarity;
 import org.apache.lucene.util.BytesRef;
@@ -765,7 +765,7 @@ public abstract class FieldType extends FieldProperties {
       readableToIndexed(externalVal, br);
       lst.add( br.toBytesRef() );
     }
-    return new TermsQuery(field.getName() , lst);
+    return new TermInSetQuery(field.getName() , lst);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/22940f5c/solr/core/src/java/org/apache/solr/search/TermsQParserPlugin.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/TermsQParserPlugin.java b/solr/core/src/java/org/apache/solr/search/TermsQParserPlugin.java
index d53dcbf..3a60149 100644
--- a/solr/core/src/java/org/apache/solr/search/TermsQParserPlugin.java
+++ b/solr/core/src/java/org/apache/solr/search/TermsQParserPlugin.java
@@ -20,13 +20,13 @@ import java.util.Arrays;
 import java.util.regex.Pattern;
 
 import org.apache.lucene.index.Term;
-import org.apache.lucene.queries.TermsQuery;
 import org.apache.lucene.search.AutomatonQuery;
 import org.apache.lucene.search.BooleanClause;
 import org.apache.lucene.search.BooleanQuery;
 import org.apache.lucene.search.DocValuesTermsQuery;
 import org.apache.lucene.search.MatchNoDocsQuery;
 import org.apache.lucene.search.Query;
+import org.apache.lucene.search.TermInSetQuery;
 import org.apache.lucene.search.TermQuery;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefBuilder;
@@ -60,7 +60,7 @@ public class TermsQParserPlugin extends QParserPlugin {
     termsFilter {
       @Override
       Filter makeFilter(String fname, BytesRef[] bytesRefs) {
-        return new QueryWrapperFilter(new TermsQuery(fname, bytesRefs));
+        return new QueryWrapperFilter(new TermInSetQuery(fname, bytesRefs));
       }
     },
     booleanQuery {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/22940f5c/solr/core/src/java/org/apache/solr/search/join/GraphQuery.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/join/GraphQuery.java b/solr/core/src/java/org/apache/solr/search/join/GraphQuery.java
index 8cfcf79..3f762e3 100644
--- a/solr/core/src/java/org/apache/solr/search/join/GraphQuery.java
+++ b/solr/core/src/java/org/apache/solr/search/join/GraphQuery.java
@@ -25,7 +25,6 @@ import java.util.TreeSet;
 
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.Term;
-import org.apache.lucene.queries.TermsQuery;
 import org.apache.lucene.search.AutomatonQuery;
 import org.apache.lucene.search.BooleanClause.Occur;
 import org.apache.lucene.search.BooleanQuery;
@@ -35,6 +34,7 @@ import org.apache.lucene.search.Explanation;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.Scorer;
+import org.apache.lucene.search.TermInSetQuery;
 import org.apache.lucene.search.Weight;
 import org.apache.lucene.search.WildcardQuery;
 import org.apache.lucene.util.BytesRef;
@@ -281,7 +281,7 @@ public class GraphQuery extends Query {
             collectorTerms.get(i, ref);
             termList.add(ref);
           }
-          q = new TermsQuery(fromField, termList);
+          q = new TermInSetQuery(fromField, termList);
         }
         
         // If there is a filter to be used while crawling the graph, add that.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/22940f5c/solr/core/src/test/org/apache/solr/search/TestSolrQueryParser.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/search/TestSolrQueryParser.java b/solr/core/src/test/org/apache/solr/search/TestSolrQueryParser.java
index d3e6a7f..76b441b 100644
--- a/solr/core/src/test/org/apache/solr/search/TestSolrQueryParser.java
+++ b/solr/core/src/test/org/apache/solr/search/TestSolrQueryParser.java
@@ -19,12 +19,12 @@ package org.apache.solr.search;
 import java.util.Locale;
 import java.util.Random;
 
-import org.apache.lucene.queries.TermsQuery;
 import org.apache.lucene.search.BooleanClause;
 import org.apache.lucene.search.BooleanQuery;
 import org.apache.lucene.search.BoostQuery;
 import org.apache.lucene.search.ConstantScoreQuery;
 import org.apache.lucene.search.Query;
+import org.apache.lucene.search.TermInSetQuery;
 import org.apache.lucene.search.TermQuery;
 import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.core.SolrInfoMBean;
@@ -224,13 +224,13 @@ public class TestSolrQueryParser extends SolrTestCaseJ4 {
     qParser = QParser.getParser("foo_s:(a b c d e f g h i j k l m n o p q r s t u v w x y z)", req);
     qParser.setIsFilter(true); // this may change in the future
     q = qParser.getQuery();
-    assertEquals(26, ((TermsQuery)q).getTermData().size());
+    assertEquals(26, ((TermInSetQuery)q).getTermData().size());
 
     // large numeric filter query should use TermsQuery (for trie fields)
     qParser = QParser.getParser("foo_i:(1 2 3 4 5 6 7 8 9 10 20 19 18 17 16 15 14 13 12 11)", req);
     qParser.setIsFilter(true); // this may change in the future
     q = qParser.getQuery();
-    assertEquals(20, ((TermsQuery)q).getTermData().size());
+    assertEquals(20, ((TermInSetQuery)q).getTermData().size());
 
     // a filter() clause inside a relevancy query should be able to use a TermsQuery
     qParser = QParser.getParser("foo_s:aaa filter(foo_s:(a b c d e f g h i j k l m n o p q r s t u v w x y z))", req);
@@ -245,7 +245,7 @@ public class TestSolrQueryParser extends SolrTestCaseJ4 {
       qq = ((FilterQuery)qq).getQuery();
     }
 
-    assertEquals(26, ((TermsQuery)qq).getTermData().size());
+    assertEquals(26, ((TermInSetQuery)qq).getTermData().size());
 
     // test mixed boolean query, including quotes (which shouldn't matter)
     qParser = QParser.getParser("foo_s:(a +aaa b -bbb c d e f bar_s:(qqq www) g h i j k l m n o p q r s t u v w x y z)", req);
@@ -255,9 +255,9 @@ public class TestSolrQueryParser extends SolrTestCaseJ4 {
     qq = null;
     for (BooleanClause clause : ((BooleanQuery)q).clauses()) {
       qq = clause.getQuery();
-      if (qq instanceof TermsQuery) break;
+      if (qq instanceof TermInSetQuery) break;
     }
-    assertEquals(26, ((TermsQuery)qq).getTermData().size());
+    assertEquals(26, ((TermInSetQuery)qq).getTermData().size());
 
     req.close();
   }


[2/2] lucene-solr:branch_6x: LUCENE-7624: Move TermsQuery into core as TermInSetQuery

Posted by ro...@apache.org.
LUCENE-7624: Move TermsQuery into core as TermInSetQuery


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

Branch: refs/heads/branch_6x
Commit: 8511f9e6991679f71e7a82c6ef9cf1b774d090aa
Parents: 31b8a44
Author: Alan Woodward <ro...@apache.org>
Authored: Mon Jan 9 14:01:33 2017 +0000
Committer: Alan Woodward <ro...@apache.org>
Committed: Mon Jan 9 14:32:17 2017 +0000

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |   3 +
 .../apache/lucene/search/TermInSetQuery.java    | 370 +++++++++++++++++++
 .../search/UsageTrackingQueryCachingPolicy.java |   5 +-
 .../lucene/search/TermInSetQueryTest.java       | 328 ++++++++++++++++
 .../apache/lucene/facet/MultiFacetQuery.java    |  12 +-
 .../org/apache/lucene/queries/TermsQuery.java   | 334 +----------------
 .../apache/lucene/queries/TermsQueryTest.java   | 339 -----------------
 .../prefix/TermQueryPrefixTreeStrategy.java     |  10 +-
 .../solr/handler/component/ExpandComponent.java |  25 +-
 .../java/org/apache/solr/schema/FieldType.java  |   4 +-
 .../apache/solr/search/TermsQParserPlugin.java  |   4 +-
 .../org/apache/solr/search/join/GraphQuery.java |   4 +-
 .../apache/solr/search/TestSolrQueryParser.java |  12 +-
 13 files changed, 746 insertions(+), 704 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8511f9e6/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 362e94f..ded1119 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -30,6 +30,9 @@ API Changes
 * LUCENE-7611: DocumentValueSourceDictionary now takes a LongValuesSource
   as a parameter, and the ValueSource equivalent is deprecated (Alan Woodward)
 
+* LUCENE-7624: TermsQuery has been renamed as TermInSetQuery and moved to core.
+  (Alan Woodward)
+
 New features
 
 * LUCENE-5867: Added BooleanSimilarity. (Robert Muir, Adrien Grand)

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8511f9e6/lucene/core/src/java/org/apache/lucene/search/TermInSetQuery.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/TermInSetQuery.java b/lucene/core/src/java/org/apache/lucene/search/TermInSetQuery.java
new file mode 100644
index 0000000..978bd2d
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/search/TermInSetQuery.java
@@ -0,0 +1,370 @@
+/*
+ * 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;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Objects;
+import java.util.Set;
+import java.util.SortedSet;
+
+import org.apache.lucene.index.Fields;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.LeafReader;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.PostingsEnum;
+import org.apache.lucene.index.PrefixCodedTerms;
+import org.apache.lucene.index.PrefixCodedTerms.TermIterator;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.index.TermContext;
+import org.apache.lucene.index.TermState;
+import org.apache.lucene.index.Terms;
+import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.search.BooleanClause.Occur;
+import org.apache.lucene.util.Accountable;
+import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefBuilder;
+import org.apache.lucene.util.DocIdSetBuilder;
+import org.apache.lucene.util.RamUsageEstimator;
+
+/**
+ * Specialization for a disjunction over many terms that behaves like a
+ * {@link ConstantScoreQuery} over a {@link BooleanQuery} containing only
+ * {@link org.apache.lucene.search.BooleanClause.Occur#SHOULD} clauses.
+ * <p>For instance in the following example, both @{code q1} and {@code q2}
+ * would yield the same scores:
+ * <pre class="prettyprint">
+ * Query q1 = new TermInSetQuery(new Term("field", "foo"), new Term("field", "bar"));
+ *
+ * BooleanQuery bq = new BooleanQuery();
+ * bq.add(new TermQuery(new Term("field", "foo")), Occur.SHOULD);
+ * bq.add(new TermQuery(new Term("field", "bar")), Occur.SHOULD);
+ * Query q2 = new ConstantScoreQuery(bq);
+ * </pre>
+ * <p>When there are few terms, this query executes like a regular disjunction.
+ * However, when there are many terms, instead of merging iterators on the fly,
+ * it will populate a bit set with matching docs and return a {@link Scorer}
+ * over this bit set.
+ * <p>NOTE: This query produces scores that are equal to its boost
+ */
+public class TermInSetQuery extends Query implements Accountable {
+
+  private static final long BASE_RAM_BYTES_USED = RamUsageEstimator.shallowSizeOfInstance(TermInSetQuery.class);
+  // Same threshold as MultiTermQueryConstantScoreWrapper
+  static final int BOOLEAN_REWRITE_TERM_COUNT_THRESHOLD = 16;
+
+  private final boolean singleField; // whether all terms are from the same field
+  private final PrefixCodedTerms termData;
+  private final int termDataHashCode; // cached hashcode of termData
+
+  /**
+   * Creates a new {@link TermInSetQuery} from the given collection. It
+   * can contain duplicate terms and multiple fields.
+   */
+  public TermInSetQuery(Collection<Term> terms) {
+    Term[] sortedTerms = terms.toArray(new Term[terms.size()]);
+    // already sorted if we are a SortedSet with natural order
+    boolean sorted = terms instanceof SortedSet && ((SortedSet<Term>)terms).comparator() == null;
+    if (!sorted) {
+      ArrayUtil.timSort(sortedTerms);
+    }
+    PrefixCodedTerms.Builder builder = new PrefixCodedTerms.Builder();
+    Set<String> fields = new HashSet<>();
+    Term previous = null;
+    for (Term term : sortedTerms) {
+      if (term.equals(previous) == false) {
+        fields.add(term.field());
+        builder.add(term);
+      }
+      previous = term;
+    }
+    singleField = fields.size() == 1;
+    termData = builder.finish();
+    termDataHashCode = termData.hashCode();
+  }
+
+  /**
+   * Creates a new {@link TermInSetQuery} from the given collection for
+   * a single field. It can contain duplicate terms.
+   */
+  public TermInSetQuery(String field, Collection<BytesRef> terms) {
+    BytesRef[] sortedTerms = terms.toArray(new BytesRef[terms.size()]);
+    // already sorted if we are a SortedSet with natural order
+    boolean sorted = terms instanceof SortedSet && ((SortedSet<BytesRef>)terms).comparator() == null;
+    if (!sorted) {
+      ArrayUtil.timSort(sortedTerms);
+    }
+    PrefixCodedTerms.Builder builder = new PrefixCodedTerms.Builder();
+    BytesRefBuilder previous = null;
+    for (BytesRef term : sortedTerms) {
+      if (previous == null) {
+        previous = new BytesRefBuilder();
+      } else if (previous.get().equals(term)) {
+        continue; // deduplicate
+      }
+      builder.add(field, term);
+      previous.copyBytes(term);
+    }
+    singleField = true;
+    termData = builder.finish();
+    termDataHashCode = termData.hashCode();
+  }
+
+  /**
+   * Creates a new {@link TermInSetQuery} from the given {@link BytesRef} array for
+   * a single field.
+   */
+  public TermInSetQuery(String field, BytesRef...terms) {
+    this(field, Arrays.asList(terms));
+  }
+
+  /**
+   * Creates a new {@link TermInSetQuery} from the given array. The array can
+   * contain duplicate terms and multiple fields.
+   */
+  public TermInSetQuery(final Term... terms) {
+    this(Arrays.asList(terms));
+  }
+
+  @Override
+  public Query rewrite(IndexReader reader) throws IOException {
+    final int threshold = Math.min(BOOLEAN_REWRITE_TERM_COUNT_THRESHOLD, BooleanQuery.getMaxClauseCount());
+    if (termData.size() <= threshold) {
+      BooleanQuery.Builder bq = new BooleanQuery.Builder();
+      TermIterator iterator = termData.iterator();
+      for (BytesRef term = iterator.next(); term != null; term = iterator.next()) {
+        bq.add(new TermQuery(new Term(iterator.field(), BytesRef.deepCopyOf(term))), Occur.SHOULD);
+      }
+      return new ConstantScoreQuery(bq.build());
+    }
+    return super.rewrite(reader);
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    return sameClassAs(other) &&
+        equalsTo(getClass().cast(other));
+  }
+
+  private boolean equalsTo(TermInSetQuery other) {
+    // termData might be heavy to compare so check the hash code first
+    return termDataHashCode == other.termDataHashCode &&
+        termData.equals(other.termData);
+  }
+
+  @Override
+  public int hashCode() {
+    return 31 * classHash() + termDataHashCode;
+  }
+
+  /** Returns the terms wrapped in a PrefixCodedTerms. */
+  public PrefixCodedTerms getTermData() {
+    return termData;
+  }
+
+  @Override
+  public String toString(String defaultField) {
+    StringBuilder builder = new StringBuilder();
+    boolean first = true;
+    TermIterator iterator = termData.iterator();
+    for (BytesRef term = iterator.next(); term != null; term = iterator.next()) {
+      if (!first) {
+        builder.append(' ');
+      }
+      first = false;
+      builder.append(new Term(iterator.field(), term).toString());
+    }
+
+    return builder.toString();
+  }
+
+  @Override
+  public long ramBytesUsed() {
+    return BASE_RAM_BYTES_USED + termData.ramBytesUsed();
+  }
+
+  @Override
+  public Collection<Accountable> getChildResources() {
+    return Collections.emptyList();
+  }
+
+  private static class TermAndState {
+    final String field;
+    final TermsEnum termsEnum;
+    final BytesRef term;
+    final TermState state;
+    final int docFreq;
+    final long totalTermFreq;
+
+    TermAndState(String field, TermsEnum termsEnum) throws IOException {
+      this.field = field;
+      this.termsEnum = termsEnum;
+      this.term = BytesRef.deepCopyOf(termsEnum.term());
+      this.state = termsEnum.termState();
+      this.docFreq = termsEnum.docFreq();
+      this.totalTermFreq = termsEnum.totalTermFreq();
+    }
+  }
+
+  private static class WeightOrDocIdSet {
+    final Weight weight;
+    final DocIdSet set;
+
+    WeightOrDocIdSet(Weight weight) {
+      this.weight = Objects.requireNonNull(weight);
+      this.set = null;
+    }
+
+    WeightOrDocIdSet(DocIdSet bitset) {
+      this.set = bitset;
+      this.weight = null;
+    }
+  }
+
+  @Override
+  public Weight createWeight(IndexSearcher searcher, boolean needsScores) throws IOException {
+    return new ConstantScoreWeight(this) {
+
+      @Override
+      public void extractTerms(Set<Term> terms) {
+        // no-op
+        // This query is for abuse cases when the number of terms is too high to
+        // run efficiently as a BooleanQuery. So likewise we hide its terms in
+        // order to protect highlighters
+      }
+
+      /**
+       * On the given leaf context, try to either rewrite to a disjunction if
+       * there are few matching terms, or build a bitset containing matching docs.
+       */
+      private WeightOrDocIdSet rewrite(LeafReaderContext context) throws IOException {
+        final LeafReader reader = context.reader();
+
+        // We will first try to collect up to 'threshold' terms into 'matchingTerms'
+        // if there are two many terms, we will fall back to building the 'builder'
+        final int threshold = Math.min(BOOLEAN_REWRITE_TERM_COUNT_THRESHOLD, BooleanQuery.getMaxClauseCount());
+        assert termData.size() > threshold : "Query should have been rewritten";
+        List<TermAndState> matchingTerms = new ArrayList<>(threshold);
+        DocIdSetBuilder builder = null;
+
+        final Fields fields = reader.fields();
+        String lastField = null;
+        Terms terms = null;
+        TermsEnum termsEnum = null;
+        PostingsEnum docs = null;
+        TermIterator iterator = termData.iterator();
+        for (BytesRef term = iterator.next(); term != null; term = iterator.next()) {
+          String field = iterator.field();
+          // comparing references is fine here
+          if (field != lastField) {
+            terms = fields.terms(field);
+            if (terms == null) {
+              termsEnum = null;
+            } else {
+              termsEnum = terms.iterator();
+            }
+            lastField = field;
+          }
+          if (termsEnum != null && termsEnum.seekExact(term)) {
+            if (matchingTerms == null) {
+              docs = termsEnum.postings(docs, PostingsEnum.NONE);
+              builder.add(docs);
+            } else if (matchingTerms.size() < threshold) {
+              matchingTerms.add(new TermAndState(field, termsEnum));
+            } else {
+              assert matchingTerms.size() == threshold;
+              if (singleField) {
+                // common case: all terms are in the same field
+                // use an optimized builder that leverages terms stats to be more efficient
+                builder = new DocIdSetBuilder(reader.maxDoc(), terms);
+              } else {
+                // corner case: different fields
+                // don't make assumptions about the docs we will get
+                builder = new DocIdSetBuilder(reader.maxDoc());
+              }
+              docs = termsEnum.postings(docs, PostingsEnum.NONE);
+              builder.add(docs);
+              for (TermAndState t : matchingTerms) {
+                t.termsEnum.seekExact(t.term, t.state);
+                docs = t.termsEnum.postings(docs, PostingsEnum.NONE);
+                builder.add(docs);
+              }
+              matchingTerms = null;
+            }
+          }
+        }
+        if (matchingTerms != null) {
+          assert builder == null;
+          BooleanQuery.Builder bq = new BooleanQuery.Builder();
+          for (TermAndState t : matchingTerms) {
+            final TermContext termContext = new TermContext(searcher.getTopReaderContext());
+            termContext.register(t.state, context.ord, t.docFreq, t.totalTermFreq);
+            bq.add(new TermQuery(new Term(t.field, t.term), termContext), Occur.SHOULD);
+          }
+          Query q = new ConstantScoreQuery(bq.build());
+          final Weight weight = searcher.rewrite(q).createWeight(searcher, needsScores);
+          weight.normalize(1f, score());
+          return new WeightOrDocIdSet(weight);
+        } else {
+          assert builder != null;
+          return new WeightOrDocIdSet(builder.build());
+        }
+      }
+
+      private Scorer scorer(DocIdSet set) throws IOException {
+        if (set == null) {
+          return null;
+        }
+        final DocIdSetIterator disi = set.iterator();
+        if (disi == null) {
+          return null;
+        }
+        return new ConstantScoreScorer(this, score(), disi);
+      }
+
+      @Override
+      public BulkScorer bulkScorer(LeafReaderContext context) throws IOException {
+        final WeightOrDocIdSet weightOrBitSet = rewrite(context);
+        if (weightOrBitSet.weight != null) {
+          return weightOrBitSet.weight.bulkScorer(context);
+        } else {
+          final Scorer scorer = scorer(weightOrBitSet.set);
+          if (scorer == null) {
+            return null;
+          }
+          return new DefaultBulkScorer(scorer);
+        }
+      }
+
+      @Override
+      public Scorer scorer(LeafReaderContext context) throws IOException {
+        final WeightOrDocIdSet weightOrBitSet = rewrite(context);
+        if (weightOrBitSet.weight != null) {
+          return weightOrBitSet.weight.scorer(context);
+        } else {
+          return scorer(weightOrBitSet.set);
+        }
+      }
+    };
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8511f9e6/lucene/core/src/java/org/apache/lucene/search/UsageTrackingQueryCachingPolicy.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/UsageTrackingQueryCachingPolicy.java b/lucene/core/src/java/org/apache/lucene/search/UsageTrackingQueryCachingPolicy.java
index ab68eeb..035947f 100644
--- a/lucene/core/src/java/org/apache/lucene/search/UsageTrackingQueryCachingPolicy.java
+++ b/lucene/core/src/java/org/apache/lucene/search/UsageTrackingQueryCachingPolicy.java
@@ -50,9 +50,8 @@ public final class UsageTrackingQueryCachingPolicy implements QueryCachingPolicy
     // DocIdSet in the first place
     return query instanceof MultiTermQuery ||
         query instanceof MultiTermQueryConstantScoreWrapper ||
-        isPointQuery(query) ||
-        // can't refer to TermsQuery directly as it is in another module
-        "TermsQuery".equals(query.getClass().getSimpleName());
+        query instanceof TermInSetQuery ||
+        isPointQuery(query);
   }
 
   static boolean isCheap(Query query) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8511f9e6/lucene/core/src/test/org/apache/lucene/search/TermInSetQueryTest.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/search/TermInSetQueryTest.java b/lucene/core/src/test/org/apache/lucene/search/TermInSetQueryTest.java
new file mode 100644
index 0000000..e694d97
--- /dev/null
+++ b/lucene/core/src/test/org/apache/lucene/search/TermInSetQueryTest.java
@@ -0,0 +1,328 @@
+/*
+ * 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;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import com.carrotsearch.randomizedtesting.generators.RandomPicks;
+import com.carrotsearch.randomizedtesting.generators.RandomStrings;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field.Store;
+import org.apache.lucene.document.StringField;
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.Fields;
+import org.apache.lucene.index.FilterDirectoryReader;
+import org.apache.lucene.index.FilterLeafReader;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.LeafReader;
+import org.apache.lucene.index.RandomIndexWriter;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.index.Terms;
+import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.search.BooleanClause.Occur;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.RamUsageTester;
+import org.apache.lucene.util.TestUtil;
+
+public class TermInSetQueryTest extends LuceneTestCase {
+
+  public void testDuel() throws IOException {
+    final int iters = atLeast(2);
+    for (int iter = 0; iter < iters; ++iter) {
+      final List<Term> allTerms = new ArrayList<>();
+      final int numTerms = TestUtil.nextInt(random(), 1, 1 << TestUtil.nextInt(random(), 1, 10));
+      for (int i = 0; i < numTerms; ++i) {
+        final String field = usually() ? "f" : "g";
+        final String value = TestUtil.randomAnalysisString(random(), 10, true);
+        allTerms.add(new Term(field, value));
+      }
+      Directory dir = newDirectory();
+      RandomIndexWriter iw = new RandomIndexWriter(random(), dir);
+      final int numDocs = atLeast(100);
+      for (int i = 0; i < numDocs; ++i) {
+        Document doc = new Document();
+        final Term term = allTerms.get(random().nextInt(allTerms.size()));
+        doc.add(new StringField(term.field(), term.text(), Store.NO));
+        iw.addDocument(doc);
+      }
+      if (numTerms > 1 && random().nextBoolean()) {
+        iw.deleteDocuments(new TermQuery(allTerms.get(0)));
+      }
+      iw.commit();
+      final IndexReader reader = iw.getReader();
+      final IndexSearcher searcher = newSearcher(reader);
+      iw.close();
+
+      if (reader.numDocs() == 0) {
+        // may occasionally happen if all documents got the same term
+        IOUtils.close(reader, dir);
+        continue;
+      }
+
+      for (int i = 0; i < 100; ++i) {
+        final float boost = random().nextFloat() * 10;
+        final int numQueryTerms = TestUtil.nextInt(random(), 1, 1 << TestUtil.nextInt(random(), 1, 8));
+        List<Term> queryTerms = new ArrayList<>();
+        for (int j = 0; j < numQueryTerms; ++j) {
+          queryTerms.add(allTerms.get(random().nextInt(allTerms.size())));
+        }
+        final BooleanQuery.Builder bq = new BooleanQuery.Builder();
+        for (Term t : queryTerms) {
+          bq.add(new TermQuery(t), Occur.SHOULD);
+        }
+        final Query q1 = new ConstantScoreQuery(bq.build());
+        final Query q2 = new TermInSetQuery(queryTerms);
+        assertSameMatches(searcher, new BoostQuery(q1, boost), new BoostQuery(q2, boost), true);
+      }
+
+      reader.close();
+      dir.close();
+    }
+  }
+
+  private void assertSameMatches(IndexSearcher searcher, Query q1, Query q2, boolean scores) throws IOException {
+    final int maxDoc = searcher.getIndexReader().maxDoc();
+    final TopDocs td1 = searcher.search(q1, maxDoc, scores ? Sort.RELEVANCE : Sort.INDEXORDER);
+    final TopDocs td2 = searcher.search(q2, maxDoc, scores ? Sort.RELEVANCE : Sort.INDEXORDER);
+    assertEquals(td1.totalHits, td2.totalHits);
+    for (int i = 0; i < td1.scoreDocs.length; ++i) {
+      assertEquals(td1.scoreDocs[i].doc, td2.scoreDocs[i].doc);
+      if (scores) {
+        assertEquals(td1.scoreDocs[i].score, td2.scoreDocs[i].score, 10e-7);
+      }
+    }
+  }
+
+  private TermInSetQuery termsQuery(boolean singleField, Term...terms) {
+    return termsQuery(singleField, Arrays.asList(terms));
+  }
+
+  private TermInSetQuery termsQuery(boolean singleField, Collection<Term> termList) {
+    if (!singleField) {
+      return new TermInSetQuery(new ArrayList<>(termList));
+    }
+    final TermInSetQuery filter;
+    List<BytesRef> bytes = new ArrayList<>();
+    String field = null;
+    for (Term term : termList) {
+        bytes.add(term.bytes());
+        if (field != null) {
+          assertEquals(term.field(), field);
+        }
+        field = term.field();
+    }
+    assertNotNull(field);
+    filter = new TermInSetQuery(field, bytes);
+    return filter;
+  }
+
+  public void testHashCodeAndEquals() {
+    int num = atLeast(100);
+    final boolean singleField = random().nextBoolean();
+    List<Term> terms = new ArrayList<>();
+    Set<Term> uniqueTerms = new HashSet<>();
+    for (int i = 0; i < num; i++) {
+      String field = "field" + (singleField ? "1" : random().nextInt(100));
+      String string = TestUtil.randomRealisticUnicodeString(random());
+      terms.add(new Term(field, string));
+      uniqueTerms.add(new Term(field, string));
+      TermInSetQuery left = termsQuery(singleField ? random().nextBoolean() : false, uniqueTerms);
+      Collections.shuffle(terms, random());
+      TermInSetQuery right = termsQuery(singleField ? random().nextBoolean() : false, terms);
+      assertEquals(right, left);
+      assertEquals(right.hashCode(), left.hashCode());
+      if (uniqueTerms.size() > 1) {
+        List<Term> asList = new ArrayList<>(uniqueTerms);
+        asList.remove(0);
+        TermInSetQuery notEqual = termsQuery(singleField ? random().nextBoolean() : false, asList);
+        assertFalse(left.equals(notEqual));
+        assertFalse(right.equals(notEqual));
+      }
+    }
+
+    TermInSetQuery tq1 = new TermInSetQuery(new Term("thing", "apple"));
+    TermInSetQuery tq2 = new TermInSetQuery(new Term("thing", "orange"));
+    assertFalse(tq1.hashCode() == tq2.hashCode());
+
+    // different fields with the same term should have differing hashcodes
+    tq1 = new TermInSetQuery(new Term("thing1", "apple"));
+    tq2 = new TermInSetQuery(new Term("thing2", "apple"));
+    assertFalse(tq1.hashCode() == tq2.hashCode());
+  }
+
+  public void testSingleFieldEquals() {
+    // Two terms with the same hash code
+    assertEquals("AaAaBB".hashCode(), "BBBBBB".hashCode());
+    TermInSetQuery left = termsQuery(true, new Term("id", "AaAaAa"), new Term("id", "AaAaBB"));
+    TermInSetQuery right = termsQuery(true, new Term("id", "AaAaAa"), new Term("id", "BBBBBB"));
+    assertFalse(left.equals(right));
+  }
+
+  public void testToString() {
+    TermInSetQuery termsQuery = new TermInSetQuery(new Term("field1", "a"),
+                                              new Term("field1", "b"),
+                                              new Term("field1", "c"));
+    assertEquals("field1:a field1:b field1:c", termsQuery.toString());
+  }
+
+  public void testDedup() {
+    Query query1 = new TermInSetQuery(new Term("foo", "bar"));
+    Query query2 = new TermInSetQuery(new Term("foo", "bar"), new Term("foo", "bar"));
+    QueryUtils.checkEqual(query1, query2);
+  }
+
+  public void testOrderDoesNotMatter() {
+    // order of terms if different
+    Query query1 = new TermInSetQuery(new Term("foo", "bar"), new Term("foo", "baz"));
+    Query query2 = new TermInSetQuery(new Term("foo", "baz"), new Term("foo", "bar"));
+    QueryUtils.checkEqual(query1, query2);
+
+    // order of fields is different
+    query1 = new TermInSetQuery(new Term("foo", "bar"), new Term("bar", "bar"));
+    query2 = new TermInSetQuery(new Term("bar", "bar"), new Term("foo", "bar"));
+    QueryUtils.checkEqual(query1, query2);
+  }
+
+  public void testRamBytesUsed() {
+    List<Term> terms = new ArrayList<>();
+    final int numTerms = 1000 + random().nextInt(1000);
+    for (int i = 0; i < numTerms; ++i) {
+      terms.add(new Term("f", RandomStrings.randomUnicodeOfLength(random(), 10)));
+    }
+    TermInSetQuery query = new TermInSetQuery(terms);
+    final long actualRamBytesUsed = RamUsageTester.sizeOf(query);
+    final long expectedRamBytesUsed = query.ramBytesUsed();
+    // error margin within 5%
+    assertEquals(actualRamBytesUsed, expectedRamBytesUsed, actualRamBytesUsed / 20);
+  }
+
+  private static class TermsCountingDirectoryReaderWrapper extends FilterDirectoryReader {
+
+    private final AtomicInteger counter;
+    
+    public TermsCountingDirectoryReaderWrapper(DirectoryReader in, AtomicInteger counter) throws IOException {
+      super(in, new TermsCountingSubReaderWrapper(counter));
+      this.counter = counter;
+    }
+
+    private static class TermsCountingSubReaderWrapper extends SubReaderWrapper {
+      private final AtomicInteger counter;
+
+      public TermsCountingSubReaderWrapper(AtomicInteger counter) {
+        this.counter = counter;
+      }
+
+      @Override
+      public LeafReader wrap(LeafReader reader) {
+        return new TermsCountingLeafReaderWrapper(reader, counter);
+      }
+    }
+
+    private static class TermsCountingLeafReaderWrapper extends FilterLeafReader {
+
+      private final AtomicInteger counter;
+
+      public TermsCountingLeafReaderWrapper(LeafReader in, AtomicInteger counter) {
+        super(in);
+        this.counter = counter;
+      }
+
+      @Override
+      public Fields fields() throws IOException {
+        return new FilterFields(in.fields()) {
+          @Override
+          public Terms terms(String field) throws IOException {
+            final Terms in = this.in.terms(field);
+            if (in == null) {
+              return null;
+            }
+            return new FilterTerms(in) {
+              @Override
+              public TermsEnum iterator() throws IOException {
+                counter.incrementAndGet();
+                return super.iterator();
+              }
+            };
+          }
+        };
+      }
+      
+    }
+
+    @Override
+    protected DirectoryReader doWrapDirectoryReader(DirectoryReader in) throws IOException {
+      return new TermsCountingDirectoryReaderWrapper(in, counter);
+    }
+
+  }
+
+  public void testPullOneTermsEnumPerField() throws Exception {
+    Directory dir = newDirectory();
+    RandomIndexWriter w = new RandomIndexWriter(random(), dir);
+    Document doc = new Document();
+    doc.add(new StringField("foo", "1", Store.NO));
+    doc.add(new StringField("bar", "2", Store.NO));
+    doc.add(new StringField("baz", "3", Store.NO));
+    w.addDocument(doc);
+    DirectoryReader reader = w.getReader();
+    w.close();
+    final AtomicInteger counter = new AtomicInteger();
+    DirectoryReader wrapped = new TermsCountingDirectoryReaderWrapper(reader, counter);
+
+    final List<Term> terms = new ArrayList<>();
+    final Set<String> fields = new HashSet<>();
+    // enough terms to avoid the rewrite
+    final int numTerms = TestUtil.nextInt(random(), TermInSetQuery.BOOLEAN_REWRITE_TERM_COUNT_THRESHOLD + 1, 100);
+    for (int i = 0; i < numTerms; ++i) {
+      final String field = RandomPicks.randomFrom(random(), new String[] {"foo", "bar", "baz"});
+      final BytesRef term = new BytesRef(RandomStrings.randomUnicodeOfCodepointLength(random(), 10));
+      fields.add(field);
+      terms.add(new Term(field, term));
+    }
+
+    new IndexSearcher(wrapped).count(new TermInSetQuery(terms));
+    assertEquals(fields.size(), counter.get());
+    wrapped.close();
+    dir.close();
+  }
+  
+  public void testBinaryToString() {
+    TermInSetQuery query = new TermInSetQuery(new Term("field", new BytesRef(new byte[] { (byte) 0xff, (byte) 0xfe })));
+    assertEquals("field:[ff fe]", query.toString());
+  }
+
+  public void testIsConsideredCostlyByQueryCache() throws IOException {
+    TermInSetQuery query = new TermInSetQuery(new Term("foo", "bar"), new Term("foo", "baz"));
+    UsageTrackingQueryCachingPolicy policy = new UsageTrackingQueryCachingPolicy();
+    assertFalse(policy.shouldCache(query));
+    policy.onUse(query);
+    policy.onUse(query);
+    // cached after two uses
+    assertTrue(policy.shouldCache(query));
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8511f9e6/lucene/facet/src/java/org/apache/lucene/facet/MultiFacetQuery.java
----------------------------------------------------------------------
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/MultiFacetQuery.java b/lucene/facet/src/java/org/apache/lucene/facet/MultiFacetQuery.java
index dd212c6..a010709 100644
--- a/lucene/facet/src/java/org/apache/lucene/facet/MultiFacetQuery.java
+++ b/lucene/facet/src/java/org/apache/lucene/facet/MultiFacetQuery.java
@@ -16,13 +16,13 @@
  */
 package org.apache.lucene.facet;
 
-import org.apache.lucene.index.Term;
-import org.apache.lucene.queries.TermsQuery;
-import org.apache.lucene.search.Query;
-
 import java.util.ArrayList;
 import java.util.Collection;
 
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.TermInSetQuery;
+
 /**
  * A multi-terms {@link Query} over a {@link FacetField}.
  * <p>
@@ -30,9 +30,9 @@ import java.util.Collection;
  * especially in cases where you don't intend to use {@link DrillSideways}
  *
  * @lucene.experimental
- * @see org.apache.lucene.queries.TermsQuery
+ * @see org.apache.lucene.search.TermInSetQuery
  */
-public class MultiFacetQuery extends TermsQuery {
+public class MultiFacetQuery extends TermInSetQuery {
 
   /**
    * Creates a new {@code MultiFacetQuery} filtering the query on the given dimension.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8511f9e6/lucene/queries/src/java/org/apache/lucene/queries/TermsQuery.java
----------------------------------------------------------------------
diff --git a/lucene/queries/src/java/org/apache/lucene/queries/TermsQuery.java b/lucene/queries/src/java/org/apache/lucene/queries/TermsQuery.java
index 5c03b2d..994c60f 100644
--- a/lucene/queries/src/java/org/apache/lucene/queries/TermsQuery.java
+++ b/lucene/queries/src/java/org/apache/lucene/queries/TermsQuery.java
@@ -16,130 +16,33 @@
  */
 package org.apache.lucene.queries;
 
-import java.io.IOException;
-import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Objects;
-import java.util.Set;
-import java.util.SortedSet;
 
-import org.apache.lucene.index.Fields;
-import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.index.LeafReader;
-import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.index.PostingsEnum;
-import org.apache.lucene.index.PrefixCodedTerms;
-import org.apache.lucene.index.PrefixCodedTerms.TermIterator;
 import org.apache.lucene.index.Term;
-import org.apache.lucene.index.TermContext;
-import org.apache.lucene.index.TermState;
-import org.apache.lucene.index.Terms;
-import org.apache.lucene.index.TermsEnum;
-import org.apache.lucene.search.BooleanClause.Occur;
-import org.apache.lucene.search.BooleanQuery;
-import org.apache.lucene.search.BulkScorer;
-import org.apache.lucene.search.ConstantScoreQuery;
-import org.apache.lucene.search.ConstantScoreScorer;
-import org.apache.lucene.search.ConstantScoreWeight;
-import org.apache.lucene.search.DocIdSet;
-import org.apache.lucene.search.DocIdSetIterator;
-import org.apache.lucene.search.IndexSearcher;
-import org.apache.lucene.search.Query;
-import org.apache.lucene.search.Scorer;
-import org.apache.lucene.search.TermQuery;
-import org.apache.lucene.search.Weight;
-import org.apache.lucene.util.Accountable;
-import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.search.TermInSetQuery;
 import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.DocIdSetBuilder;
-import org.apache.lucene.util.BytesRefBuilder;
-import org.apache.lucene.util.RamUsageEstimator;
 
 /**
- * Specialization for a disjunction over many terms that behaves like a
- * {@link ConstantScoreQuery} over a {@link BooleanQuery} containing only
- * {@link org.apache.lucene.search.BooleanClause.Occur#SHOULD} clauses.
- * <p>For instance in the following example, both @{code q1} and {@code q2}
- * would yield the same scores:
- * <pre class="prettyprint">
- * Query q1 = new TermsQuery(new Term("field", "foo"), new Term("field", "bar"));
- *
- * BooleanQuery bq = new BooleanQuery();
- * bq.add(new TermQuery(new Term("field", "foo")), Occur.SHOULD);
- * bq.add(new TermQuery(new Term("field", "bar")), Occur.SHOULD);
- * Query q2 = new ConstantScoreQuery(bq);
- * </pre>
- * <p>When there are few terms, this query executes like a regular disjunction.
- * However, when there are many terms, instead of merging iterators on the fly,
- * it will populate a bit set with matching docs and return a {@link Scorer}
- * over this bit set.
- * <p>NOTE: This query produces scores that are equal to its boost
+ * @deprecated Use {@link org.apache.lucene.search.TermInSetQuery}
  */
-public class TermsQuery extends Query implements Accountable {
-
-  private static final long BASE_RAM_BYTES_USED = RamUsageEstimator.shallowSizeOfInstance(TermsQuery.class);
-  // Same threshold as MultiTermQueryConstantScoreWrapper
-  static final int BOOLEAN_REWRITE_TERM_COUNT_THRESHOLD = 16;
-
-  private final boolean singleField; // whether all terms are from the same field
-  private final PrefixCodedTerms termData;
-  private final int termDataHashCode; // cached hashcode of termData
+@Deprecated
+public class TermsQuery extends TermInSetQuery {
 
   /**
    * Creates a new {@link TermsQuery} from the given collection. It
    * can contain duplicate terms and multiple fields.
    */
   public TermsQuery(Collection<Term> terms) {
-    Term[] sortedTerms = terms.toArray(new Term[terms.size()]);
-    // already sorted if we are a SortedSet with natural order
-    boolean sorted = terms instanceof SortedSet && ((SortedSet<Term>)terms).comparator() == null;
-    if (!sorted) {
-      ArrayUtil.timSort(sortedTerms);
-    }
-    PrefixCodedTerms.Builder builder = new PrefixCodedTerms.Builder();
-    Set<String> fields = new HashSet<>();
-    Term previous = null;
-    for (Term term : sortedTerms) {
-      if (term.equals(previous) == false) {
-        fields.add(term.field());
-        builder.add(term);
-      }
-      previous = term;
-    }
-    singleField = fields.size() == 1;
-    termData = builder.finish();
-    termDataHashCode = termData.hashCode();
+    super(terms);
   }
-  
+
   /**
    * Creates a new {@link TermsQuery} from the given collection for
    * a single field. It can contain duplicate terms.
    */
   public TermsQuery(String field, Collection<BytesRef> terms) {
-    BytesRef[] sortedTerms = terms.toArray(new BytesRef[terms.size()]);
-    // already sorted if we are a SortedSet with natural order
-    boolean sorted = terms instanceof SortedSet && ((SortedSet<BytesRef>)terms).comparator() == null;
-    if (!sorted) {
-      ArrayUtil.timSort(sortedTerms);
-    }
-    PrefixCodedTerms.Builder builder = new PrefixCodedTerms.Builder();
-    BytesRefBuilder previous = null;
-    for (BytesRef term : sortedTerms) {
-      if (previous == null) {
-        previous = new BytesRefBuilder();
-      } else if (previous.get().equals(term)) {
-        continue; // deduplicate
-      }
-      builder.add(field, term);
-      previous.copyBytes(term);
-    }
-    singleField = true;
-    termData = builder.finish();
-    termDataHashCode = termData.hashCode();
+    super(field, terms);
   }
 
   /**
@@ -147,7 +50,7 @@ public class TermsQuery extends Query implements Accountable {
    * a single field.
    */
   public TermsQuery(String field, BytesRef...terms) {
-   this(field, Arrays.asList(terms));
+    this(field, Arrays.asList(terms));
   }
 
   /**
@@ -158,225 +61,4 @@ public class TermsQuery extends Query implements Accountable {
     this(Arrays.asList(terms));
   }
 
-  @Override
-  public Query rewrite(IndexReader reader) throws IOException {
-    final int threshold = Math.min(BOOLEAN_REWRITE_TERM_COUNT_THRESHOLD, BooleanQuery.getMaxClauseCount());
-    if (termData.size() <= threshold) {
-      BooleanQuery.Builder bq = new BooleanQuery.Builder();
-      TermIterator iterator = termData.iterator();
-      for (BytesRef term = iterator.next(); term != null; term = iterator.next()) {
-        bq.add(new TermQuery(new Term(iterator.field(), BytesRef.deepCopyOf(term))), Occur.SHOULD);
-      }
-      return new ConstantScoreQuery(bq.build());
-    }
-    return super.rewrite(reader);
-  }
-
-  @Override
-  public boolean equals(Object other) {
-    return sameClassAs(other) &&
-           equalsTo(getClass().cast(other));
-  }
-
-  private boolean equalsTo(TermsQuery other) {
-    // termData might be heavy to compare so check the hash code first
-    return termDataHashCode == other.termDataHashCode && 
-           termData.equals(other.termData);
-  }
-
-  @Override
-  public int hashCode() {
-    return 31 * classHash() + termDataHashCode;
-  }
-
-  /** Returns the terms wrapped in a PrefixCodedTerms. */
-  public PrefixCodedTerms getTermData() {
-    return termData;
-  }
-
-  @Override
-  public String toString(String defaultField) {
-    StringBuilder builder = new StringBuilder();
-    boolean first = true;
-    TermIterator iterator = termData.iterator();
-    for (BytesRef term = iterator.next(); term != null; term = iterator.next()) {
-      if (!first) {
-        builder.append(' ');
-      }
-      first = false;
-      builder.append(new Term(iterator.field(), term).toString());
-    }
-
-    return builder.toString();
-  }
-
-  @Override
-  public long ramBytesUsed() {
-    return BASE_RAM_BYTES_USED + termData.ramBytesUsed();
-  }
-
-  @Override
-  public Collection<Accountable> getChildResources() {
-    return Collections.emptyList();
-  }
-
-  private static class TermAndState {
-    final String field;
-    final TermsEnum termsEnum;
-    final BytesRef term;
-    final TermState state;
-    final int docFreq;
-    final long totalTermFreq;
-
-    TermAndState(String field, TermsEnum termsEnum) throws IOException {
-      this.field = field;
-      this.termsEnum = termsEnum;
-      this.term = BytesRef.deepCopyOf(termsEnum.term());
-      this.state = termsEnum.termState();
-      this.docFreq = termsEnum.docFreq();
-      this.totalTermFreq = termsEnum.totalTermFreq();
-    }
-  }
-
-  private static class WeightOrDocIdSet {
-    final Weight weight;
-    final DocIdSet set;
-
-    WeightOrDocIdSet(Weight weight) {
-      this.weight = Objects.requireNonNull(weight);
-      this.set = null;
-    }
-
-    WeightOrDocIdSet(DocIdSet bitset) {
-      this.set = bitset;
-      this.weight = null;
-    }
-  }
-
-  @Override
-  public Weight createWeight(IndexSearcher searcher, boolean needsScores) throws IOException {
-    return new ConstantScoreWeight(this) {
-
-      @Override
-      public void extractTerms(Set<Term> terms) {
-        // no-op
-        // This query is for abuse cases when the number of terms is too high to
-        // run efficiently as a BooleanQuery. So likewise we hide its terms in
-        // order to protect highlighters
-      }
-
-      /**
-       * On the given leaf context, try to either rewrite to a disjunction if
-       * there are few matching terms, or build a bitset containing matching docs.
-       */
-      private WeightOrDocIdSet rewrite(LeafReaderContext context) throws IOException {
-        final LeafReader reader = context.reader();
-
-        // We will first try to collect up to 'threshold' terms into 'matchingTerms'
-        // if there are two many terms, we will fall back to building the 'builder'
-        final int threshold = Math.min(BOOLEAN_REWRITE_TERM_COUNT_THRESHOLD, BooleanQuery.getMaxClauseCount());
-        assert termData.size() > threshold : "Query should have been rewritten";
-        List<TermAndState> matchingTerms = new ArrayList<>(threshold);
-        DocIdSetBuilder builder = null;
-
-        final Fields fields = reader.fields();
-        String lastField = null;
-        Terms terms = null;
-        TermsEnum termsEnum = null;
-        PostingsEnum docs = null;
-        TermIterator iterator = termData.iterator();
-        for (BytesRef term = iterator.next(); term != null; term = iterator.next()) {
-          String field = iterator.field();
-          // comparing references is fine here
-          if (field != lastField) {
-            terms = fields.terms(field);
-            if (terms == null) {
-              termsEnum = null;
-            } else {
-              termsEnum = terms.iterator();
-            }
-            lastField = field;
-          }
-          if (termsEnum != null && termsEnum.seekExact(term)) {
-            if (matchingTerms == null) {
-              docs = termsEnum.postings(docs, PostingsEnum.NONE);
-              builder.add(docs);
-            } else if (matchingTerms.size() < threshold) {
-              matchingTerms.add(new TermAndState(field, termsEnum));
-            } else {
-              assert matchingTerms.size() == threshold;
-              if (singleField) {
-                // common case: all terms are in the same field
-                // use an optimized builder that leverages terms stats to be more efficient
-                builder = new DocIdSetBuilder(reader.maxDoc(), terms);
-              } else {
-                // corner case: different fields
-                // don't make assumptions about the docs we will get
-                builder = new DocIdSetBuilder(reader.maxDoc());
-              }
-              docs = termsEnum.postings(docs, PostingsEnum.NONE);
-              builder.add(docs);
-              for (TermAndState t : matchingTerms) {
-                t.termsEnum.seekExact(t.term, t.state);
-                docs = t.termsEnum.postings(docs, PostingsEnum.NONE);
-                builder.add(docs);
-              }
-              matchingTerms = null;
-            }
-          }
-        }
-        if (matchingTerms != null) {
-          assert builder == null;
-          BooleanQuery.Builder bq = new BooleanQuery.Builder();
-          for (TermAndState t : matchingTerms) {
-            final TermContext termContext = new TermContext(searcher.getTopReaderContext());
-            termContext.register(t.state, context.ord, t.docFreq, t.totalTermFreq);
-            bq.add(new TermQuery(new Term(t.field, t.term), termContext), Occur.SHOULD);
-          }
-          Query q = new ConstantScoreQuery(bq.build());
-          final Weight weight = searcher.rewrite(q).createWeight(searcher, needsScores);
-          weight.normalize(1f, score());
-          return new WeightOrDocIdSet(weight);
-        } else {
-          assert builder != null;
-          return new WeightOrDocIdSet(builder.build());
-        }
-      }
-
-      private Scorer scorer(DocIdSet set) throws IOException {
-        if (set == null) {
-          return null;
-        }
-        final DocIdSetIterator disi = set.iterator();
-        if (disi == null) {
-          return null;
-        }
-        return new ConstantScoreScorer(this, score(), disi);
-      }
-
-      @Override
-      public BulkScorer bulkScorer(LeafReaderContext context) throws IOException {
-        final WeightOrDocIdSet weightOrBitSet = rewrite(context);
-        if (weightOrBitSet.weight != null) {
-          return weightOrBitSet.weight.bulkScorer(context);
-        } else {
-          final Scorer scorer = scorer(weightOrBitSet.set);
-          if (scorer == null) {
-            return null;
-          }
-          return new DefaultBulkScorer(scorer);
-        }
-      }
-
-      @Override
-      public Scorer scorer(LeafReaderContext context) throws IOException {
-        final WeightOrDocIdSet weightOrBitSet = rewrite(context);
-        if (weightOrBitSet.weight != null) {
-          return weightOrBitSet.weight.scorer(context);
-        } else {
-          return scorer(weightOrBitSet.set);
-        }
-      }
-    };
-  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8511f9e6/lucene/queries/src/test/org/apache/lucene/queries/TermsQueryTest.java
----------------------------------------------------------------------
diff --git a/lucene/queries/src/test/org/apache/lucene/queries/TermsQueryTest.java b/lucene/queries/src/test/org/apache/lucene/queries/TermsQueryTest.java
deleted file mode 100644
index f8b10ef..0000000
--- a/lucene/queries/src/test/org/apache/lucene/queries/TermsQueryTest.java
+++ /dev/null
@@ -1,339 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.lucene.queries;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-import java.util.concurrent.atomic.AtomicInteger;
-
-import org.apache.lucene.document.Document;
-import org.apache.lucene.document.Field.Store;
-import org.apache.lucene.document.StringField;
-import org.apache.lucene.index.DirectoryReader;
-import org.apache.lucene.index.Fields;
-import org.apache.lucene.index.FilterDirectoryReader;
-import org.apache.lucene.index.FilterLeafReader;
-import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.index.LeafReader;
-import org.apache.lucene.index.RandomIndexWriter;
-import org.apache.lucene.index.Term;
-import org.apache.lucene.index.Terms;
-import org.apache.lucene.index.TermsEnum;
-import org.apache.lucene.search.BooleanClause.Occur;
-import org.apache.lucene.search.BooleanQuery;
-import org.apache.lucene.search.BoostQuery;
-import org.apache.lucene.search.ConstantScoreQuery;
-import org.apache.lucene.search.IndexSearcher;
-import org.apache.lucene.search.Query;
-import org.apache.lucene.search.QueryUtils;
-import org.apache.lucene.search.Sort;
-import org.apache.lucene.search.TermQuery;
-import org.apache.lucene.search.TopDocs;
-import org.apache.lucene.search.UsageTrackingQueryCachingPolicy;
-import org.apache.lucene.store.Directory;
-import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.IOUtils;
-import org.apache.lucene.util.LuceneTestCase;
-import org.apache.lucene.util.RamUsageTester;
-import org.apache.lucene.util.TestUtil;
-
-import com.carrotsearch.randomizedtesting.generators.RandomPicks;
-import com.carrotsearch.randomizedtesting.generators.RandomStrings;
-
-public class TermsQueryTest extends LuceneTestCase {
-
-  public void testDuel() throws IOException {
-    final int iters = atLeast(2);
-    for (int iter = 0; iter < iters; ++iter) {
-      final List<Term> allTerms = new ArrayList<>();
-      final int numTerms = TestUtil.nextInt(random(), 1, 1 << TestUtil.nextInt(random(), 1, 10));
-      for (int i = 0; i < numTerms; ++i) {
-        final String field = usually() ? "f" : "g";
-        final String value = TestUtil.randomAnalysisString(random(), 10, true);
-        allTerms.add(new Term(field, value));
-      }
-      Directory dir = newDirectory();
-      RandomIndexWriter iw = new RandomIndexWriter(random(), dir);
-      final int numDocs = atLeast(100);
-      for (int i = 0; i < numDocs; ++i) {
-        Document doc = new Document();
-        final Term term = allTerms.get(random().nextInt(allTerms.size()));
-        doc.add(new StringField(term.field(), term.text(), Store.NO));
-        iw.addDocument(doc);
-      }
-      if (numTerms > 1 && random().nextBoolean()) {
-        iw.deleteDocuments(new TermQuery(allTerms.get(0)));
-      }
-      iw.commit();
-      final IndexReader reader = iw.getReader();
-      final IndexSearcher searcher = newSearcher(reader);
-      iw.close();
-
-      if (reader.numDocs() == 0) {
-        // may occasionally happen if all documents got the same term
-        IOUtils.close(reader, dir);
-        continue;
-      }
-
-      for (int i = 0; i < 100; ++i) {
-        final float boost = random().nextFloat() * 10;
-        final int numQueryTerms = TestUtil.nextInt(random(), 1, 1 << TestUtil.nextInt(random(), 1, 8));
-        List<Term> queryTerms = new ArrayList<>();
-        for (int j = 0; j < numQueryTerms; ++j) {
-          queryTerms.add(allTerms.get(random().nextInt(allTerms.size())));
-        }
-        final BooleanQuery.Builder bq = new BooleanQuery.Builder();
-        for (Term t : queryTerms) {
-          bq.add(new TermQuery(t), Occur.SHOULD);
-        }
-        final Query q1 = new ConstantScoreQuery(bq.build());
-        final Query q2 = new TermsQuery(queryTerms);
-        assertSameMatches(searcher, new BoostQuery(q1, boost), new BoostQuery(q2, boost), true);
-      }
-
-      reader.close();
-      dir.close();
-    }
-  }
-
-  private void assertSameMatches(IndexSearcher searcher, Query q1, Query q2, boolean scores) throws IOException {
-    final int maxDoc = searcher.getIndexReader().maxDoc();
-    final TopDocs td1 = searcher.search(q1, maxDoc, scores ? Sort.RELEVANCE : Sort.INDEXORDER);
-    final TopDocs td2 = searcher.search(q2, maxDoc, scores ? Sort.RELEVANCE : Sort.INDEXORDER);
-    assertEquals(td1.totalHits, td2.totalHits);
-    for (int i = 0; i < td1.scoreDocs.length; ++i) {
-      assertEquals(td1.scoreDocs[i].doc, td2.scoreDocs[i].doc);
-      if (scores) {
-        assertEquals(td1.scoreDocs[i].score, td2.scoreDocs[i].score, 10e-7);
-      }
-    }
-  }
-
-  private TermsQuery termsQuery(boolean singleField, Term...terms) {
-    return termsQuery(singleField, Arrays.asList(terms));
-  }
-
-  private TermsQuery termsQuery(boolean singleField, Collection<Term> termList) {
-    if (!singleField) {
-      return new TermsQuery(new ArrayList<>(termList));
-    }
-    final TermsQuery filter;
-    List<BytesRef> bytes = new ArrayList<>();
-    String field = null;
-    for (Term term : termList) {
-        bytes.add(term.bytes());
-        if (field != null) {
-          assertEquals(term.field(), field);
-        }
-        field = term.field();
-    }
-    assertNotNull(field);
-    filter = new TermsQuery(field, bytes);
-    return filter;
-  }
-
-  public void testHashCodeAndEquals() {
-    int num = atLeast(100);
-    final boolean singleField = random().nextBoolean();
-    List<Term> terms = new ArrayList<>();
-    Set<Term> uniqueTerms = new HashSet<>();
-    for (int i = 0; i < num; i++) {
-      String field = "field" + (singleField ? "1" : random().nextInt(100));
-      String string = TestUtil.randomRealisticUnicodeString(random());
-      terms.add(new Term(field, string));
-      uniqueTerms.add(new Term(field, string));
-      TermsQuery left = termsQuery(singleField ? random().nextBoolean() : false, uniqueTerms);
-      Collections.shuffle(terms, random());
-      TermsQuery right = termsQuery(singleField ? random().nextBoolean() : false, terms);
-      assertEquals(right, left);
-      assertEquals(right.hashCode(), left.hashCode());
-      if (uniqueTerms.size() > 1) {
-        List<Term> asList = new ArrayList<>(uniqueTerms);
-        asList.remove(0);
-        TermsQuery notEqual = termsQuery(singleField ? random().nextBoolean() : false, asList);
-        assertFalse(left.equals(notEqual));
-        assertFalse(right.equals(notEqual));
-      }
-    }
-
-    TermsQuery tq1 = new TermsQuery(new Term("thing", "apple"));
-    TermsQuery tq2 = new TermsQuery(new Term("thing", "orange"));
-    assertFalse(tq1.hashCode() == tq2.hashCode());
-
-    // different fields with the same term should have differing hashcodes
-    tq1 = new TermsQuery(new Term("thing1", "apple"));
-    tq2 = new TermsQuery(new Term("thing2", "apple"));
-    assertFalse(tq1.hashCode() == tq2.hashCode());
-  }
-
-  public void testSingleFieldEquals() {
-    // Two terms with the same hash code
-    assertEquals("AaAaBB".hashCode(), "BBBBBB".hashCode());
-    TermsQuery left = termsQuery(true, new Term("id", "AaAaAa"), new Term("id", "AaAaBB"));
-    TermsQuery right = termsQuery(true, new Term("id", "AaAaAa"), new Term("id", "BBBBBB"));
-    assertFalse(left.equals(right));
-  }
-
-  public void testToString() {
-    TermsQuery termsQuery = new TermsQuery(new Term("field1", "a"),
-                                              new Term("field1", "b"),
-                                              new Term("field1", "c"));
-    assertEquals("field1:a field1:b field1:c", termsQuery.toString());
-  }
-
-  public void testDedup() {
-    Query query1 = new TermsQuery(new Term("foo", "bar"));
-    Query query2 = new TermsQuery(new Term("foo", "bar"), new Term("foo", "bar"));
-    QueryUtils.checkEqual(query1, query2);
-  }
-
-  public void testOrderDoesNotMatter() {
-    // order of terms if different
-    Query query1 = new TermsQuery(new Term("foo", "bar"), new Term("foo", "baz"));
-    Query query2 = new TermsQuery(new Term("foo", "baz"), new Term("foo", "bar"));
-    QueryUtils.checkEqual(query1, query2);
-
-    // order of fields is different
-    query1 = new TermsQuery(new Term("foo", "bar"), new Term("bar", "bar"));
-    query2 = new TermsQuery(new Term("bar", "bar"), new Term("foo", "bar"));
-    QueryUtils.checkEqual(query1, query2);
-  }
-
-  public void testRamBytesUsed() {
-    List<Term> terms = new ArrayList<>();
-    final int numTerms = 1000 + random().nextInt(1000);
-    for (int i = 0; i < numTerms; ++i) {
-      terms.add(new Term("f", RandomStrings.randomUnicodeOfLength(random(), 10)));
-    }
-    TermsQuery query = new TermsQuery(terms);
-    final long actualRamBytesUsed = RamUsageTester.sizeOf(query);
-    final long expectedRamBytesUsed = query.ramBytesUsed();
-    // error margin within 5%
-    assertEquals(actualRamBytesUsed, expectedRamBytesUsed, actualRamBytesUsed / 20);
-  }
-
-  private static class TermsCountingDirectoryReaderWrapper extends FilterDirectoryReader {
-
-    private final AtomicInteger counter;
-    
-    public TermsCountingDirectoryReaderWrapper(DirectoryReader in, AtomicInteger counter) throws IOException {
-      super(in, new TermsCountingSubReaderWrapper(counter));
-      this.counter = counter;
-    }
-
-    private static class TermsCountingSubReaderWrapper extends SubReaderWrapper {
-      private final AtomicInteger counter;
-
-      public TermsCountingSubReaderWrapper(AtomicInteger counter) {
-        this.counter = counter;
-      }
-
-      @Override
-      public LeafReader wrap(LeafReader reader) {
-        return new TermsCountingLeafReaderWrapper(reader, counter);
-      }
-    }
-
-    private static class TermsCountingLeafReaderWrapper extends FilterLeafReader {
-
-      private final AtomicInteger counter;
-
-      public TermsCountingLeafReaderWrapper(LeafReader in, AtomicInteger counter) {
-        super(in);
-        this.counter = counter;
-      }
-
-      @Override
-      public Fields fields() throws IOException {
-        return new FilterFields(in.fields()) {
-          @Override
-          public Terms terms(String field) throws IOException {
-            final Terms in = this.in.terms(field);
-            if (in == null) {
-              return null;
-            }
-            return new FilterTerms(in) {
-              @Override
-              public TermsEnum iterator() throws IOException {
-                counter.incrementAndGet();
-                return super.iterator();
-              }
-            };
-          }
-        };
-      }
-      
-    }
-
-    @Override
-    protected DirectoryReader doWrapDirectoryReader(DirectoryReader in) throws IOException {
-      return new TermsCountingDirectoryReaderWrapper(in, counter);
-    }
-
-  }
-
-  public void testPullOneTermsEnumPerField() throws Exception {
-    Directory dir = newDirectory();
-    RandomIndexWriter w = new RandomIndexWriter(random(), dir);
-    Document doc = new Document();
-    doc.add(new StringField("foo", "1", Store.NO));
-    doc.add(new StringField("bar", "2", Store.NO));
-    doc.add(new StringField("baz", "3", Store.NO));
-    w.addDocument(doc);
-    DirectoryReader reader = w.getReader();
-    w.close();
-    final AtomicInteger counter = new AtomicInteger();
-    DirectoryReader wrapped = new TermsCountingDirectoryReaderWrapper(reader, counter);
-
-    final List<Term> terms = new ArrayList<>();
-    final Set<String> fields = new HashSet<>();
-    // enough terms to avoid the rewrite
-    final int numTerms = TestUtil.nextInt(random(), TermsQuery.BOOLEAN_REWRITE_TERM_COUNT_THRESHOLD + 1, 100);
-    for (int i = 0; i < numTerms; ++i) {
-      final String field = RandomPicks.randomFrom(random(), new String[] {"foo", "bar", "baz"});
-      final BytesRef term = new BytesRef(RandomStrings.randomUnicodeOfCodepointLength(random(), 10));
-      fields.add(field);
-      terms.add(new Term(field, term));
-    }
-
-    new IndexSearcher(wrapped).count(new TermsQuery(terms));
-    assertEquals(fields.size(), counter.get());
-    wrapped.close();
-    dir.close();
-  }
-  
-  public void testBinaryToString() {
-    TermsQuery query = new TermsQuery(new Term("field", new BytesRef(new byte[] { (byte) 0xff, (byte) 0xfe })));
-    assertEquals("field:[ff fe]", query.toString());
-  }
-
-  public void testIsConsideredCostlyByQueryCache() throws IOException {
-    TermsQuery query = new TermsQuery(new Term("foo", "bar"), new Term("foo", "baz"));
-    UsageTrackingQueryCachingPolicy policy = new UsageTrackingQueryCachingPolicy();
-    assertFalse(policy.shouldCache(query));
-    policy.onUse(query);
-    policy.onUse(query);
-    // cached after two uses
-    assertTrue(policy.shouldCache(query));
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8511f9e6/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/TermQueryPrefixTreeStrategy.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/TermQueryPrefixTreeStrategy.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/TermQueryPrefixTreeStrategy.java
index 0273466..4e37f5d 100644
--- a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/TermQueryPrefixTreeStrategy.java
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/TermQueryPrefixTreeStrategy.java
@@ -19,10 +19,8 @@ package org.apache.lucene.spatial.prefix;
 import java.util.ArrayList;
 import java.util.List;
 
-import org.locationtech.spatial4j.shape.Point;
-import org.locationtech.spatial4j.shape.Shape;
-import org.apache.lucene.queries.TermsQuery;
 import org.apache.lucene.search.Query;
+import org.apache.lucene.search.TermInSetQuery;
 import org.apache.lucene.spatial.prefix.tree.Cell;
 import org.apache.lucene.spatial.prefix.tree.CellIterator;
 import org.apache.lucene.spatial.prefix.tree.SpatialPrefixTree;
@@ -31,10 +29,12 @@ import org.apache.lucene.spatial.query.SpatialOperation;
 import org.apache.lucene.spatial.query.UnsupportedSpatialOperation;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefBuilder;
+import org.locationtech.spatial4j.shape.Point;
+import org.locationtech.spatial4j.shape.Shape;
 
 /**
  * A basic implementation of {@link PrefixTreeStrategy} using a large
- * {@link TermsQuery} of all the cells from
+ * {@link TermInSetQuery} of all the cells from
  * {@link SpatialPrefixTree#getTreeCellIterator(org.locationtech.spatial4j.shape.Shape, int)}.
  * It only supports the search of indexed Point shapes.
  * <p>
@@ -105,7 +105,7 @@ public class TermQueryPrefixTreeStrategy extends PrefixTreeStrategy {
     }
     //unfortunately TermsQuery will needlessly sort & dedupe
     //TODO an automatonQuery might be faster?
-    return new TermsQuery(getFieldName(), terms);
+    return new TermInSetQuery(getFieldName(), terms);
   }
 
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8511f9e6/solr/core/src/java/org/apache/solr/handler/component/ExpandComponent.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/component/ExpandComponent.java b/solr/core/src/java/org/apache/solr/handler/component/ExpandComponent.java
index 9222627..53ce387 100644
--- a/solr/core/src/java/org/apache/solr/handler/component/ExpandComponent.java
+++ b/solr/core/src/java/org/apache/solr/handler/component/ExpandComponent.java
@@ -26,6 +26,15 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 
+import com.carrotsearch.hppc.IntHashSet;
+import com.carrotsearch.hppc.IntObjectHashMap;
+import com.carrotsearch.hppc.LongHashSet;
+import com.carrotsearch.hppc.LongObjectHashMap;
+import com.carrotsearch.hppc.LongObjectMap;
+import com.carrotsearch.hppc.cursors.IntObjectCursor;
+import com.carrotsearch.hppc.cursors.LongCursor;
+import com.carrotsearch.hppc.cursors.LongObjectCursor;
+import com.carrotsearch.hppc.cursors.ObjectCursor;
 import org.apache.lucene.index.DocValues;
 import org.apache.lucene.index.DocValuesType;
 import org.apache.lucene.index.FieldInfo;
@@ -36,7 +45,6 @@ import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.MultiDocValues;
 import org.apache.lucene.index.NumericDocValues;
 import org.apache.lucene.index.SortedDocValues;
-import org.apache.lucene.queries.TermsQuery;
 import org.apache.lucene.search.BooleanClause.Occur;
 import org.apache.lucene.search.BooleanQuery;
 import org.apache.lucene.search.Collector;
@@ -46,6 +54,7 @@ import org.apache.lucene.search.Query;
 import org.apache.lucene.search.ScoreDoc;
 import org.apache.lucene.search.Scorer;
 import org.apache.lucene.search.Sort;
+import org.apache.lucene.search.TermInSetQuery;
 import org.apache.lucene.search.TopDocs;
 import org.apache.lucene.search.TopDocsCollector;
 import org.apache.lucene.search.TopFieldCollector;
@@ -83,16 +92,6 @@ import org.apache.solr.uninverting.UninvertingReader;
 import org.apache.solr.util.plugin.PluginInfoInitialized;
 import org.apache.solr.util.plugin.SolrCoreAware;
 
-import com.carrotsearch.hppc.IntHashSet;
-import com.carrotsearch.hppc.IntObjectHashMap;
-import com.carrotsearch.hppc.LongHashSet;
-import com.carrotsearch.hppc.LongObjectHashMap;
-import com.carrotsearch.hppc.LongObjectMap;
-import com.carrotsearch.hppc.cursors.IntObjectCursor;
-import com.carrotsearch.hppc.cursors.LongCursor;
-import com.carrotsearch.hppc.cursors.LongObjectCursor;
-import com.carrotsearch.hppc.cursors.ObjectCursor;
-
 /**
  * The ExpandComponent is designed to work with the CollapsingPostFilter.
  * The CollapsingPostFilter collapses a result set on a field.
@@ -676,7 +675,7 @@ public class ExpandComponent extends SearchComponent implements PluginInfoInitia
       bytesRefs[++index] = term.toBytesRef();
     }
 
-    return new SolrConstantScoreQuery(new QueryWrapperFilter(new TermsQuery(fname, bytesRefs)));
+    return new SolrConstantScoreQuery(new QueryWrapperFilter(new TermInSetQuery(fname, bytesRefs)));
   }
 
   private Query getGroupQuery(String fname,
@@ -689,7 +688,7 @@ public class ExpandComponent extends SearchComponent implements PluginInfoInitia
       IntObjectCursor<BytesRef> cursor = it.next();
       bytesRefs[++index] = cursor.value;
     }
-    return new SolrConstantScoreQuery(new QueryWrapperFilter(new TermsQuery(fname, bytesRefs)));
+    return new SolrConstantScoreQuery(new QueryWrapperFilter(new TermInSetQuery(fname, bytesRefs)));
   }
 
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8511f9e6/solr/core/src/java/org/apache/solr/schema/FieldType.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/schema/FieldType.java b/solr/core/src/java/org/apache/solr/schema/FieldType.java
index bf94d8b..4ba4106 100644
--- a/solr/core/src/java/org/apache/solr/schema/FieldType.java
+++ b/solr/core/src/java/org/apache/solr/schema/FieldType.java
@@ -39,7 +39,6 @@ import org.apache.lucene.document.Field;
 import org.apache.lucene.index.IndexOptions;
 import org.apache.lucene.index.IndexableField;
 import org.apache.lucene.index.Term;
-import org.apache.lucene.queries.TermsQuery;
 import org.apache.lucene.queries.function.ValueSource;
 import org.apache.lucene.search.BooleanClause;
 import org.apache.lucene.search.BooleanQuery;
@@ -51,6 +50,7 @@ import org.apache.lucene.search.Query;
 import org.apache.lucene.search.SortField;
 import org.apache.lucene.search.SortedNumericSelector;
 import org.apache.lucene.search.SortedSetSelector;
+import org.apache.lucene.search.TermInSetQuery;
 import org.apache.lucene.search.TermQuery;
 import org.apache.lucene.search.similarities.Similarity;
 import org.apache.lucene.util.BytesRef;
@@ -780,7 +780,7 @@ public abstract class FieldType extends FieldProperties {
       readableToIndexed(externalVal, br);
       lst.add( br.toBytesRef() );
     }
-    return new TermsQuery(field.getName() , lst);
+    return new TermInSetQuery(field.getName() , lst);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8511f9e6/solr/core/src/java/org/apache/solr/search/TermsQParserPlugin.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/TermsQParserPlugin.java b/solr/core/src/java/org/apache/solr/search/TermsQParserPlugin.java
index 728cc01..de771bb 100644
--- a/solr/core/src/java/org/apache/solr/search/TermsQParserPlugin.java
+++ b/solr/core/src/java/org/apache/solr/search/TermsQParserPlugin.java
@@ -20,13 +20,13 @@ import java.util.Arrays;
 import java.util.regex.Pattern;
 
 import org.apache.lucene.index.Term;
-import org.apache.lucene.queries.TermsQuery;
 import org.apache.lucene.search.AutomatonQuery;
 import org.apache.lucene.search.BooleanClause;
 import org.apache.lucene.search.BooleanQuery;
 import org.apache.lucene.search.DocValuesTermsQuery;
 import org.apache.lucene.search.MatchNoDocsQuery;
 import org.apache.lucene.search.Query;
+import org.apache.lucene.search.TermInSetQuery;
 import org.apache.lucene.search.TermQuery;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefBuilder;
@@ -60,7 +60,7 @@ public class TermsQParserPlugin extends QParserPlugin {
     termsFilter {
       @Override
       Filter makeFilter(String fname, BytesRef[] bytesRefs) {
-        return new QueryWrapperFilter(new TermsQuery(fname, bytesRefs));
+        return new QueryWrapperFilter(new TermInSetQuery(fname, bytesRefs));
       }
     },
     booleanQuery {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8511f9e6/solr/core/src/java/org/apache/solr/search/join/GraphQuery.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/join/GraphQuery.java b/solr/core/src/java/org/apache/solr/search/join/GraphQuery.java
index 3fca05a..58bbac8 100644
--- a/solr/core/src/java/org/apache/solr/search/join/GraphQuery.java
+++ b/solr/core/src/java/org/apache/solr/search/join/GraphQuery.java
@@ -25,7 +25,6 @@ import java.util.TreeSet;
 
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.Term;
-import org.apache.lucene.queries.TermsQuery;
 import org.apache.lucene.search.AutomatonQuery;
 import org.apache.lucene.search.BooleanClause.Occur;
 import org.apache.lucene.search.BooleanQuery;
@@ -35,6 +34,7 @@ import org.apache.lucene.search.Explanation;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.Scorer;
+import org.apache.lucene.search.TermInSetQuery;
 import org.apache.lucene.search.Weight;
 import org.apache.lucene.search.WildcardQuery;
 import org.apache.lucene.util.BytesRef;
@@ -291,7 +291,7 @@ public class GraphQuery extends Query {
             collectorTerms.get(i, ref);
             termList.add(ref);
           }
-          q = new TermsQuery(fromField, termList);
+          q = new TermInSetQuery(fromField, termList);
         }
         
         // If there is a filter to be used while crawling the graph, add that.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8511f9e6/solr/core/src/test/org/apache/solr/search/TestSolrQueryParser.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/search/TestSolrQueryParser.java b/solr/core/src/test/org/apache/solr/search/TestSolrQueryParser.java
index d3e6a7f..76b441b 100644
--- a/solr/core/src/test/org/apache/solr/search/TestSolrQueryParser.java
+++ b/solr/core/src/test/org/apache/solr/search/TestSolrQueryParser.java
@@ -19,12 +19,12 @@ package org.apache.solr.search;
 import java.util.Locale;
 import java.util.Random;
 
-import org.apache.lucene.queries.TermsQuery;
 import org.apache.lucene.search.BooleanClause;
 import org.apache.lucene.search.BooleanQuery;
 import org.apache.lucene.search.BoostQuery;
 import org.apache.lucene.search.ConstantScoreQuery;
 import org.apache.lucene.search.Query;
+import org.apache.lucene.search.TermInSetQuery;
 import org.apache.lucene.search.TermQuery;
 import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.core.SolrInfoMBean;
@@ -224,13 +224,13 @@ public class TestSolrQueryParser extends SolrTestCaseJ4 {
     qParser = QParser.getParser("foo_s:(a b c d e f g h i j k l m n o p q r s t u v w x y z)", req);
     qParser.setIsFilter(true); // this may change in the future
     q = qParser.getQuery();
-    assertEquals(26, ((TermsQuery)q).getTermData().size());
+    assertEquals(26, ((TermInSetQuery)q).getTermData().size());
 
     // large numeric filter query should use TermsQuery (for trie fields)
     qParser = QParser.getParser("foo_i:(1 2 3 4 5 6 7 8 9 10 20 19 18 17 16 15 14 13 12 11)", req);
     qParser.setIsFilter(true); // this may change in the future
     q = qParser.getQuery();
-    assertEquals(20, ((TermsQuery)q).getTermData().size());
+    assertEquals(20, ((TermInSetQuery)q).getTermData().size());
 
     // a filter() clause inside a relevancy query should be able to use a TermsQuery
     qParser = QParser.getParser("foo_s:aaa filter(foo_s:(a b c d e f g h i j k l m n o p q r s t u v w x y z))", req);
@@ -245,7 +245,7 @@ public class TestSolrQueryParser extends SolrTestCaseJ4 {
       qq = ((FilterQuery)qq).getQuery();
     }
 
-    assertEquals(26, ((TermsQuery)qq).getTermData().size());
+    assertEquals(26, ((TermInSetQuery)qq).getTermData().size());
 
     // test mixed boolean query, including quotes (which shouldn't matter)
     qParser = QParser.getParser("foo_s:(a +aaa b -bbb c d e f bar_s:(qqq www) g h i j k l m n o p q r s t u v w x y z)", req);
@@ -255,9 +255,9 @@ public class TestSolrQueryParser extends SolrTestCaseJ4 {
     qq = null;
     for (BooleanClause clause : ((BooleanQuery)q).clauses()) {
       qq = clause.getQuery();
-      if (qq instanceof TermsQuery) break;
+      if (qq instanceof TermInSetQuery) break;
     }
-    assertEquals(26, ((TermsQuery)qq).getTermData().size());
+    assertEquals(26, ((TermInSetQuery)qq).getTermData().size());
 
     req.close();
   }