You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by si...@apache.org on 2011/05/02 15:51:22 UTC

svn commit: r1098566 [19/22] - in /lucene/dev/branches/docvalues: ./ dev-tools/eclipse/ dev-tools/idea/.idea/ dev-tools/idea/lucene/contrib/ant/ dev-tools/idea/lucene/contrib/db/bdb-je/ dev-tools/idea/lucene/contrib/db/bdb/ dev-tools/idea/lucene/contri...

Modified: lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/core/QuerySenderListener.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/core/QuerySenderListener.java?rev=1098566&r1=1098565&r2=1098566&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/core/QuerySenderListener.java (original)
+++ lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/core/QuerySenderListener.java Mon May  2 13:50:57 2011
@@ -32,7 +32,7 @@ import java.util.List;
 /**
  * @version $Id$
  */
-class QuerySenderListener extends AbstractSolrEventListener {
+public class QuerySenderListener extends AbstractSolrEventListener {
   public QuerySenderListener(SolrCore core) {
     super(core);
   }

Modified: lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/core/RequestHandlers.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/core/RequestHandlers.java?rev=1098566&r1=1098565&r2=1098566&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/core/RequestHandlers.java (original)
+++ lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/core/RequestHandlers.java Mon May  2 13:50:57 2011
@@ -17,25 +17,28 @@
 
 package org.apache.solr.core;
 
+import java.net.URL;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrException.ErrorCode;
+import org.apache.solr.common.params.CommonParams;
+import org.apache.solr.common.params.CommonParams.EchoParamStyle;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.SimpleOrderedMap;
+import org.apache.solr.handler.component.SearchHandler;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.request.SolrRequestHandler;
 import org.apache.solr.response.SolrQueryResponse;
-import org.apache.solr.util.plugin.SolrCoreAware;
 import org.apache.solr.util.plugin.PluginInfoInitialized;
+import org.apache.solr.util.plugin.SolrCoreAware;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.net.URL;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.LinkedHashMap;
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
-
 /**
  */
 final class RequestHandlers {

Modified: lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/handler/AnalysisRequestHandlerBase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/handler/AnalysisRequestHandlerBase.java?rev=1098566&r1=1098565&r2=1098566&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/handler/AnalysisRequestHandlerBase.java (original)
+++ lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/handler/AnalysisRequestHandlerBase.java Mon May  2 13:50:57 2011
@@ -337,6 +337,7 @@ public abstract class AnalysisRequestHan
     @Override
     public boolean incrementToken() throws IOException {
       if (tokenIterator.hasNext()) {
+        clearAttributes();
         AttributeSource next = tokenIterator.next();
         Iterator<Class<? extends Attribute>> atts = next.getAttributeClassesIterator();
         while (atts.hasNext()) // make sure all att impls in the token exist here

Modified: lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/handler/ContentStreamHandlerBase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/handler/ContentStreamHandlerBase.java?rev=1098566&r1=1098565&r2=1098566&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/handler/ContentStreamHandlerBase.java (original)
+++ lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/handler/ContentStreamHandlerBase.java Mon May  2 13:50:57 2011
@@ -24,6 +24,8 @@ import org.apache.solr.request.SolrQuery
 import org.apache.solr.response.SolrQueryResponse;
 import org.apache.solr.update.processor.UpdateRequestProcessor;
 import org.apache.solr.update.processor.UpdateRequestProcessorChain;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 
 /**
@@ -31,12 +33,22 @@ import org.apache.solr.update.processor.
  *
  **/
 public abstract class ContentStreamHandlerBase extends RequestHandlerBase {
+  public static Logger log = LoggerFactory.getLogger(XmlUpdateRequestHandler.class);
 
   @Override
   public void handleRequestBody(SolrQueryRequest req, SolrQueryResponse rsp) throws Exception {
     SolrParams params = req.getParams();
+    String updateChainName = null;
+    if(params.get(UpdateParams.UPDATE_CHAIN_DEPRECATED) != null) {
+    	log.warn("Use of deprecated update request parameter "+UpdateParams.UPDATE_CHAIN_DEPRECATED+
+    			 " detected. Please use the new parameter "+UpdateParams.UPDATE_CHAIN+" instead, as support"+
+    			 " for "+UpdateParams.UPDATE_CHAIN_DEPRECATED+" will be removed in a later version.");
+    	updateChainName = params.get(UpdateParams.UPDATE_CHAIN_DEPRECATED);
+    } else {
+    	updateChainName = params.get(UpdateParams.UPDATE_CHAIN);
+    }
     UpdateRequestProcessorChain processorChain =
-            req.getCore().getUpdateProcessingChain(params.get(UpdateParams.UPDATE_PROCESSOR));
+            req.getCore().getUpdateProcessingChain(updateChainName);
 
     UpdateRequestProcessor processor = processorChain.createProcessor(req, rsp);
 

Modified: lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/handler/ReplicationHandler.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/handler/ReplicationHandler.java?rev=1098566&r1=1098565&r2=1098566&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/handler/ReplicationHandler.java (original)
+++ lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/handler/ReplicationHandler.java Mon May  2 13:50:57 2011
@@ -843,9 +843,13 @@ public class ReplicationHandler extends 
                 indexCommitPoint = reader.getIndexCommit();
               }
             } finally {
+              // We don't need to save commit points for replication, the SolrDeletionPolicy
+              // always saves the last commit point (and the last optimized commit point, if needed)
+              /***
               if(indexCommitPoint != null){
                 core.getDeletionPolicy().saveCommitPoint(indexCommitPoint.getVersion());
               }
+              ***/
             }
           }
           if (core.getUpdateHandler() instanceof DirectUpdateHandler2) {

Modified: lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/handler/admin/CoreAdminHandler.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/handler/admin/CoreAdminHandler.java?rev=1098566&r1=1098565&r2=1098566&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/handler/admin/CoreAdminHandler.java (original)
+++ lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/handler/admin/CoreAdminHandler.java Mon May  2 13:50:57 2011
@@ -190,7 +190,7 @@ public class CoreAdminHandler extends Re
         }
 
         UpdateRequestProcessorChain processorChain =
-                core.getUpdateProcessingChain(params.get(UpdateParams.UPDATE_PROCESSOR));
+                core.getUpdateProcessingChain(params.get(UpdateParams.UPDATE_CHAIN));
         wrappedReq = new LocalSolrQueryRequest(core, req.getParams());
         UpdateRequestProcessor processor =
                 processorChain.createProcessor(wrappedReq, rsp);

Modified: lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/handler/component/FacetComponent.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/handler/component/FacetComponent.java?rev=1098566&r1=1098565&r2=1098566&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/handler/component/FacetComponent.java (original)
+++ lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/handler/component/FacetComponent.java Mon May  2 13:50:57 2011
@@ -17,23 +17,23 @@
 
 package org.apache.solr.handler.component;
 
-import java.io.IOException;
-import java.net.URL;
-import java.util.*;
-
+import org.apache.lucene.queryParser.ParseException;
+import org.apache.lucene.util.OpenBitSet;
+import org.apache.solr.common.SolrException;
 import org.apache.solr.common.params.CommonParams;
 import org.apache.solr.common.params.FacetParams;
-import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.SimpleOrderedMap;
 import org.apache.solr.common.util.StrUtils;
-import org.apache.solr.common.SolrException;
 import org.apache.solr.request.SimpleFacets;
-import org.apache.lucene.util.OpenBitSet;
-import org.apache.solr.search.QueryParsing;
 import org.apache.solr.schema.FieldType;
-import org.apache.lucene.queryParser.ParseException;
+import org.apache.solr.search.QueryParsing;
+
+import java.io.IOException;
+import java.net.URL;
+import java.util.*;
 
 /**
  * TODO!
@@ -222,11 +222,37 @@ public class FacetComponent extends Sear
           sreq.params.remove(paramStart + FacetParams.FACET_MINCOUNT);
           sreq.params.remove(paramStart + FacetParams.FACET_OFFSET);
 
-          dff.initialLimit = dff.offset + dff.limit;
+          dff.initialLimit = dff.limit <= 0 ? dff.limit : dff.offset + dff.limit;
 
-          if(dff.sort.equals(FacetParams.FACET_SORT_COUNT) && dff.limit > 0) {
-            // set the initial limit higher to increase accuracy
-            dff.initialLimit = (int)(dff.initialLimit * 1.5) + 10;
+          if (dff.sort.equals(FacetParams.FACET_SORT_COUNT)) {
+            if (dff.limit > 0) {
+              // set the initial limit higher to increase accuracy
+              dff.initialLimit = (int)(dff.initialLimit * 1.5) + 10;
+              dff.initialMincount = 0;      // TODO: we could change this to 1, but would then need more refinement for small facet result sets?
+            } else {
+              // if limit==-1, then no need to artificially lower mincount to 0 if it's 1
+              dff.initialMincount = Math.min(dff.minCount, 1);
+            }
+          } else {
+            // we're sorting by index order.
+            // if minCount==0, we should always be able to get accurate results w/o over-requesting or refining
+            // if minCount==1, we should be able to get accurate results w/o over-requesting, but we'll need to refine
+            // if minCount==n (>1), we can set the initialMincount to minCount/nShards, rounded up.
+            // For example, we know that if minCount=10 and we have 3 shards, then at least one shard must have a count of 4 for the term
+            // For the minCount>1 case, we can generate too short of a list (miss terms at the end of the list) unless limit==-1
+            // For example: each shard could produce a list of top 10, but some of those could fail to make it into the combined list (i.e.
+            //   we needed to go beyond the top 10 to generate the top 10 combined).  Overrequesting can help a little here, but not as
+            //   much as when sorting by count.
+            if (dff.minCount <= 1) {
+              dff.initialMincount = dff.minCount;
+            } else {
+              dff.initialMincount = (int)Math.ceil((double)dff.minCount / rb.slices.length);
+              // dff.initialMincount = 1;
+            }
+          }
+
+          if (dff.initialMincount != 0) {
+            sreq.params.set(paramStart + FacetParams.FACET_MINCOUNT, dff.initialMincount);
           }
 
           // Currently this is for testing only and allows overriding of the
@@ -286,8 +312,95 @@ public class FacetComponent extends Sear
           dff.add(shardNum, (NamedList)facet_fields.get(dff.getKey()), dff.initialLimit);
         }
       }
-    }
 
+      // Distributed facet_dates
+      //
+      // The implementation below uses the first encountered shard's 
+      // facet_dates as the basis for subsequent shards' data to be merged.
+      // (the "NOW" param should ensure consistency)
+      @SuppressWarnings("unchecked")
+      SimpleOrderedMap<SimpleOrderedMap<Object>> facet_dates = 
+        (SimpleOrderedMap<SimpleOrderedMap<Object>>) 
+        facet_counts.get("facet_dates");
+      
+      if (facet_dates != null) {
+
+        // go through each facet_date
+        for (Map.Entry<String,SimpleOrderedMap<Object>> entry : facet_dates) {
+          final String field = entry.getKey();
+          if (fi.dateFacets.get(field) == null) { 
+            // first time we've seen this field, no merging
+            fi.dateFacets.add(field, entry.getValue());
+
+          } else { 
+            // not the first time, merge current field
+
+            SimpleOrderedMap<Object> shardFieldValues 
+              = entry.getValue();
+            SimpleOrderedMap<Object> existFieldValues 
+              = fi.dateFacets.get(field);
+
+            for (Map.Entry<String,Object> existPair : existFieldValues) {
+              final String key = existPair.getKey();
+              if (key.equals("gap") || 
+                  key.equals("end") || 
+                  key.equals("start")) {
+                // we can skip these, must all be the same across shards
+                continue; 
+              }
+              // can be null if inconsistencies in shards responses
+              Integer newValue = (Integer) shardFieldValues.get(key);
+              if  (null != newValue) {
+                Integer oldValue = ((Integer) existPair.getValue());
+                existPair.setValue(oldValue + newValue);
+              }
+            }
+          }
+        }
+      }
+
+      // Distributed facet_ranges
+      //
+      // The implementation below uses the first encountered shard's 
+      // facet_ranges as the basis for subsequent shards' data to be merged.
+      @SuppressWarnings("unchecked")
+      SimpleOrderedMap<SimpleOrderedMap<Object>> facet_ranges = 
+        (SimpleOrderedMap<SimpleOrderedMap<Object>>) 
+        facet_counts.get("facet_ranges");
+      
+      if (facet_ranges != null) {
+
+        // go through each facet_range
+        for (Map.Entry<String,SimpleOrderedMap<Object>> entry : facet_ranges) {
+          final String field = entry.getKey();
+          if (fi.rangeFacets.get(field) == null) { 
+            // first time we've seen this field, no merging
+            fi.rangeFacets.add(field, entry.getValue());
+
+          } else { 
+            // not the first time, merge current field counts
+
+            @SuppressWarnings("unchecked")
+            NamedList<Integer> shardFieldValues 
+              = (NamedList<Integer>) entry.getValue().get("counts");
+
+            @SuppressWarnings("unchecked")
+            NamedList<Integer> existFieldValues 
+              = (NamedList<Integer>) fi.rangeFacets.get(field).get("counts");
+
+            for (Map.Entry<String,Integer> existPair : existFieldValues) {
+              final String key = existPair.getKey();
+              // can be null if inconsistencies in shards responses
+              Integer newValue = shardFieldValues.get(key);
+              if  (null != newValue) {
+                Integer oldValue = existPair.getValue();
+                existPair.setValue(oldValue + newValue);
+              }
+            }
+          }
+        }
+      }
+    }
 
     //
     // This code currently assumes that there will be only a single
@@ -296,15 +409,18 @@ public class FacetComponent extends Sear
     //
 
     for (DistribFieldFacet dff : fi.facets.values()) {
-      if (dff.limit <= 0) continue; // no need to check these facets for refinement
-      if (dff.minCount <= 1 && dff.sort.equals(FacetParams.FACET_SORT_INDEX)) continue;
+       // no need to check these facets for refinement
+      if (dff.initialLimit <= 0 && dff.initialMincount == 0) continue;
 
-      @SuppressWarnings("unchecked") // generic array's are anoying
+      // only other case where index-sort doesn't need refinement is if minCount==0
+      if (dff.minCount == 0 && dff.sort.equals(FacetParams.FACET_SORT_INDEX)) continue;
+
+      @SuppressWarnings("unchecked") // generic array's are annoying
       List<String>[] tmp = (List<String>[]) new List[rb.shards.length];
       dff._toRefine = tmp;
 
       ShardFacetCount[] counts = dff.getCountSorted();
-      int ntop = Math.min(counts.length, dff.offset + dff.limit);
+      int ntop = Math.min(counts.length, dff.limit >= 0 ? dff.offset + dff.limit : Integer.MAX_VALUE);
       long smallestCount = counts.length == 0 ? 0 : counts[ntop-1].count;
 
       for (int i=0; i<counts.length; i++) {
@@ -313,8 +429,11 @@ public class FacetComponent extends Sear
 
         if (i<ntop) {
           // automatically flag the top values for refinement
+          // this should always be true for facet.sort=index
           needRefinement = true;
         } else {
+          // this logic should only be invoked for facet.sort=index (for now)
+
           // calculate the maximum value that this term may have
           // and if it is >= smallestCount, then flag for refinement
           long maxCount = sfc.count;
@@ -422,13 +541,32 @@ public class FacetComponent extends Sear
           counts = dff.getLexSorted();
       }
 
-      int end = dff.limit < 0 ? counts.length : Math.min(dff.offset + dff.limit, counts.length);
-      for (int i=dff.offset; i<end; i++) {
-        if (counts[i].count < dff.minCount) {
-          if (countSorted) break;  // if sorted by count, we can break out of loop early
-          else continue;
+      if (countSorted) {
+        int end = dff.limit < 0 ? counts.length : Math.min(dff.offset + dff.limit, counts.length);
+        for (int i=dff.offset; i<end; i++) {
+          if (counts[i].count < dff.minCount) {
+            break;
+          }
+          fieldCounts.add(counts[i].name, num(counts[i].count));
+        }
+      } else {
+        int off = dff.offset;
+        int lim = dff.limit >= 0 ? dff.limit : Integer.MAX_VALUE;
+
+        // index order...
+        for (int i=0; i<counts.length; i++) {
+          long count = counts[i].count;
+          if (count < dff.minCount) continue;
+          if (off > 0) {
+            off--;
+            continue;
+          }
+          if (lim <= 0) {
+            break;
+          }
+          lim--;
+          fieldCounts.add(counts[i].name, num(count));
         }
-        fieldCounts.add(counts[i].name, num(counts[i].count));
       }
 
       if (dff.missing) {
@@ -436,9 +574,8 @@ public class FacetComponent extends Sear
       }
     }
 
-    // TODO: facet dates & numbers
-    facet_counts.add("facet_dates", new SimpleOrderedMap());
-    facet_counts.add("facet_ranges", new SimpleOrderedMap());
+    facet_counts.add("facet_dates", fi.dateFacets);
+    facet_counts.add("facet_ranges", fi.rangeFacets);
 
     rb.rsp.add("facet_counts", facet_counts);
 
@@ -490,8 +627,14 @@ public class FacetComponent extends Sear
    * <b>This API is experimental and subject to change</b>
    */
   public static class FacetInfo {
+
     public LinkedHashMap<String,QueryFacet> queryFacets;
     public LinkedHashMap<String,DistribFieldFacet> facets;
+    public SimpleOrderedMap<SimpleOrderedMap<Object>> dateFacets
+      = new SimpleOrderedMap<SimpleOrderedMap<Object>>();
+    public SimpleOrderedMap<SimpleOrderedMap<Object>> rangeFacets
+      = new SimpleOrderedMap<SimpleOrderedMap<Object>>();
+
     public List<String> exceptionList;
 
     void parse(SolrParams params, ResponseBuilder rb) {
@@ -631,7 +774,8 @@ public class FacetComponent extends Sear
     public HashMap<String,ShardFacetCount> counts = new HashMap<String,ShardFacetCount>(128);
     public int termNum;
 
-    public int initialLimit;  // how many terms requested in first phase
+    public int initialLimit;     // how many terms requested in first phase
+    public int initialMincount;  // mincount param sent to each shard
     public boolean needRefinements;
     public ShardFacetCount[] countSorted;
 
@@ -671,11 +815,10 @@ public class FacetComponent extends Sear
         }
       }
 
-      // the largest possible missing term is 0 if we received less
-      // than the number requested (provided mincount==0 like it should be for
-      // a shard request)
+      // the largest possible missing term is initialMincount if we received less
+      // than the number requested.
       if (numRequested<0 || numRequested != 0 && numReceived < numRequested) {
-        last = 0;
+        last = initialMincount;
       }
 
       missingMaxPossible += last;

Modified: lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/handler/component/ResponseBuilder.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/handler/component/ResponseBuilder.java?rev=1098566&r1=1098565&r2=1098566&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/handler/component/ResponseBuilder.java (original)
+++ lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/handler/component/ResponseBuilder.java Mon May  2 13:50:57 2011
@@ -163,6 +163,25 @@ public class ResponseBuilder
     debugInfo.add( name, val );
   }
 
+  public void addDebug(Object val, String... path) {
+    if( debugInfo == null ) {
+      debugInfo = new SimpleOrderedMap<Object>();
+    }
+
+    NamedList<Object> target = debugInfo;
+    for (int i=0; i<path.length-1; i++) {
+      String elem = path[i];
+      NamedList<Object> newTarget = (NamedList<Object>)debugInfo.get(elem);
+      if (newTarget == null) {
+        newTarget = new SimpleOrderedMap<Object>();
+        target.add(elem, newTarget);
+      }
+      target = newTarget;
+    }
+
+    target.add(path[path.length-1], val);
+  }
+
   //-------------------------------------------------------------------------
   //-------------------------------------------------------------------------
 

Modified: lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/handler/component/SpellCheckComponent.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/handler/component/SpellCheckComponent.java?rev=1098566&r1=1098565&r2=1098566&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/handler/component/SpellCheckComponent.java (original)
+++ lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/handler/component/SpellCheckComponent.java Mon May  2 13:50:57 2011
@@ -162,7 +162,7 @@ public class SpellCheckComponent extends
 
       } else {
         throw new SolrException(SolrException.ErrorCode.NOT_FOUND,
-            "Specified dictionary does not exist.");
+            "Specified dictionary does not exist: " + getDictionaryName(params));
       }
     }
   }

