You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by jb...@apache.org on 2015/01/12 16:01:07 UTC

svn commit: r1651109 [1/3] - in /lucene/dev/branches/branch_5x: ./ solr/ solr/core/ solr/core/src/java/org/apache/solr/handler/component/ solr/core/src/java/org/apache/solr/search/ solr/core/src/test/org/apache/solr/handler/component/ solr/core/src/tes...

Author: jbernste
Date: Mon Jan 12 15:01:06 2015
New Revision: 1651109

URL: http://svn.apache.org/r1651109
Log:
SOLR-6581: Efficient DocValues support and numeric collapse field implementations for Collapse and Expand

Modified:
    lucene/dev/branches/branch_5x/   (props changed)
    lucene/dev/branches/branch_5x/solr/   (props changed)
    lucene/dev/branches/branch_5x/solr/CHANGES.txt   (contents, props changed)
    lucene/dev/branches/branch_5x/solr/core/   (props changed)
    lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/handler/component/ExpandComponent.java
    lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/search/CollapsingQParserPlugin.java
    lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/handler/component/TestExpandComponent.java
    lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/search/TestCollapseQParserPlugin.java

Modified: lucene/dev/branches/branch_5x/solr/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/CHANGES.txt?rev=1651109&r1=1651108&r2=1651109&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/CHANGES.txt (original)
+++ lucene/dev/branches/branch_5x/solr/CHANGES.txt Mon Jan 12 15:01:06 2015
@@ -248,6 +248,9 @@ New Features
 
 * SOLR-6916: Toggle payload support for the default highlighter via hl.payloads.  It's auto
   enabled when the index has payloads. (David Smiley)
+
+* SOLR-6581: Efficient DocValues support and numeric collapse field implementations 
+  for Collapse and Expand (Joel Bernstein)
   
 
 Bug Fixes

Modified: lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/handler/component/ExpandComponent.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/handler/component/ExpandComponent.java?rev=1651109&r1=1651108&r2=1651109&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/handler/component/ExpandComponent.java (original)
+++ lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/handler/component/ExpandComponent.java Mon Jan 12 15:01:06 2015
@@ -24,11 +24,29 @@ import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Iterator;
+import java.util.Arrays;
 
+import com.carrotsearch.hppc.IntObjectOpenHashMap;
+import com.carrotsearch.hppc.LongOpenHashSet;
+import com.carrotsearch.hppc.LongObjectOpenHashMap;
+import com.carrotsearch.hppc.cursors.IntObjectCursor;
+import com.carrotsearch.hppc.cursors.LongCursor;
+import com.carrotsearch.hppc.cursors.LongObjectCursor;
+import com.carrotsearch.hppc.IntOpenHashSet;
+import com.carrotsearch.hppc.cursors.ObjectCursor;
+import com.carrotsearch.hppc.LongObjectMap;
 import org.apache.lucene.index.DocValues;
+import org.apache.lucene.index.DocValuesType;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.FieldInfos;
+import org.apache.lucene.index.FilterLeafReader;
 import org.apache.lucene.index.LeafReader;
 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.TermsFilter;
 import org.apache.lucene.search.Collector;
 import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.search.LeafCollector;
@@ -40,35 +58,37 @@ import org.apache.lucene.search.TopDocs;
 import org.apache.lucene.search.TopDocsCollector;
 import org.apache.lucene.search.TopFieldCollector;
 import org.apache.lucene.search.TopScoreDocCollector;
+import org.apache.lucene.uninverting.UninvertingReader;
 import org.apache.lucene.util.BitSetIterator;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefBuilder;
 import org.apache.lucene.util.CharsRefBuilder;
 import org.apache.lucene.util.FixedBitSet;
+import org.apache.lucene.util.LongValues;
 import org.apache.solr.common.SolrDocumentList;
 import org.apache.solr.common.params.ExpandParams;
-import org.apache.solr.common.params.ShardParams;
 import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.core.PluginInfo;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.schema.FieldType;