Modified: lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/highlight/DefaultSolrHighlighter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/highlight/DefaultSolrHighlighter.java?rev=1098566&r1=1098565&r2=1098566&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/highlight/DefaultSolrHighlighter.java (original)
+++ lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/highlight/DefaultSolrHighlighter.java Mon May  2 13:50:57 2011
@@ -435,12 +435,20 @@ public class DefaultSolrHighlighter exte
         // fall back to analyzer
         tstream = createAnalyzerTStream(schema, fieldName, docTexts[j]);
       }
-                   
+      
+      int maxCharsToAnalyze = params.getFieldInt(fieldName,
+          HighlightParams.MAX_CHARS,
+          Highlighter.DEFAULT_MAX_CHARS_TO_ANALYZE);
+      
       Highlighter highlighter;
       if (Boolean.valueOf(req.getParams().get(HighlightParams.USE_PHRASE_HIGHLIGHTER, "true"))) {
         // TODO: this is not always necessary - eventually we would like to avoid this wrap
         //       when it is not needed.
-        tstream = new CachingTokenFilter(tstream);
+        if (maxCharsToAnalyze < 0) {
+          tstream = new CachingTokenFilter(tstream);
+        } else {
+          tstream = new CachingTokenFilter(new OffsetLimitTokenFilter(tstream, maxCharsToAnalyze));
+        }
         
         // get highlighter
         highlighter = getPhraseHighlighter(query, fieldName, req, (CachingTokenFilter) tstream);
@@ -453,9 +461,6 @@ public class DefaultSolrHighlighter exte
         highlighter = getHighlighter(query, fieldName, req);
       }
       