+import org.apache.solr.schema.TrieFloatField;
+import org.apache.solr.schema.TrieIntField;
+import org.apache.solr.schema.TrieLongField;
+import org.apache.solr.schema.TrieDoubleField;
+import org.apache.solr.schema.StrField;
 import org.apache.solr.search.CollapsingQParserPlugin;
 import org.apache.solr.search.DocIterator;
 import org.apache.solr.search.DocList;
 import org.apache.solr.search.DocSlice;
 import org.apache.solr.search.QParser;
 import org.apache.solr.search.QueryParsing;
+import org.apache.solr.search.SolrConstantScoreQuery;
 import org.apache.solr.search.SolrIndexSearcher;
 import org.apache.solr.util.plugin.PluginInfoInitialized;
 import org.apache.solr.util.plugin.SolrCoreAware;
 
-import com.carrotsearch.hppc.IntObjectMap;
-import com.carrotsearch.hppc.IntObjectOpenHashMap;
-import com.carrotsearch.hppc.IntOpenHashSet;
-import com.carrotsearch.hppc.cursors.IntObjectCursor;
-import com.carrotsearch.hppc.cursors.ObjectCursor;
-
 /**
  * The ExpandComponent is designed to work with the CollapsingPostFilter.
  * The CollapsingPostFilter collapses a result set on a field.
@@ -118,6 +138,7 @@ public class ExpandComponent extends Sea
     SolrParams params = req.getParams();
 
     String field = params.get(ExpandParams.EXPAND_FIELD);
+    String hint = null;
     if (field == null) {
       List<Query> filters = rb.getFilters();
       if (filters != null) {
@@ -125,6 +146,7 @@ public class ExpandComponent extends Sea
           if (q instanceof CollapsingQParserPlugin.CollapsingPostFilter) {
             CollapsingQParserPlugin.CollapsingPostFilter cp = (CollapsingQParserPlugin.CollapsingPostFilter) q;
             field = cp.getField();
+            hint = cp.hint;
           }
         }
       }
@@ -183,26 +205,168 @@ public class ExpandComponent extends Sea
 
     SolrIndexSearcher searcher = req.getSearcher();
     LeafReader reader = searcher.getLeafReader();
-    SortedDocValues values = DocValues.getSorted(reader, field);
-    FixedBitSet groupBits = new FixedBitSet(values.getValueCount());
+
+    FieldType fieldType = searcher.getSchema().getField(field).getType();
+
+    SortedDocValues values = null;
+    long nullValue = 0;
+
+    if(fieldType instanceof StrField) {
+      //Get The Top Level SortedDocValues
+      if(CollapsingQParserPlugin.HINT_TOP_FC.equals(hint)) {
+        Map<String, UninvertingReader.Type> mapping = new HashMap();
+        mapping.put(field, UninvertingReader.Type.SORTED);
+        UninvertingReader uninvertingReader = new UninvertingReader(new ReaderWrapper(searcher.getLeafReader(), field), mapping);
+        values = uninvertingReader.getSortedDocValues(field);
+      } else {
+        values = DocValues.getSorted(reader, field);
+      }
+    } else {
+      //Get the nullValue for the numeric collapse field
+      String defaultValue = searcher.getSchema().getField(field).getDefaultValue();
+      if(defaultValue != null) {
+        if(fieldType instanceof TrieIntField || fieldType instanceof TrieLongField) {
+          nullValue = Long.parseLong(defaultValue);
+        } else if(fieldType instanceof TrieFloatField){
+          nullValue = Float.floatToIntBits(Float.parseFloat(defaultValue));
+        } else if(fieldType instanceof TrieDoubleField){
+          nullValue = Double.doubleToLongBits(Double.parseDouble(defaultValue));
+        }
+      } else {
+        if(fieldType instanceof TrieFloatField){
+          nullValue = Float.floatToIntBits(0.0f);
+        } else if(fieldType instanceof TrieDoubleField){
+          nullValue = Double.doubleToLongBits(0.0f);
+        }
+      }
+    }
+
+    FixedBitSet groupBits = null;
+    LongOpenHashSet groupSet = null;
     DocList docList = rb.getResults().docList;
     IntOpenHashSet collapsedSet = new IntOpenHashSet(docList.size() * 2);
 
+    //Gather the groups for the current page of documents
     DocIterator idit = docList.iterator();
-
+    int[] globalDocs = new int[docList.size()];
+    int docsIndex = -1;
     while (idit.hasNext()) {
-      int doc = idit.nextDoc();
-      int ord = values.getOrd(doc);
-      if (ord > -1) {
-        groupBits.set(ord);
-        collapsedSet.add(doc);
+      globalDocs[++docsIndex] = idit.nextDoc();
+    }
+
+    Arrays.sort(globalDocs);
+    Query groupQuery = null;
+
+    /*
+    * This code gathers the group information for the current page.
+    */
+    List<LeafReaderContext> contexts = searcher.getTopReaderContext().leaves();
+    int currentContext = 0;
+    int currentDocBase = contexts.get(currentContext).docBase;
+    int nextDocBase = (currentContext+1)<contexts.size() ? contexts.get(currentContext+1).docBase : Integer.MAX_VALUE;
+    IntObjectOpenHashMap<BytesRef> ordBytes = null;
+    if(values != null) {
+      groupBits = new FixedBitSet(values.getValueCount());
+      MultiDocValues.OrdinalMap ordinalMap = null;
+      SortedDocValues[] sortedDocValues = null;
+      LongValues segmentOrdinalMap = null;
+      SortedDocValues currentValues = null;
+      if(values instanceof  MultiDocValues.MultiSortedDocValues) {
+        ordinalMap = ((MultiDocValues.MultiSortedDocValues)values).mapping;
+        sortedDocValues = ((MultiDocValues.MultiSortedDocValues)values).values;
+        currentValues = sortedDocValues[currentContext];
+        segmentOrdinalMap = ordinalMap.getGlobalOrds(currentContext);
+      }
+      int count = 0;
+
+      ordBytes = new IntObjectOpenHashMap();
+
+      for(int i=0; i<globalDocs.length; i++) {
+        int globalDoc = globalDocs[i];
+        while(globalDoc >= nextDocBase) {
+          currentContext++;
+          currentDocBase = contexts.get(currentContext).docBase;
+          nextDocBase = (currentContext+1) < contexts.size() ? contexts.get(currentContext+1).docBase : Integer.MAX_VALUE;
+          if(ordinalMap != null) {
+            currentValues = sortedDocValues[currentContext];
+            segmentOrdinalMap = ordinalMap.getGlobalOrds(currentContext);
+          }
+        }
+
+        int contextDoc = globalDoc - currentDocBase;
+        if(ordinalMap != null) {
+          int ord = currentValues.getOrd(contextDoc);
+          if(ord > -1) {
+            ++count;
+            BytesRef ref = currentValues.lookupOrd(ord);
+            ord = (int)segmentOrdinalMap.get(ord);
+            ordBytes.put(ord, BytesRef.deepCopyOf(ref));
+            groupBits.set(ord);
+            collapsedSet.add(globalDoc);
+          }
+        } else {
+          int ord = values.getOrd(globalDoc);
+          if(ord > -1) {
+            ++count;
+            BytesRef ref = values.lookupOrd(ord);
+            ordBytes.put(ord, BytesRef.deepCopyOf(ref));
+            groupBits.set(ord);
+            collapsedSet.add(globalDoc);
+          }
+        }
+      }
+
+      if(count > 0 && count < 200) {
+        try {
+          groupQuery = getGroupQuery(field, count, ordBytes);
+        } catch(Exception e) {
+          throw new IOException(e);
+        }
+      }
+    } else {
+      groupSet = new LongOpenHashSet((int)(docList.size()*1.25));
+      NumericDocValues collapseValues = contexts.get(currentContext).reader().getNumericDocValues(field);
+      int count = 0;
+      for(int i=0; i<globalDocs.length; i++) {
+        int globalDoc = globalDocs[i];
+        while(globalDoc >= nextDocBase) {
+          currentContext++;
+          currentDocBase = contexts.get(currentContext).docBase;
+          nextDocBase = currentContext+1 < contexts.size() ? contexts.get(currentContext+1).docBase : Integer.MAX_VALUE;
+          collapseValues = contexts.get(currentContext).reader().getNumericDocValues(field);
+        }
+        int contextDoc = globalDoc - currentDocBase;
+        long value = collapseValues.get(contextDoc);
+        if(value != nullValue) {
+          ++count;
+          groupSet.add(value);
+          collapsedSet.add(globalDoc);
+        }
+      }
+
+      if(count > 0 && count < 200) {
+        groupQuery = getGroupQuery(field, fieldType, count, groupSet);
       }
     }
 
     Collector collector;
     if (sort != null)
       sort = sort.rewrite(searcher);
-    GroupExpandCollector groupExpandCollector = new GroupExpandCollector(values, groupBits, collapsedSet, limit, sort);
+
+
+    Collector groupExpandCollector = null;
+
+    if(values != null) {
+      groupExpandCollector = new GroupExpandCollector(values, groupBits, collapsedSet, limit, sort);
+    } else {
+      groupExpandCollector = new NumericGroupExpandCollector(field, nullValue, groupSet, collapsedSet, limit, sort);
+    }
+
+    if(groupQuery !=  null) {
+      //Limits the results to documents that are in the same group as the documents in the page.
+      newFilters.add(groupQuery);
+    }
+
     SolrIndexSearcher.ProcessedFilter pfilter = searcher.getProcessedFilter(null, newFilters);
     if (pfilter.postFilter != null) {
       pfilter.postFilter.setLastDelegate(groupExpandCollector);
@@ -212,12 +376,11 @@ public class ExpandComponent extends Sea
     }
 
     searcher.search(query, pfilter.filter, collector);
-    IntObjectMap groups = groupExpandCollector.getGroups();
+    LongObjectMap groups = ((GroupCollector)groupExpandCollector).getGroups();
     Map<String, DocSlice> outMap = new HashMap<>();
     CharsRefBuilder charsRef = new CharsRefBuilder();