-      int maxCharsToAnalyze = params.getFieldInt(fieldName,
-          HighlightParams.MAX_CHARS,
-          Highlighter.DEFAULT_MAX_CHARS_TO_ANALYZE);
       if (maxCharsToAnalyze < 0) {
         highlighter.setMaxDocCharsToAnalyze(docTexts[j].length());
       } else {

Modified: lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/request/SimpleFacets.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/request/SimpleFacets.java?rev=1098566&r1=1098565&r2=1098566&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/request/SimpleFacets.java (original)
+++ lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/request/SimpleFacets.java Mon May  2 13:50:57 2011
@@ -21,6 +21,7 @@ import org.apache.lucene.index.*;
 import org.apache.lucene.queryParser.ParseException;
 import org.apache.lucene.search.*;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.StringHelper;
 import org.apache.lucene.util.packed.Direct16;
 import org.apache.lucene.util.packed.Direct32;
 import org.apache.lucene.util.packed.Direct8;
@@ -682,14 +683,15 @@ public class SimpleFacets {
 
             if (deState==null) {
               deState = new SolrIndexSearcher.DocsEnumState();
+              deState.fieldName = StringHelper.intern(field);
               deState.deletedDocs = MultiFields.getDeletedDocs(r);
               deState.termsEnum = termsEnum;
-              deState.reuse = docsEnum;
+              deState.docsEnum = docsEnum;
             }
 
-            c = searcher.numDocs(new TermQuery(t), docs, deState);
+            c = searcher.numDocs(docs, deState);
 
-            docsEnum = deState.reuse;
+            docsEnum = deState.docsEnum;
           } else {
             // iterate over TermDocs to calculate the intersection
 

Modified: lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/request/SolrRequestInfo.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/request/SolrRequestInfo.java?rev=1098566&r1=1098565&r2=1098566&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/request/SolrRequestInfo.java (original)
+++ lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/request/SolrRequestInfo.java Mon May  2 13:50:57 2011
@@ -17,11 +17,15 @@
 
 package org.apache.solr.request;
 
+import org.apache.solr.common.SolrException;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.handler.component.ResponseBuilder;
 import org.apache.solr.response.SolrQueryResponse;
 
+import java.io.Closeable;
 import java.util.Date;
+import java.util.LinkedList;
+import java.util.List;
 
 
 public class SolrRequestInfo {
@@ -31,6 +35,8 @@ public class SolrRequestInfo {
   protected SolrQueryResponse rsp;
   protected Date now;
   protected ResponseBuilder rb;
+  protected List<Closeable> closeHooks;
+
 
   public static SolrRequestInfo getRequestInfo() {
     return threadLocal.get();
@@ -48,7 +54,20 @@ public class SolrRequestInfo {
   }
 
   public static void clearRequestInfo() {
-    threadLocal.remove();
+    try {
+      SolrRequestInfo info = threadLocal.get();
+      if (info != null && info.closeHooks != null) {
+        for (Closeable hook : info.closeHooks) {
+          try {
+            hook.close();
+          } catch (Throwable throwable) {
+            SolrException.log(SolrCore.log, "Exception during close hook", throwable);
+          }
+        }
+      }
+    } finally {
+      threadLocal.remove();
+    }
   }
 
   public SolrRequestInfo(SolrQueryRequest req, SolrQueryResponse rsp) {
@@ -88,4 +107,14 @@ public class SolrRequestInfo {
   public void setResponseBuilder(ResponseBuilder rb) {
     this.rb = rb;
   }
+
+  public void addCloseHook(Closeable hook) {
+    // is this better here, or on SolrQueryRequest?
+    synchronized (this) {
+      if (closeHooks == null) {
+        closeHooks = new LinkedList<Closeable>();
+      }
+      closeHooks.add(hook);
+    }
+  }
 }

Modified: lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/request/UnInvertedField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/request/UnInvertedField.java?rev=1098566&r1=1098565&r2=1098566&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/request/UnInvertedField.java (original)
+++ lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/request/UnInvertedField.java Mon May  2 13:50:57 2011
@@ -18,16 +18,12 @@
 package org.apache.solr.request;
 
 import org.apache.lucene.search.FieldCache;
-import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.DocTermOrds;
 import org.apache.lucene.index.Term;
-import org.apache.lucene.index.DocsEnum;
-import org.apache.lucene.index.DocsAndPositionsEnum;
 import org.apache.lucene.index.TermsEnum;
-import org.apache.lucene.index.Terms;
-import org.apache.lucene.index.MultiFields;
 import org.apache.lucene.search.TermQuery;
 import org.apache.lucene.search.TermRangeQuery;
-import org.apache.lucene.util.PagedBytes;
+import org.apache.lucene.util.StringHelper;
 import org.apache.noggit.CharArr;
 import org.apache.solr.common.params.FacetParams;
 import org.apache.solr.common.util.NamedList;
@@ -44,15 +40,11 @@ import org.apache.solr.handler.component
 import org.apache.solr.handler.component.FieldFacetStats;
 import org.apache.lucene.util.OpenBitSet;
 import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.Bits;
 
 import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.HashMap;
 import java.util.LinkedHashMap;
 import java.util.Map;
-import java.util.Comparator;
 
 import java.util.concurrent.atomic.AtomicLong;
 
@@ -86,7 +78,7 @@ import java.util.concurrent.atomic.Atomi
  *   much like Lucene's own internal term index).
  *
  */
-public class UnInvertedField {
+public class UnInvertedField extends DocTermOrds {
   private static int TNUM_OFFSET=2;
 
   static class TopTerm {
@@ -100,362 +92,113 @@ public class UnInvertedField {
     }
   }
 
-  String field;
-  int numTermsInField;
-  int termsInverted;  // number of unique terms that were un-inverted
-  long termInstances; // total number of references to term numbers
-  final TermIndex ti;
   long memsz;
-  int total_time;  // total time to uninvert the field
-  int phase1_time;  // time for phase1 of the uninvert process
   final AtomicLong use = new AtomicLong(); // number of uses
 
-  int[] index;
-  byte[][] tnums = new byte[256][];
-  int[] maxTermCounts;
+  int[] maxTermCounts = new int[1024];
+
   final Map<Integer,TopTerm> bigTerms = new LinkedHashMap<Integer,TopTerm>();
 
+  private SolrIndexSearcher.DocsEnumState deState;
+  private final SolrIndexSearcher searcher;
+
+  @Override
+  protected void visitTerm(TermsEnum te, int termNum) throws IOException {
+
+    if (termNum >= maxTermCounts.length) {
+      // resize by doubling - for very large number of unique terms, expanding
+      // by 4K and resultant GC will dominate uninvert times.  Resize at end if material
+      int[] newMaxTermCounts = new int[maxTermCounts.length*2];
+      System.arraycopy(maxTermCounts, 0, newMaxTermCounts, 0, termNum);
+      maxTermCounts = newMaxTermCounts;
+    }
+
+    final BytesRef term = te.term();
+
+    if (te.docFreq() > maxTermDocFreq) {
+      TopTerm topTerm = new TopTerm();
+      topTerm.term = new BytesRef(term);
+      topTerm.termNum = termNum;
+      bigTerms.put(topTerm.termNum, topTerm);
+
+      if (deState == null) {
+        deState = new SolrIndexSearcher.DocsEnumState();
+        deState.fieldName = StringHelper.intern(field);
+        // deState.termsEnum = te.tenum;
+        deState.termsEnum = te;  // TODO: check for MultiTermsEnum in SolrIndexSearcher could now fail?
+        deState.docsEnum = docsEnum;
+        deState.minSetSizeCached = maxTermDocFreq;
+      }
+      docsEnum = deState.docsEnum;
+      DocSet set = searcher.getDocSet(deState);
+      maxTermCounts[termNum] = set.size();
+    }
+  }
+
+  @Override
+  protected void setActualDocFreq(int termNum, int docFreq) {
+    maxTermCounts[termNum] = docFreq;
+  }
 
   public long memSize() {
     // can cache the mem size since it shouldn't change
     if (memsz!=0) return memsz;
-    long sz = 8*8 + 32; // local fields
+    long sz = super.ramUsedInBytes();
+    sz += 8*8 + 32; // local fields
     sz += bigTerms.size() * 64;
     for (TopTerm tt : bigTerms.values()) {
       sz += tt.memSize();
     }
-    if (index != null) sz += index.length * 4;
-    if (tnums!=null) {
-      for (byte[] arr : tnums)
-        if (arr != null) sz += arr.length;
-    }
     if (maxTermCounts != null)
       sz += maxTermCounts.length * 4;
-    sz += ti.memSize();
+    if (indexedTermsArray != null) {
+      // assume 8 byte references?
+      sz += 8+8+8+8+(indexedTermsArray.length<<3)+sizeOfIndexedStrings;
+    }
     memsz = sz;
     return sz;
   }
 
-
-  /** Number of bytes to represent an unsigned int as a vint. */
-  static int vIntSize(int x) {
-    if ((x & (0xffffffff << (7*1))) == 0 ) {
-      return 1;
-    }
-    if ((x & (0xffffffff << (7*2))) == 0 ) {
-      return 2;
-    }
-    if ((x & (0xffffffff << (7*3))) == 0 ) {
-      return 3;
-    }
-    if ((x & (0xffffffff << (7*4))) == 0 ) {
-      return 4;
-    }
-    return 5;
-  }
-
-
-  // todo: if we know the size of the vInt already, we could do
-  // a single switch on the size
-  static int writeInt(int x, byte[] arr, int pos) {
-    int a;
-    a = (x >>> (7*4));
-    if (a != 0) {
-      arr[pos++] = (byte)(a | 0x80);
-    }
-    a = (x >>> (7*3));
-    if (a != 0) {
-      arr[pos++] = (byte)(a | 0x80);
-    }
-    a = (x >>> (7*2));
-    if (a != 0) {
-      arr[pos++] = (byte)(a | 0x80);
-    }
-    a = (x >>> (7*1));
-    if (a != 0) {
-      arr[pos++] = (byte)(a | 0x80);
-    }
-    arr[pos++] = (byte)(x & 0x7f);
-    return pos;
-  }
-
-
-
   public UnInvertedField(String field, SolrIndexSearcher searcher) throws IOException {
-    this.field = field;
-    this.ti = new TermIndex(field,
-            TrieField.getMainValuePrefix(searcher.getSchema().getFieldType(field)));
-    uninvert(searcher);
-  }
-
-
-  private void uninvert(SolrIndexSearcher searcher) throws IOException {
-    long startTime = System.currentTimeMillis();
-
-    IndexReader reader = searcher.getIndexReader();
-    int maxDoc = reader.maxDoc();
-
-    int[] index = new int[maxDoc];       // immediate term numbers, or the index into the byte[] representing the last number
-    this.index = index;
-    final int[] lastTerm = new int[maxDoc];    // last term we saw for this document
-    final byte[][] bytes = new byte[maxDoc][]; // list of term numbers for the doc (delta encoded vInts)
-    maxTermCounts = new int[1024];
-
-    NumberedTermsEnum te = ti.getEnumerator(reader);
-
-    // threshold, over which we use set intersections instead of counting
-    // to (1) save memory, and (2) speed up faceting.
-    // Add 2 for testing purposes so that there will always be some terms under
-    // the threshold even when the index is very small.
-    int threshold = maxDoc / 20 + 2;
-    // threshold = 2000000000; //////////////////////////////// USE FOR TESTING
-
-    // we need a minimum of 9 bytes, but round up to 12 since the space would
-    // be wasted with most allocators anyway.
-    byte[] tempArr = new byte[12];
-
-    //
-    // enumerate all terms, and build an intermediate form of the un-inverted field.
-    //
-    // During this intermediate form, every document has a (potential) byte[]
-    // and the int[maxDoc()] array either contains the termNumber list directly
-    // or the *end* offset of the termNumber list in it's byte array (for faster
-    // appending and faster creation of the final form).
-    //
-    // idea... if things are too large while building, we could do a range of docs
-    // at a time (but it would be a fair amount slower to build)
-    // could also do ranges in parallel to take advantage of multiple CPUs
-
-    // OPTIONAL: remap the largest df terms to the lowest 128 (single byte)
-    // values.  This requires going over the field first to find the most
-    // frequent terms ahead of time.
-
-    SolrIndexSearcher.DocsEnumState deState = null;
-
-    for (;;) {
-      BytesRef t = te.term();
-      if (t==null) break;
-
-      int termNum = te.getTermNumber();
-
-      if (termNum >= maxTermCounts.length) {
-        // resize by doubling - for very large number of unique terms, expanding
-        // by 4K and resultant GC will dominate uninvert times.  Resize at end if material
-        int[] newMaxTermCounts = new int[maxTermCounts.length*2];
-        System.arraycopy(maxTermCounts, 0, newMaxTermCounts, 0, termNum);
-        maxTermCounts = newMaxTermCounts;
-      }
-
-      int df = te.docFreq();
-      if (df >= threshold) {
-        TopTerm topTerm = new TopTerm();
-        topTerm.term = new BytesRef(t);
-        topTerm.termNum = termNum;
-        bigTerms.put(topTerm.termNum, topTerm);
-
-        if (deState == null) {
-          deState = new SolrIndexSearcher.DocsEnumState();
-          deState.termsEnum = te.tenum;
-          deState.reuse = te.docsEnum;
-        }
-        DocSet set = searcher.getDocSet(new TermQuery(new Term(ti.field, topTerm.term)), deState);
-        te.docsEnum = deState.reuse;
-
-        maxTermCounts[termNum] = set.size();
-
-        te.next();
-        continue;
-      }
-
-      termsInverted++;
-
-      DocsEnum docsEnum = te.getDocsEnum();
-
-      DocsEnum.BulkReadResult bulkResult = docsEnum.getBulkResult();
-
-      for(;;) {
-        int n = docsEnum.read();
-        if (n <= 0) break;
-
-        maxTermCounts[termNum] += n;
-
-        for (int i=0; i<n; i++) {
-          termInstances++;
-          int doc = bulkResult.docs.ints[i];
-          // add 2 to the term number to make room for special reserved values:
-          // 0 (end term) and 1 (index into byte array follows)
-          int delta = termNum - lastTerm[doc] + TNUM_OFFSET;
-          lastTerm[doc] = termNum;
-          int val = index[doc];
-
-          if ((val & 0xff)==1) {
-            // index into byte array (actually the end of
-            // the doc-specific byte[] when building)
-            int pos = val >>> 8;
-            int ilen = vIntSize(delta);
-            byte[] arr = bytes[doc];
-            int newend = pos+ilen;
-            if (newend > arr.length) {
-              // We avoid a doubling strategy to lower memory usage.
-              // this faceting method isn't for docs with many terms.
-              // In hotspot, objects have 2 words of overhead, then fields, rounded up to a 64-bit boundary.
-              // TODO: figure out what array lengths we can round up to w/o actually using more memory
-              // (how much space does a byte[] take up?  Is data preceded by a 32 bit length only?
-              // It should be safe to round up to the nearest 32 bits in any case.
-              int newLen = (newend + 3) & 0xfffffffc;  // 4 byte alignment
-              byte[] newarr = new byte[newLen];
-              System.arraycopy(arr, 0, newarr, 0, pos);
-              arr = newarr;
-              bytes[doc] = newarr;
-            }
-            pos = writeInt(delta, arr, pos);
-            index[doc] = (pos<<8) | 1;  // update pointer to end index in byte[]
-          } else {
-            // OK, this int has data in it... find the end (a zero starting byte - not
-            // part of another number, hence not following a byte with the high bit set).
-            int ipos;
-            if (val==0) {
-              ipos=0;
-            } else if ((val & 0x0000ff80)==0) {
-              ipos=1;
-            } else if ((val & 0x00ff8000)==0) {
-              ipos=2;
-            } else if ((val & 0xff800000)==0) {
-              ipos=3;
-            } else {
-              ipos=4;
-            }
-
-            int endPos = writeInt(delta, tempArr, ipos);
-            if (endPos <= 4) {
-              // value will fit in the integer... move bytes back
-              for (int j=ipos; j<endPos; j++) {
-                val |= (tempArr[j] & 0xff) << (j<<3);
-              }
-              index[doc] = val;
-            } else {
-              // value won't fit... move integer into byte[]
-              for (int j=0; j<ipos; j++) {
-                tempArr[j] = (byte)val;
-                val >>>=8;
-              }
-              // point at the end index in the byte[]
-              index[doc] = (endPos<<8) | 1;
-              bytes[doc] = tempArr;
-              tempArr = new byte[12];
-            }
-
-          }
-
+    super(field,
+          // threshold, over which we use set intersections instead of counting
+          // to (1) save memory, and (2) speed up faceting.
+          // Add 2 for testing purposes so that there will always be some terms under
+          // the threshold even when the index is very
+          // small.
+          searcher.maxDoc()/20 + 2,
+          DEFAULT_INDEX_INTERVAL_BITS);
+    //System.out.println("maxTermDocFreq=" + maxTermDocFreq + " maxDoc=" + searcher.maxDoc());
+
+    final String prefix = TrieField.getMainValuePrefix(searcher.getSchema().getFieldType(field));
+    this.searcher = searcher;
+    try {
+      uninvert(searcher.getIndexReader(), prefix == null ? null : new BytesRef(prefix));
+    } catch (IllegalStateException ise) {
+      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, ise.getMessage());
+    }
+    if (tnums != null) {
+      for(byte[] target : tnums) {
+        if (target != null && target.length > (1<<24)*.9) {
+          SolrCore.log.warn("Approaching too many values for UnInvertedField faceting on field '"+field+"' : bucket size=" + target.length);
         }
-
       }
-
-      te.next();
     }
 
-    numTermsInField = te.getTermNumber();
-    te.close();
-
     // free space if outrageously wasteful (tradeoff memory/cpu) 
-
     if ((maxTermCounts.length - numTermsInField) > 1024) { // too much waste!
       int[] newMaxTermCounts = new int[numTermsInField];
       System.arraycopy(maxTermCounts, 0, newMaxTermCounts, 0, numTermsInField);
       maxTermCounts = newMaxTermCounts;
-   }
-
-    long midPoint = System.currentTimeMillis();
-
-    if (termInstances == 0) {
-      // we didn't invert anything
-      // lower memory consumption.
-      index = this.index = null;
-      tnums = null;
-    } else {
-
-      //
-      // transform intermediate form into the final form, building a single byte[]
-      // at a time, and releasing the intermediate byte[]s as we go to avoid
-      // increasing the memory footprint.
-      //
-      for (int pass = 0; pass<256; pass++) {
-        byte[] target = tnums[pass];
-        int pos=0;  // end in target;
-        if (target != null) {
-          pos = target.length;
-        } else {
-          target = new byte[4096];
-        }
-
-        // loop over documents, 0x00ppxxxx, 0x01ppxxxx, 0x02ppxxxx
-        // where pp is the pass (which array we are building), and xx is all values.
-        // each pass shares the same byte[] for termNumber lists.
-        for (int docbase = pass<<16; docbase<maxDoc; docbase+=(1<<24)) {
-          int lim = Math.min(docbase + (1<<16), maxDoc);
-          for (int doc=docbase; doc<lim; doc++) {
-            int val = index[doc];
-            if ((val&0xff) == 1) {
-              int len = val >>> 8;
-              index[doc] = (pos<<8)|1; // change index to point to start of array
-              if ((pos & 0xff000000) != 0) {
-                // we only have 24 bits for the array index
-                throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Too many values for UnInvertedField faceting on field "+field);
-              }
-              byte[] arr = bytes[doc];
-              bytes[doc] = null;        // IMPORTANT: allow GC to avoid OOM
-              if (target.length <= pos + len) {
-                int newlen = target.length;
-                /*** we don't have to worry about the array getting too large
-                 * since the "pos" param will overflow first (only 24 bits available)
-                if ((newlen<<1) <= 0) {
-                  // overflow...
-                  newlen = Integer.MAX_VALUE;
-                  if (newlen <= pos + len) {
-                    throw new SolrException(400,"Too many terms to uninvert field!");
-                  }
-                } else {
-                  while (newlen <= pos + len) newlen<<=1;  // doubling strategy
-                }
-                ****/
-                while (newlen <= pos + len) newlen<<=1;  // doubling strategy                 
-                byte[] newtarget = new byte[newlen];
-                System.arraycopy(target, 0, newtarget, 0, pos);
-                target = newtarget;
-              }
-              System.arraycopy(arr, 0, target, pos, len);
-              pos += len + 1;  // skip single byte at end and leave it 0 for terminator
-            }
-          }
-        }
-
-        // shrink array
-        if (pos < target.length) {
-          byte[] newtarget = new byte[pos];
-          System.arraycopy(target, 0, newtarget, 0, pos);
-          target = newtarget;
-          if (target.length > (1<<24)*.9) {
-            SolrCore.log.warn("Approaching too many values for UnInvertedField faceting on field '"+field+"' : bucket size=" + target.length);
-          }
-        }
-        
-        tnums[pass] = target;
-
-        if ((pass << 16) > maxDoc)
-          break;
-      }
     }
 
-    long endTime = System.currentTimeMillis();
-
-    total_time = (int)(endTime-startTime);
-    phase1_time = (int)(midPoint-startTime);
-
     SolrCore.log.info("UnInverted multi-valued field " + toString());
+    //System.out.println("CREATED: " + toString() + " ti.index=" + ti.index);
   }
 
-
-
+  public int getNumTerms() {
+    return numTermsInField;
+  }
 
   public NamedList<Integer> getCounts(SolrIndexSearcher searcher, DocSet baseDocs, int offset, int limit, Integer mincount, boolean missing, String sort, String prefix) throws IOException {
     use.incrementAndGet();
@@ -468,6 +211,7 @@ public class UnInvertedField {
     int baseSize = docs.size();
     int maxDoc = searcher.maxDoc();
 
+    //System.out.println("GET COUNTS field=" + field + " baseSize=" + baseSize + " minCount=" + mincount + " maxDoc=" + maxDoc + " numTermsInField=" + numTermsInField);
     if (baseSize >= mincount) {
 
       final int[] index = this.index;
@@ -481,14 +225,20 @@ public class UnInvertedField {
       int startTerm = 0;
       int endTerm = numTermsInField;  // one past the end
 
-      NumberedTermsEnum te = ti.getEnumerator(searcher.getIndexReader());
+      TermsEnum te = getOrdTermsEnum(searcher.getIndexReader());
       if (prefix != null && prefix.length() > 0) {
         BytesRef prefixBr = new BytesRef(prefix);
-        te.skipTo(prefixBr);
-        startTerm = te.getTermNumber();
+        if (te.seek(prefixBr, true) == TermsEnum.SeekStatus.END) {
+          startTerm = numTermsInField;
+        } else {
+          startTerm = (int) te.ord();
+        }
         prefixBr.append(ByteUtils.bigTerm);
-        te.skipTo(prefixBr);
-        endTerm = te.getTermNumber();
+        if (te.seek(prefixBr, true) == TermsEnum.SeekStatus.END) {
+          endTerm = numTermsInField;
+        } else {
+          endTerm = (int) te.ord();
+        }
       }
 
       /***********
@@ -514,13 +264,18 @@ public class UnInvertedField {
         docs = new BitDocSet(bs, maxDoc - baseSize);
         // simply negating will mean that we have deleted docs in the set.
         // that should be OK, as their entries in our table should be empty.
+        //System.out.println("  NEG");
       }
 
       // For the biggest terms, do straight set intersections
       for (TopTerm tt : bigTerms.values()) {
+        //System.out.println("  do big termNum=" + tt.termNum + " term=" + tt.term.utf8ToString());
         // TODO: counts could be deferred if sorted==false
         if (tt.termNum >= startTerm && tt.termNum < endTerm) {
-          counts[tt.termNum] = searcher.numDocs(new TermQuery(new Term(ti.field, tt.term)), docs);
+          counts[tt.termNum] = searcher.numDocs(new TermQuery(new Term(field, tt.term)), docs);
+          //System.out.println("    count=" + counts[tt.termNum]);
+        } else {
+          //System.out.println("SKIP term=" + tt.termNum);
         }
       }
 
@@ -537,9 +292,11 @@ public class UnInvertedField {
         DocIterator iter = docs.iterator();
         while (iter.hasNext()) {
           int doc = iter.nextDoc();
+          //System.out.println("iter doc=" + doc);
           int code = index[doc];
 
           if ((code & 0xff)==1) {
+            //System.out.println("  ptr");
             int pos = code>>>8;
             int whichArray = (doc >>> 16) & 0xff;
             byte[] arr = tnums[whichArray];
@@ -553,9 +310,11 @@ public class UnInvertedField {
               }
               if (delta == 0) break;
               tnum += delta - TNUM_OFFSET;
+              //System.out.println("    tnum=" + tnum);
               counts[tnum]++;
             }
           } else {
+            //System.out.println("  inlined");
             int tnum = 0;
             int delta = 0;
             for (;;) {
@@ -563,6 +322,7 @@ public class UnInvertedField {
               if ((code & 0x80)==0) {
                 if (delta==0) break;
                 tnum += delta - TNUM_OFFSET;
+                //System.out.println("    tnum=" + tnum);
                 counts[tnum]++;
                 delta = 0;
               }
@@ -583,6 +343,7 @@ public class UnInvertedField {
         LongPriorityQueue queue = new LongPriorityQueue(Math.min(maxsize,1000), maxsize, Long.MIN_VALUE);
 
         int min=mincount-1;  // the smallest value in the top 'N' values
+        //System.out.println("START=" + startTerm + " END=" + endTerm);
         for (int i=startTerm; i<endTerm; i++) {
           int c = doNegative ? maxTermCounts[i] - counts[i] : counts[i];
           if (c>min) {
@@ -641,11 +402,14 @@ public class UnInvertedField {
           }
         });
 
-        // convert the term numbers to term values and set as the label
+        // convert the term numbers to term values and set
+        // as the label
+        //System.out.println("sortStart=" + sortedIdxStart + " end=" + sortedIdxEnd);
         for (int i=sortedIdxStart; i<sortedIdxEnd; i++) {
           int idx = indirect[i];
           int tnum = (int)sorted[idx];
           String label = getReadableValue(getTermValue(te, tnum), ft, spare);
+          //System.out.println("  label=" + label);
           res.setName(idx - sortedIdxStart, label);
         }
 
@@ -668,8 +432,6 @@ public class UnInvertedField {
           res.add(label, c);
         }
       }
-
-      te.close();
     }
 
 
@@ -678,6 +440,8 @@ public class UnInvertedField {
       res.add(null, SimpleFacets.getFieldMissingCount(searcher, baseDocs, field));
     }
 
+    //System.out.println("  res=" + res);
+
     return res;
   }
 
@@ -731,8 +495,7 @@ public class UnInvertedField {
     final int[] index = this.index;
     final int[] counts = new int[numTermsInField];//keep track of the number of times we see each word in the field for all the documents in the docset
 
-    NumberedTermsEnum te = ti.getEnumerator(searcher.getIndexReader());
-
+    TermsEnum te = getOrdTermsEnum(searcher.getIndexReader());
 
     boolean doNegative = false;
     if (finfo.length == 0) {
@@ -755,7 +518,7 @@ public class UnInvertedField {
     for (TopTerm tt : bigTerms.values()) {
       // TODO: counts could be deferred if sorted==false
       if (tt.termNum >= 0 && tt.termNum < numTermsInField) {
-        final Term t = new Term(ti.field, tt.term);
+        final Term t = new Term(field, tt.term);
         if (finfo.length == 0) {
           counts[tt.termNum] = searcher.numDocs(new TermQuery(t), docs);
         } else {
@@ -836,7 +599,6 @@ public class UnInvertedField {
         f.accumulateTermNum(i, value);
       }
     }
-    te.close();
 
     int c = missing.size();
     allstats.addMissing(c);
@@ -870,23 +632,26 @@ public class UnInvertedField {
   }
 
   /** may return a reused BytesRef */
-  BytesRef getTermValue(NumberedTermsEnum te, int termNum) throws IOException {
+  BytesRef getTermValue(TermsEnum te, int termNum) throws IOException {
+    //System.out.println("getTermValue termNum=" + termNum + " this=" + this + " numTerms=" + numTermsInField);
     if (bigTerms.size() > 0) {
       // see if the term is one of our big terms.
       TopTerm tt = bigTerms.get(termNum);
       if (tt != null) {
+        //System.out.println("  return big " + tt.term);
         return tt.term;
       }
     }
 
-    return te.skipTo(termNum);
+    return lookupTerm(te, termNum);
   }
 
   @Override
   public String toString() {
+    final long indexSize = indexedTermsArray == null ? 0 : (8+8+8+8+(indexedTermsArray.length<<3)+sizeOfIndexedStrings); // assume 8 byte references?
     return "{field=" + field
             + ",memSize="+memSize()
-            + ",tindexSize="+ti.memSize()
+            + ",tindexSize="+indexSize
             + ",time="+total_time
             + ",phase1="+phase1_time
             + ",nTerms="+numTermsInField
@@ -896,7 +661,6 @@ public class UnInvertedField {
             + "}";
   }
 
-
   //////////////////////////////////////////////////////////////////
   //////////////////////////// caching /////////////////////////////
   //////////////////////////////////////////////////////////////////
@@ -920,287 +684,3 @@ public class UnInvertedField {
     return uif;
   }
 }
-
-
-// How to share TermDocs (int[] score[])???
-// Hot to share TermPositions?
-/***
-class TermEnumListener {
-  void doTerm(Term t) {
-  }
-  void done() {
-  }
-}
-***/
-
-
-class NumberedTermsEnum extends TermsEnum {
-  protected final IndexReader reader;
-  protected final TermIndex tindex;
-  protected TermsEnum tenum;
-  protected int pos=-1;
-  protected BytesRef termText;
-  protected DocsEnum docsEnum;
-  protected Bits deletedDocs;
-
-
-  NumberedTermsEnum(IndexReader reader, TermIndex tindex) throws IOException {
-    this.reader = reader;
-    this.tindex = tindex;
-  }
-
-
-  NumberedTermsEnum(IndexReader reader, TermIndex tindex, BytesRef termValue, int pos) throws IOException {
-    this.reader = reader;
-    this.tindex = tindex;
-    this.pos = pos;
-    Terms terms = MultiFields.getTerms(reader, tindex.field);
-    deletedDocs = MultiFields.getDeletedDocs(reader);
-    if (terms != null) {
-      tenum = terms.iterator();
-      tenum.seek(termValue);
-      setTerm();
-    }
-  }
-
-  @Override
-  public Comparator<BytesRef> getComparator() throws IOException {
-    return tenum.getComparator();
-  }
-
-  public DocsEnum getDocsEnum() throws IOException {
-    docsEnum = tenum.docs(deletedDocs, docsEnum);
-    return docsEnum;
-  }
-
-  protected BytesRef setTerm() throws IOException {
-    termText = tenum.term();
-    if (tindex.prefix != null && !termText.startsWith(tindex.prefix)) {
-      termText = null;
-    }
-    return termText;
-  }
-
-  @Override
-  public BytesRef next() throws IOException {
-    pos++;
-    if (tenum.next() == null) {
-      termText = null;
-      return null;
-    }
-    return setTerm();  // this is extra work if we know we are in bounds...
-  }
-
-  @Override
-  public BytesRef term() {
-    return termText;
-  }
-
-  @Override
-  public int docFreq() throws IOException {
-    return tenum.docFreq();
-  }
-
-  @Override
-  public long totalTermFreq() throws IOException {
-    return tenum.totalTermFreq();
-  }
-
-  public BytesRef skipTo(BytesRef target) throws IOException {
-
-    // already here
-    if (termText != null && termText.equals(target)) return termText;
-
-    if (tenum == null) {
-      return null;
-    }
-
-    int startIdx = Arrays.binarySearch(tindex.index,target);
-
-    if (startIdx >= 0) {
-      // we hit the term exactly... lucky us!
-      TermsEnum.SeekStatus seekStatus = tenum.seek(target);
-      assert seekStatus == TermsEnum.SeekStatus.FOUND;
-      pos = startIdx << tindex.intervalBits;
-      return setTerm();
-    }
-
-    // we didn't hit the term exactly
-    startIdx=-startIdx-1;
-    
-    if (startIdx == 0) {
-      // our target occurs *before* the first term
-      TermsEnum.SeekStatus seekStatus = tenum.seek(target);
-      assert seekStatus == TermsEnum.SeekStatus.NOT_FOUND;
-      pos = 0;
-      return setTerm();
-    }
-
-    // back up to the start of the block
-    startIdx--;
-
-    if ((pos >> tindex.intervalBits) == startIdx && termText != null && termText.compareTo(target)<=0) {
-      // we are already in the right block and the current term is before the term we want,
-      // so we don't need to seek.
-    } else {
-      // seek to the right block
-      TermsEnum.SeekStatus seekStatus = tenum.seek(tindex.index[startIdx]);
-      assert seekStatus == TermsEnum.SeekStatus.FOUND;
-      pos = startIdx << tindex.intervalBits;
-      setTerm();  // should be non-null since it's in the index
-    }
-
-    while (termText != null && termText.compareTo(target) < 0) {
-      next();
-    }
-
-    return termText;
-  }
-
-  public BytesRef skipTo(int termNumber) throws IOException {
-    int delta = termNumber - pos;
-    if (delta < 0 || delta > tindex.interval || tenum==null) {
-      int idx = termNumber >>> tindex.intervalBits;
-      BytesRef base = tindex.index[idx];
-      pos = idx << tindex.intervalBits;
-      delta = termNumber - pos;
-      TermsEnum.SeekStatus seekStatus = tenum.seek(base);
-      assert seekStatus == TermsEnum.SeekStatus.FOUND;
-    }
-    while (--delta >= 0) {
-      BytesRef br = tenum.next();
-      if (br == null) {
-        termText = null;
-        return null;
-      }
-      ++pos;
-    }
-    return setTerm();
-  }
-
-  protected void close() throws IOException {
-    // no-op, needed so the anon subclass that does indexing
-    // can build its index
-  }
-
-  /** The current term number, starting at 0.
-   * Only valid if the previous call to next() or skipTo() returned true.
-   */
-  public int getTermNumber() {
-    return pos;
-  }
-
-  @Override
-  public long ord() {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public SeekStatus seek(long ord) {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public DocsEnum docs(Bits skipDocs, DocsEnum reuse) {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public DocsAndPositionsEnum docsAndPositions(Bits skipDocs, DocsAndPositionsEnum reuse) {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public SeekStatus seek(BytesRef target, boolean useCache) {
-    throw new UnsupportedOperationException();
-  }
-}
-
-
-/**
- * Class to save memory by only storing every nth term (for random access), while
- * numbering the terms, allowing them to be retrieved later by number.
- * This is only valid when used with the IndexReader it was created with.
- * The IndexReader is not actually stored to facilitate caching by using it as a key in
- * a weak hash map.
- */
-class TermIndex {
-  final static int intervalBits = 7;  // decrease to a low number like 2 for testing
-  final static int intervalMask = 0xffffffff >>> (32-intervalBits);
-  final static int interval = 1 << intervalBits;
-
-  final String field;
-  final BytesRef prefix;
-  BytesRef[] index;
-  int nTerms;
-  long sizeOfStrings;
-
-  TermIndex(String field) {
-    this(field, null);
-  }
-
-  TermIndex(String field, String prefix) {
-    this.field = field;
-    this.prefix = prefix == null ? null : new BytesRef(prefix);
-  }
-
-  NumberedTermsEnum getEnumerator(IndexReader reader, int termNumber) throws IOException {
-    NumberedTermsEnum te = new NumberedTermsEnum(reader, this);
-    te.skipTo(termNumber);
-    return te;
-  }
-
-  /* The first time an enumerator is requested, it should be used
-     with next() to fully traverse all of the terms so the index
-     will be built.
-   */
-  NumberedTermsEnum getEnumerator(IndexReader reader) throws IOException {
-    if (index==null) return new NumberedTermsEnum(reader,this, prefix==null?new BytesRef():prefix, 0) {
-      ArrayList<BytesRef> lst;
-      PagedBytes bytes;
-
-      @Override
-      protected BytesRef setTerm() throws IOException {
-        BytesRef br = super.setTerm();
-        if (br != null && (pos & intervalMask)==0) {
-          sizeOfStrings += br.length;
-          if (lst==null) {
-            lst = new ArrayList<BytesRef>();
-            bytes = new PagedBytes(15);
-          }
-          BytesRef out = new BytesRef();
-          bytes.copy(br, out);
-          lst.add(out);
-        }
-        return br;
-      }
-
-      @Override
-      public BytesRef skipTo(int termNumber) throws IOException {
-        throw new UnsupportedOperationException();
-      }
-
-      @Override
-      public void close() throws IOException {
-        nTerms=pos;
-        super.close();
-        index = lst!=null ? lst.toArray(new BytesRef[lst.size()]) : new BytesRef[0];
-      }
-    };
-    else return new NumberedTermsEnum(reader,this,new BytesRef(),0);
-  }
-
-
-  /**
-   * Returns the approximate amount of memory taken by this TermIndex.
-   * This is only an approximation and doesn't take into account java object overhead.
-   *
-   * @return
-   * the approximate memory consumption in bytes
-   */
-  public long memSize() {
-    // assume 8 byte references?
-    return 8+8+8+8+(index.length<<3)+sizeOfStrings;
-  }
-}
-

Modified: lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/response/JSONResponseWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/response/JSONResponseWriter.java?rev=1098566&r1=1098565&r2=1098566&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/response/JSONResponseWriter.java (original)
+++ lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/response/JSONResponseWriter.java Mon May  2 13:50:57 2011
@@ -316,7 +316,8 @@ class JSONWriter extends TextResponseWri
     if( idx > 0 ) {
       writeArraySeparator();
     }
-    
+
+    indent();
     writeMapOpener(doc.size()); 
     incLevel();
 

Modified: lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/response/PageTool.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/response/PageTool.java?rev=1098566&r1=1098565&r2=1098566&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/response/PageTool.java (original)
+++ lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/response/PageTool.java Mon May  2 13:50:57 2011
@@ -19,8 +19,10 @@ package org.apache.solr.response;
 
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.response.SolrQueryResponse;
+import org.apache.solr.search.DocList;
 import org.apache.solr.search.DocSlice;
 import org.apache.solr.common.SolrDocumentList;
+import org.apache.solr.common.SolrException;
 
 public class PageTool {
   private long start;
@@ -42,10 +44,16 @@ public class PageTool {
         DocSlice doc_slice = (DocSlice) docs;
         results_found = doc_slice.matches();
         start = doc_slice.offset();
-      } else {
+      } else if(docs instanceof ResultContext) {
+        DocList dl = ((ResultContext) docs).docs;
+        results_found = dl.matches();
+        start = dl.offset();
+      } else if(docs instanceof SolrDocumentList) {
         SolrDocumentList doc_list = (SolrDocumentList) docs;
         results_found = doc_list.getNumFound();
         start = doc_list.getStart();
+      } else {
+	  throw new SolrException(SolrException.ErrorCode.UNKNOWN, "Unknown response type "+docs+". Expected one of DocSlice, ResultContext or SolrDocumentList");
       }
     }
 

Modified: lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/response/transform/DocIdAugmenterFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/response/transform/DocIdAugmenterFactory.java?rev=1098566&r1=1098565&r2=1098566&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/response/transform/DocIdAugmenterFactory.java (original)
+++ lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/response/transform/DocIdAugmenterFactory.java Mon May  2 13:50:57 2011
@@ -19,6 +19,7 @@ package org.apache.solr.response.transfo
 import org.apache.solr.common.SolrDocument;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrException.ErrorCode;
+import org.apache.solr.request.SolrQueryRequest;
 
 /**
  * @version $Id$
@@ -27,7 +28,7 @@ import org.apache.solr.common.SolrExcept
 public class DocIdAugmenterFactory extends TransformerFactory
 {
   @Override
-  public DocTransformer create(String field, String arg) {
+  public DocTransformer create(String field, String arg, SolrQueryRequest req) {
     if( arg != null ) {
       throw new SolrException( ErrorCode.BAD_REQUEST,
           "DocIdAugmenter does not take any arguments" );

Modified: lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/response/transform/ExplainAugmenterFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/response/transform/ExplainAugmenterFactory.java?rev=1098566&r1=1098565&r2=1098566&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/response/transform/ExplainAugmenterFactory.java (original)
+++ lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/response/transform/ExplainAugmenterFactory.java Mon May  2 13:50:57 2011
@@ -23,6 +23,7 @@ import org.apache.solr.common.SolrDocume
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrException.ErrorCode;
 import org.apache.solr.common.util.NamedList;
+import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.util.SolrPluginUtils;
 
 /**
@@ -62,7 +63,7 @@ public class ExplainAugmenterFactory ext
   }
 
   @Override
-  public DocTransformer create(String field, String arg) {
+  public DocTransformer create(String field, String arg, SolrQueryRequest req) {
     Style style = (arg==null)?defaultStyle:getStyle(arg);
     return new ExplainAugmenter( field, style );
   }

Modified: lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/response/transform/ShardAugmenterFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/response/transform/ShardAugmenterFactory.java?rev=1098566&r1=1098565&r2=1098566&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/response/transform/ShardAugmenterFactory.java (original)
+++ lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/response/transform/ShardAugmenterFactory.java Mon May  2 13:50:57 2011
@@ -16,6 +16,8 @@
  */
 package org.apache.solr.response.transform;
 
+import org.apache.solr.request.SolrQueryRequest;
+
 
 /**
  * @version $Id$
@@ -24,7 +26,7 @@ package org.apache.solr.response.transfo
 public class ShardAugmenterFactory extends TransformerFactory
 {
   @Override
-  public DocTransformer create(String field, String arg) {
+  public DocTransformer create(String field, String arg, SolrQueryRequest req) {
     String id = "TODO... find ID";
     // Maybe it is stored in the context?
     // is it a request variable?

Modified: lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/response/transform/TransformerFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/response/transform/TransformerFactory.java?rev=1098566&r1=1098565&r2=1098566&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/response/transform/TransformerFactory.java (original)
+++ lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/response/transform/TransformerFactory.java Mon May  2 13:50:57 2011
@@ -20,7 +20,9 @@ package org.apache.solr.response.transfo
 import java.util.HashMap;
 import java.util.Map;
 
+import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.NamedList;
+import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.util.plugin.NamedListInitializedPlugin;
 
 /**
@@ -36,7 +38,7 @@ public abstract class TransformerFactory
     defaultUserArgs = (String)args.get( "args" );
   }
 
-  public abstract DocTransformer create(String field, String args);
+  public abstract DocTransformer create(String field, String args, SolrQueryRequest req);
 
   public static final Map<String,TransformerFactory> defaultFactories = new HashMap<String,TransformerFactory>();
   static {

Modified: lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/response/transform/ValueAugmenterFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/response/transform/ValueAugmenterFactory.java?rev=1098566&r1=1098565&r2=1098566&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/response/transform/ValueAugmenterFactory.java (original)
+++ lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/response/transform/ValueAugmenterFactory.java Mon May  2 13:50:57 2011
@@ -21,6 +21,7 @@ import org.apache.solr.common.SolrExcept
 import org.apache.solr.common.SolrException.ErrorCode;
 import org.apache.solr.common.util.DateUtil;
 import org.apache.solr.common.util.NamedList;
+import org.apache.solr.request.SolrQueryRequest;
 
 /**
  * @version $Id$
@@ -60,7 +61,7 @@ public class ValueAugmenterFactory exten
   }
 
   @Override
-  public DocTransformer create(String field, String arg) {
+  public DocTransformer create(String field, String arg, SolrQueryRequest req) {
     Object val = value;
     if( val == null ) {
       val = (arg==null)?defaultValue:getObjectFrom(arg);

Modified: lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/response/transform/ValueSourceAugmenter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/response/transform/ValueSourceAugmenter.java?rev=1098566&r1=1098565&r2=1098566&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/response/transform/ValueSourceAugmenter.java (original)
+++ lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/response/transform/ValueSourceAugmenter.java Mon May  2 13:50:57 2011
@@ -16,10 +16,19 @@
  */
 package org.apache.solr.response.transform;
 
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.util.ReaderUtil;
 import org.apache.solr.common.SolrDocument;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.core.SolrCore;
 import org.apache.solr.search.QParser;
+import org.apache.solr.search.SolrIndexSearcher;
+import org.apache.solr.search.function.DocValues;
 import org.apache.solr.search.function.ValueSource;
 
+import java.io.IOException;
+import java.util.Map;
+
 /**
  * Add values from a ValueSource (function query etc)
  *
@@ -32,13 +41,15 @@ public class ValueSourceAugmenter extend
 {
   public final String name;
   public final QParser qparser;
-  public final ValueSource values;
+  public final ValueSource valueSource;
+
+
 
-  public ValueSourceAugmenter( String name, QParser qparser, ValueSource values )
+  public ValueSourceAugmenter( String name, QParser qparser, ValueSource valueSource )
   {
     this.name = name;
     this.qparser = qparser;
-    this.values = values;
+    this.valueSource = valueSource;
   }
 
   @Override
@@ -49,14 +60,42 @@ public class ValueSourceAugmenter extend
 
   @Override
   public void setContext( TransformContext context ) {
-    // maybe we do something here?
+    IndexReader reader = qparser.getReq().getSearcher().getIndexReader();
+    readerContexts = reader.getTopReaderContext().leaves();
+    docValuesArr = new DocValues[readerContexts.length];
+
+    searcher = qparser.getReq().getSearcher();
+    this.fcontext = valueSource.newContext(searcher);
   }
 
+
+  Map fcontext;
+  SolrIndexSearcher searcher;
+  IndexReader.AtomicReaderContext[] readerContexts;
+  DocValues docValuesArr[];
+
+
   @Override
   public void transform(SolrDocument doc, int docid) {
-    // TODO, should know what the real type is -- not always string
-    // how do we get to docvalues?
-    Object v = "now what..."; //values.g.strVal( docid );
-    doc.setField( name, v );
+    // This is only good for random-access functions
+
+    try {
+
+      // TODO: calculate this stuff just once across diff functions
+      int idx = ReaderUtil.subIndex(docid, readerContexts);
+      IndexReader.AtomicReaderContext rcontext = readerContexts[idx];
+      DocValues values = docValuesArr[idx];
+      if (values == null) {
+        docValuesArr[idx] = values = valueSource.getValues(fcontext, rcontext);
+      }
+
+      int localId = docid - rcontext.docBase;
+      Object val = values.objectVal(localId);
+      if (val != null) {
+        doc.setField( name, val );
+      }
+    } catch (IOException e) {
+      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "exception at docid " + docid + " for valuesource " + valueSource, e, false);
+    }
   }
 }

Modified: lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/schema/DateField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/schema/DateField.java?rev=1098566&r1=1098565&r2=1098566&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/schema/DateField.java (original)
+++ lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/schema/DateField.java Mon May  2 13:50:57 2011
@@ -180,6 +180,14 @@ public class DateField extends FieldType
                                "Invalid Date Math String:'" +val+'\'',e);
     }
   }
+
+  public Fieldable createField(SchemaField field, Object value, float boost) {
+    // Convert to a string before indexing
+    if(value instanceof Date) {
+      value = toInternal( (Date)value ) + 'Z';
+    }
+    return super.createField(field, value, boost);
+  }
   
   public String toInternal(Date val) {
     return formatDate(val);
@@ -479,6 +487,17 @@ class DateFieldSource extends FieldCache
       }
 
       @Override
+      public Object objectVal(int doc) {
+        int ord=termsIndex.getOrd(doc);
+        if (ord == 0) {
+          return null;
+        } else {
+          BytesRef br = termsIndex.lookup(ord, new BytesRef());
+          return ft.toObject(null, br);
+        }
+      }
+
+      @Override
       public String toString(int doc) {
         return description() + '=' + intVal(doc);
       }

Modified: lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/schema/FieldProperties.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/schema/FieldProperties.java?rev=1098566&r1=1098565&r2=1098566&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/schema/FieldProperties.java (original)
+++ lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/schema/FieldProperties.java Mon May  2 13:50:57 2011
@@ -22,29 +22,31 @@ import java.util.HashMap;
 
 /**
  * @version $Id$
+ * 
+ * @lucene.internal
  */
-abstract class FieldProperties {
+public abstract class FieldProperties {
 
   // use a bitfield instead of many different boolean variables since
   // many of the variables are independent or semi-independent.
 
   // bit values for boolean field properties.
-  final static int INDEXED             = 0x00000001;
-  final static int TOKENIZED           = 0x00000002;
-  final static int STORED              = 0x00000004;
-  final static int BINARY              = 0x00000008;
-  final static int OMIT_NORMS          = 0x00000010;
-  final static int OMIT_TF_POSITIONS   = 0x00000020;
-  final static int STORE_TERMVECTORS   = 0x00000040;
-  final static int STORE_TERMPOSITIONS = 0x00000080;
-  final static int STORE_TERMOFFSETS   = 0x00000100;
+  protected final static int INDEXED             = 0x00000001;
+  protected final static int TOKENIZED           = 0x00000002;
+  protected final static int STORED              = 0x00000004;
+  protected final static int BINARY              = 0x00000008;
+  protected final static int OMIT_NORMS          = 0x00000010;
+  protected final static int OMIT_TF_POSITIONS   = 0x00000020;
+  protected final static int STORE_TERMVECTORS   = 0x00000040;
+  protected final static int STORE_TERMPOSITIONS = 0x00000080;
+  protected final static int STORE_TERMOFFSETS   = 0x00000100;
 
 
-  final static int MULTIVALUED         = 0x00000200;
-  final static int SORT_MISSING_FIRST  = 0x00000400;
-  final static int SORT_MISSING_LAST   = 0x00000800;
+  protected final static int MULTIVALUED         = 0x00000200;
+  protected final static int SORT_MISSING_FIRST  = 0x00000400;
+  protected final static int SORT_MISSING_LAST   = 0x00000800;
   
-  final static int REQUIRED            = 0x00001000;
+  protected final static int REQUIRED            = 0x00001000;
   
   static final String[] propertyNames = {
           "indexed", "tokenized", "stored",

Modified: lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/schema/FieldType.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/schema/FieldType.java?rev=1098566&r1=1098565&r2=1098566&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/schema/FieldType.java (original)
+++ lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/schema/FieldType.java Mon May  2 13:50:57 2011
@@ -25,6 +25,7 @@ import org.apache.lucene.document.Field;
 import org.apache.lucene.document.Fieldable;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.search.Query;
+import org.apache.lucene.search.Similarity;
 import org.apache.lucene.search.SortField;
 import org.apache.lucene.search.TermQuery;
 import org.apache.lucene.search.TermRangeQuery;
@@ -83,6 +84,11 @@ public abstract class FieldType extends 
   public boolean isMultiValued() {
     return (properties & MULTIVALUED) != 0;
   }
+  
+  /** Check if a property is set */
+  protected boolean hasProperty( int p ) {
+    return (properties & p) != 0;
+  }
 
   /**
    * A "polyField" is a FieldType that can produce more than one Fieldable instance for a single value, via the {@link #createFields(org.apache.solr.schema.SchemaField, Object, float)} method.  This is useful
@@ -503,6 +509,34 @@ public abstract class FieldType extends 
     throw e;
   }
 
+  /** @lucene.internal */
+  protected Similarity similarity;
+  
+  /**
+   * Gets the Similarity used when scoring fields of this type
+   * 
+   * <p>
+   * The default implementation returns null, which means this type
+   * has no custom similarity associated with it.
+   * </p>
+   * 
+   * This method exists to internally support SolrSimilarityProvider. 
+   * Custom application code interested in a field's Similarity should
+   * instead query via the searcher's SimilarityProvider.
+   * @lucene.internal
+   */
+  public Similarity getSimilarity() {
+    return similarity;
+  }
+  
+  /**
+   * Sets the Similarity used when scoring fields of this type
+   * @lucene.internal
+   */
+  public void setSimilarity(Similarity similarity) {
+    this.similarity = similarity;
+  }
+  
   /**
    * calls back to TextResponseWriter to write the field value
    */