-    FieldType fieldType = searcher.getSchema().getField(field).getType();
-    for (IntObjectCursor cursor : (Iterable<IntObjectCursor>) groups) {
-      int ord = cursor.key;
+    for (LongObjectCursor cursor : (Iterable<LongObjectCursor>) groups) {
+      long groupValue = cursor.key;
       TopDocsCollector topDocsCollector = (TopDocsCollector) cursor.value;
       TopDocs topDocs = topDocsCollector.topDocs();
       ScoreDoc[] scoreDocs = topDocs.scoreDocs;
@@ -230,10 +393,21 @@ public class ExpandComponent extends Sea
           scores[i] = scoreDoc.score;
         }
         DocSlice slice = new DocSlice(0, docs.length, docs, scores, topDocs.totalHits, topDocs.getMaxScore());
-        final BytesRef bytesRef = values.lookupOrd(ord);
-        fieldType.indexedToReadable(bytesRef, charsRef);
-        String group = charsRef.toString();
-        outMap.put(group, slice);
+
+        if(fieldType instanceof StrField) {
+          final BytesRef bytesRef = ordBytes.get((int)groupValue);
+          fieldType.indexedToReadable(bytesRef, charsRef);
+          String group = charsRef.toString();
+          outMap.put(group, slice);
+        } else {
+          if(fieldType instanceof TrieIntField || fieldType instanceof TrieLongField ) {
+            outMap.put(Long.toString(groupValue), slice);
+          } else if(fieldType instanceof TrieFloatField) {
+            outMap.put(Float.toString(Float.intBitsToFloat((int)groupValue)), slice);
+          } else if(fieldType instanceof TrieDoubleField) {
+            outMap.put(Double.toString(Double.longBitsToDouble(groupValue)), slice);
+          }
+        }
       }
     }
 
@@ -306,16 +480,20 @@ public class ExpandComponent extends Sea
     rb.rsp.add("expanded", expanded);
   }
 
-  private class GroupExpandCollector implements Collector {
+  private class GroupExpandCollector implements Collector, GroupCollector {
     private SortedDocValues docValues;
-    private IntObjectMap<Collector> groups;
-    private int docBase;
+    private MultiDocValues.OrdinalMap ordinalMap;
+    private SortedDocValues segmentValues;
+    private LongValues segmentOrdinalMap;
+    private MultiDocValues.MultiSortedDocValues multiSortedDocValues;
+
+    private LongObjectMap<Collector> groups;
     private FixedBitSet groupBits;
     private IntOpenHashSet collapsedSet;
 
     public GroupExpandCollector(SortedDocValues docValues, FixedBitSet groupBits, IntOpenHashSet collapsedSet, int limit, Sort sort) throws IOException {
       int numGroups = collapsedSet.size();
-      groups = new IntObjectOpenHashMap<>(numGroups * 2);
+      groups = new LongObjectOpenHashMap<>(numGroups * 2);
       DocIdSetIterator iterator = new BitSetIterator(groupBits, 0); // cost is not useful here
       int group;
       while ((group = iterator.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
@@ -326,12 +504,22 @@ public class ExpandComponent extends Sea
       this.collapsedSet = collapsedSet;
       this.groupBits = groupBits;
       this.docValues = docValues;
+      if(docValues instanceof MultiDocValues.MultiSortedDocValues) {
+        this.multiSortedDocValues = (MultiDocValues.MultiSortedDocValues)docValues;
+        this.ordinalMap = multiSortedDocValues.mapping;
+      }
     }
 
     public LeafCollector getLeafCollector(LeafReaderContext context) throws IOException {
       final int docBase = context.docBase;
-      final IntObjectMap<LeafCollector> leafCollectors = new IntObjectOpenHashMap<>();
-      for (IntObjectCursor<Collector> entry : groups) {
+
+      if(ordinalMap != null) {
+        this.segmentValues = this.multiSortedDocValues.values[context.ord];
+        this.segmentOrdinalMap = ordinalMap.getGlobalOrds(context.ord);
+      }
+
+      final LongObjectMap<LeafCollector> leafCollectors = new LongObjectOpenHashMap<>();
+      for (LongObjectCursor<Collector> entry : groups) {
         leafCollectors.put(entry.key, entry.value.getLeafCollector(context));
       }
       return new LeafCollector() {
@@ -345,9 +533,18 @@ public class ExpandComponent extends Sea
 
         @Override
         public void collect(int docId) throws IOException {
-          int doc = docId + docBase;
-          int ord = docValues.getOrd(doc);
-          if (ord > -1 && groupBits.get(ord) && !collapsedSet.contains(doc)) {
+          int globalDoc = docId + docBase;
+          int ord = -1;
+          if(ordinalMap != null) {
+            ord = segmentValues.getOrd(docId);
+            if(ord > -1) {
+              ord = (int)segmentOrdinalMap.get(ord);
+            }
+          } else {
+            ord = docValues.getOrd(globalDoc);
+          }
+
+          if (ord > -1 && groupBits.get(ord) && !collapsedSet.contains(globalDoc)) {
             LeafCollector c = leafCollectors.get(ord);
             c.collect(docId);
           }
@@ -360,12 +557,125 @@ public class ExpandComponent extends Sea
       };
     }
 
-    public IntObjectMap<Collector> getGroups() {
+    public LongObjectMap<Collector> getGroups() {
+      return groups;
+    }
+  }
+
+  private class NumericGroupExpandCollector implements Collector, GroupCollector {
+    private NumericDocValues docValues;
+
+
+    private String field;
+    private LongObjectOpenHashMap<Collector> groups;
+
+    private IntOpenHashSet collapsedSet;
+    private long nullValue;
+
+    public NumericGroupExpandCollector(String field, long nullValue, LongOpenHashSet groupSet, IntOpenHashSet collapsedSet, int limit, Sort sort) throws IOException {
+      int numGroups = collapsedSet.size();
+      this.nullValue = nullValue;
+      groups = new LongObjectOpenHashMap(numGroups * 2);
+      Iterator<LongCursor> iterator = groupSet.iterator();
+      while (iterator.hasNext()) {
+        LongCursor cursor = iterator.next();
+        Collector collector = (sort == null) ? TopScoreDocCollector.create(limit, true) : TopFieldCollector.create(sort, limit, false, false, false, true);
+        groups.put(cursor.value, collector);
+      }
+
+      this.field = field;
+      this.collapsedSet = collapsedSet;
+    }
+
+    public LeafCollector getLeafCollector(LeafReaderContext context) throws IOException {
+      final int docBase = context.docBase;
+      this.docValues = context.reader().getNumericDocValues(this.field);
+
+      final LongObjectOpenHashMap<LeafCollector> leafCollectors = new LongObjectOpenHashMap<>();
+
+      for (LongObjectCursor<Collector> entry : groups) {
+        leafCollectors.put(entry.key, entry.value.getLeafCollector(context));
+      }
+
+      return new LeafCollector() {
+
+        @Override
+        public void setScorer(Scorer scorer) throws IOException {
+          for (ObjectCursor<LeafCollector> c : leafCollectors.values()) {
+            c.value.setScorer(scorer);
+          }
+        }
+
+        @Override
+        public void collect(int docId) throws IOException {
+          long value = docValues.get(docId);
+          if (value != nullValue && leafCollectors.containsKey(value) && !collapsedSet.contains(docId + docBase)) {
+            LeafCollector c = leafCollectors.lget();
+            c.collect(docId);
+          }
+        }
+
+        @Override
+        public boolean acceptsDocsOutOfOrder() {
+          return false;
+        }
+      };
+    }
+
+    public LongObjectOpenHashMap<Collector> getGroups() {
       return groups;
     }
 
   }
 
+  private interface GroupCollector {
+    public LongObjectMap getGroups();
+
+  }
+
+  private Query getGroupQuery(String fname,
+                           FieldType ft,
+                           int size,
+                           LongOpenHashSet groupSet) {
+
+    BytesRef[] bytesRefs = new BytesRef[size];
+    BytesRefBuilder term = new BytesRefBuilder();
+    Iterator<LongCursor> it = groupSet.iterator();
+    int index = -1;
+    String stringVal =  null;
+    while (it.hasNext()) {
+      LongCursor cursor = it.next();
+      if(ft instanceof TrieIntField || ft instanceof TrieLongField) {
+        stringVal = Long.toString(cursor.value);
+      } else {
+        if(ft instanceof TrieFloatField) {
+          stringVal = Float.toString(Float.intBitsToFloat((int)cursor.value));
+        } else {
+          stringVal = Double.toString(Double.longBitsToDouble(cursor.value));
+        }
+      }
+      ft.readableToIndexed(stringVal, term);
+      bytesRefs[++index] = term.toBytesRef();
+    }
+
+    return new SolrConstantScoreQuery(new TermsFilter(fname, bytesRefs));
+  }
+
+  private Query getGroupQuery(String fname,
+                              int size,
+                              IntObjectOpenHashMap<BytesRef> ordBytes) throws Exception {
+
+    BytesRef[] bytesRefs = new BytesRef[size];
+    int index = -1;
+    Iterator<IntObjectCursor<BytesRef>>it = ordBytes.iterator();
+    while (it.hasNext()) {
+      IntObjectCursor<BytesRef> cursor = it.next();
+      bytesRefs[++index] = cursor.value;
+    }
+    return new SolrConstantScoreQuery(new TermsFilter(fname, bytesRefs));
+  }
+
+
   ////////////////////////////////////////////
   ///  SolrInfoMBean
   ////////////////////////////////////////////
@@ -385,4 +695,49 @@ public class ExpandComponent extends Sea
       throw new RuntimeException(e);
     }
   }
+
+  private class ReaderWrapper extends FilterLeafReader {
+
+    private String field;
+
+    public ReaderWrapper(LeafReader leafReader, String field) {
+      super(leafReader);
+      this.field = field;
+    }
+
+    public SortedDocValues getSortedDocValues(String field) {
+      return null;
+    }
+
+    public Object getCoreCacheKey() {
+      return in.getCoreCacheKey();
+    }
+
+    public FieldInfos getFieldInfos() {
+      Iterator<FieldInfo> it = in.getFieldInfos().iterator();
+      List<FieldInfo> newInfos = new ArrayList();
+      while(it.hasNext()) {
+        FieldInfo fieldInfo = it.next();
+
+        if(fieldInfo.name.equals(field)) {
+          FieldInfo f = new FieldInfo(fieldInfo.name,
+              fieldInfo.number,
+              fieldInfo.hasVectors(),
+              fieldInfo.hasNorms(),
+              fieldInfo.hasPayloads(),
+              fieldInfo.getIndexOptions(),
+              DocValuesType.NONE,
+              fieldInfo.getDocValuesGen(),
+              fieldInfo.attributes());
+          newInfos.add(f);
+
+        } else {
+          newInfos.add(fieldInfo);
+        }
+      }
+      FieldInfos infos = new FieldInfos(newInfos.toArray(new FieldInfo[newInfos.size()]));
+      return infos;
+    }
+  }
+
 }