You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ho...@apache.org on 2014/01/06 22:15:46 UTC

svn commit: r1556036 [1/2] - in /lucene/dev/trunk/solr: ./ core/src/java/org/apache/solr/handler/component/ core/src/java/org/apache/solr/search/ core/src/test-files/solr/collection1/conf/ core/src/test/org/apache/solr/ core/src/test/org/apache/solr/cl...

Author: hossman
Date: Mon Jan  6 21:15:45 2014
New Revision: 1556036

URL: http://svn.apache.org/r1556036
Log:
SOLR-5463: new 'cursorMark' request param for deep paging of sorted result sets

Added:
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/CursorMark.java   (with props)
    lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/schema-sorts.xml   (with props)
    lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/solrconfig-deeppaging.xml   (with props)
    lucene/dev/trunk/solr/core/src/test/org/apache/solr/CursorPagingTest.java   (with props)
    lucene/dev/trunk/solr/core/src/test/org/apache/solr/TestCursorMarkWithoutUniqueKey.java   (with props)
    lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/DistribCursorPagingTest.java   (with props)
    lucene/dev/trunk/solr/core/src/test/org/apache/solr/search/CursorMarkTest.java   (with props)
    lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/common/params/CursorMarkParams.java   (with props)
Modified:
    lucene/dev/trunk/solr/CHANGES.txt
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/QueryComponent.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/ResponseBuilder.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/QParser.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
    lucene/dev/trunk/solr/core/src/test/org/apache/solr/BasicFunctionalityTest.java
    lucene/dev/trunk/solr/core/src/test/org/apache/solr/schema/SortableBinaryField.java
    lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/client/solrj/response/QueryResponse.java
    lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/common/params/CommonParams.java
    lucene/dev/trunk/solr/test-framework/src/java/org/apache/solr/BaseDistributedSearchTestCase.java
    lucene/dev/trunk/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java

Modified: lucene/dev/trunk/solr/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/CHANGES.txt?rev=1556036&r1=1556035&r2=1556036&view=diff
==============================================================================
--- lucene/dev/trunk/solr/CHANGES.txt (original)
+++ lucene/dev/trunk/solr/CHANGES.txt Mon Jan  6 21:15:45 2014
@@ -53,6 +53,9 @@ New Features
   Andrzej Bialecki, Patrick Hunt, Wolfgang Hoschek, Roman Shaposhnik, 
   Eric Wong)
 
+* SOLR-5463: new 'cursorMark' request param for deep paging of sorted result sets
+  (sarowe, hossman)
+
 Other Changes
 ----------------------
 

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/QueryComponent.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/QueryComponent.java?rev=1556036&r1=1556035&r2=1556036&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/QueryComponent.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/QueryComponent.java Mon Jan  6 21:15:45 2014
@@ -17,8 +17,6 @@
 
 package org.apache.solr.handler.component;
 
-import org.apache.lucene.document.Field;
-import org.apache.lucene.document.StringField;
 import org.apache.lucene.index.AtomicReaderContext;
 import org.apache.lucene.index.IndexReaderContext;
 import org.apache.lucene.index.ReaderUtil;
@@ -33,12 +31,12 @@ import org.apache.lucene.search.grouping
 import org.apache.lucene.search.grouping.SearchGroup;
 import org.apache.lucene.search.grouping.TopGroups;
 import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.CharsRef;
 import org.apache.solr.client.solrj.SolrServerException;
 import org.apache.solr.common.SolrDocument;
 import org.apache.solr.common.SolrDocumentList;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.params.*;
+import org.apache.solr.common.params.CursorMarkParams;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.SimpleOrderedMap;
 import org.apache.solr.common.util.StrUtils;
@@ -48,6 +46,7 @@ import org.apache.solr.response.SolrQuer
 import org.apache.solr.schema.FieldType;
 import org.apache.solr.schema.IndexSchema;
 import org.apache.solr.schema.SchemaField;
+import org.apache.solr.search.CursorMark;
 import org.apache.solr.search.DocIterator;
 import org.apache.solr.search.DocList;
 import org.apache.solr.search.DocListAndSet;
@@ -82,6 +81,8 @@ import org.apache.solr.search.grouping.e
 import org.apache.solr.search.grouping.endresulttransformer.SimpleEndResultTransformer;
 import org.apache.solr.util.SolrPluginUtils;
 
+import org.apache.commons.lang.StringUtils;
+
 import java.io.IOException;
 import java.io.PrintWriter;
 import java.io.StringWriter;
@@ -147,8 +148,15 @@ public class QueryComponent extends Sear
       rb.setQuery( q );
       rb.setSortSpec( parser.getSort(true) );
       rb.setQparser(parser);
-      rb.setScoreDoc(parser.getPaging());
       
+      final String cursorStr = rb.req.getParams().get(CursorMarkParams.CURSOR_MARK_PARAM);
+      if (null != cursorStr) {
+        final CursorMark cursorMark = new CursorMark(rb.req.getSchema(),
+                                                     rb.getSortSpec());
+        cursorMark.parseSerializedTotem(cursorStr);
+        rb.setCursorMark(cursorMark);
+      }
+
       String[] fqs = req.getParams().getParams(CommonParams.FQ);
       if (fqs!=null && fqs.length!=0) {
         List<Query> filters = rb.getFilters();
@@ -171,11 +179,23 @@ public class QueryComponent extends Sear
       throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, e);
     }
 
-    boolean grouping = params.getBool(GroupParams.GROUP, false);
-    if (!grouping) {
-      return;
+    if (params.getBool(GroupParams.GROUP, false)) {
+      prepareGrouping(rb);
     }
+  }
+
+  private void prepareGrouping(ResponseBuilder rb) throws IOException {
+
+    SolrQueryRequest req = rb.req;
+    SolrParams params = req.getParams();
 
+    if (null != rb.getCursorMark()) {
+      // It's hard to imagine, conceptually, what it would mean to combine
+      // grouping with a cursor - so for now we just don't allow the combination at all
+      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Can not use Grouping with " + 
+                              CursorMarkParams.CURSOR_MARK_PARAM);
+    }
+ 
     SolrIndexSearcher.QueryCommand cmd = rb.getQueryCommand();
     SolrIndexSearcher searcher = rb.req.getSearcher();
     GroupingSpecification groupingSpec = new GroupingSpecification();
@@ -242,6 +262,11 @@ public class QueryComponent extends Sear
 
     // -1 as flag if not set.
     long timeAllowed = (long)params.getInt( CommonParams.TIME_ALLOWED, -1 );
+    if (null != rb.getCursorMark() && 0 < timeAllowed) {
+      // fundementally incompatible
+      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Can not search using both " + 
+                              CursorMarkParams.CURSOR_MARK_PARAM + " and " + CommonParams.TIME_ALLOWED);
+    }
 
     // Optional: This could also be implemented by the top-level searcher sending
     // a filter that lists the ids... that would be transparent to
@@ -434,13 +459,18 @@ public class QueryComponent extends Sear
     searcher.search(result,cmd);
     rb.setResult( result );
 
-
     ResultContext ctx = new ResultContext();
     ctx.docs = rb.getResults().docList;
     ctx.query = rb.getQuery();
     rsp.add("response", ctx);
     rsp.getToLog().add("hits", rb.getResults().docList.matches());
 
+    if ( ! rb.req.getParams().getBool(ShardParams.IS_SHARD,false) ) {
+      if (null != rb.getNextCursorMark()) {
+        rb.rsp.add(CursorMarkParams.CURSOR_MARK_NEXT, 
+                   rb.getNextCursorMark().getSerializedTotem());
+      }
+    }
     doFieldSortValues(rb, searcher);
     doPrefetch(rb);
   }
@@ -452,6 +482,8 @@ public class QueryComponent extends Sear
     // The query cache doesn't currently store sort field values, and SolrIndexSearcher doesn't
     // currently have an option to return sort field values.  Because of this, we
     // take the documents given and re-derive the sort values.
+    //
+    // TODO: See SOLR-5595
     boolean fsv = req.getParams().getBool(ResponseBuilder.FIELD_SORT_VALUES,false);
     if(fsv){
       NamedList<Object[]> sortVals = new NamedList<Object[]>(); // order is important for the sort fields
@@ -696,6 +728,10 @@ public class QueryComponent extends Sear
     }
 
     rb.rsp.add("response", rb._responseDocs);
+    if (null != rb.getNextCursorMark()) {
+      rb.rsp.add(CursorMarkParams.CURSOR_MARK_NEXT, 
+                 rb.getNextCursorMark().getSerializedTotem());
+    }
   }
 
   private void createDistributedIdf(ResponseBuilder rb) {
@@ -904,11 +940,66 @@ public class QueryComponent extends Sear
       // TODO: use ResponseBuilder (w/ comments) or the request context?
       rb.resultIds = resultIds;
       rb._responseDocs = responseDocs;
+
+      populateNextCursorMarkFromMergedShards(rb);
+
       if (partialResults) {
         rb.rsp.getResponseHeader().add( "partialResults", Boolean.TRUE );
       }
   }
 
+  /**
+   * Inspects the state of the {@link ResponseBuilder} and populates the next 
+   * {@link ResponseBuilder#setNextCursorMark} as appropriate based on the merged 
+   * sort values from individual shards
+   *
+   * @param rb A <code>ResponseBuilder</code> that already contains merged 
+   *           <code>ShardDocs</code> in <code>resultIds</code>, may or may not be 
+   *           part of a Cursor based request (method will NOOP if not needed)
+   */
+  private void populateNextCursorMarkFromMergedShards(ResponseBuilder rb) {
+
+    final CursorMark lastCursorMark = rb.getCursorMark();
+    if (null == lastCursorMark) {
+      // Not a cursor based request
+      return; // NOOP
+    }
+
+    assert null != rb.resultIds : "resultIds was not set in ResponseBuilder";
+
+    Collection<ShardDoc> docsOnThisPage = rb.resultIds.values();
+
+    if (0 == docsOnThisPage.size()) {
+      // nothing more matching query, re-use existing totem so user can "resume" 
+      // search later if it makes sense for this sort.
+      rb.setNextCursorMark(lastCursorMark);
+      return;
+    }
+
+    ShardDoc lastDoc = null;
+    // ShardDoc and rb.resultIds are weird structures to work with...
+    for (ShardDoc eachDoc : docsOnThisPage) {
+      if (null == lastDoc || lastDoc.positionInResponse  < eachDoc.positionInResponse) {
+        lastDoc = eachDoc;
+      }
+    }
+    SortField[] sortFields = lastCursorMark.getSortSpec().getSort().getSort();
+    List<Object> nextCursorMarkValues = new ArrayList<Object>(sortFields.length);
+    for (SortField sf : sortFields) {
+      if (sf.getType().equals(SortField.Type.SCORE)) {
+        assert null != lastDoc.score : "lastDoc has null score";
+        nextCursorMarkValues.add(lastDoc.score);
+      } else {
+        assert null != sf.getField() : "SortField has null field";
+        List<Object> fieldVals = (List<Object>) lastDoc.sortFieldValues.get(sf.getField());
+        nextCursorMarkValues.add(fieldVals.get(lastDoc.orderInShard));
+      }
+    }
+    CursorMark nextCursorMark = lastCursorMark.createNext(nextCursorMarkValues);
+    assert null != nextCursorMark : "null nextCursorMark";
+    rb.setNextCursorMark(nextCursorMark);
+  }
+
   private NamedList unmarshalSortValues(SortSpec sortSpec, 
                                         NamedList sortFieldValues, 
                                         IndexSchema schema) {
@@ -982,6 +1073,7 @@ public class QueryComponent extends Sear
 
       // no need for a sort, we already have order
       sreq.params.remove(CommonParams.SORT);
+      sreq.params.remove(CursorMarkParams.CURSOR_MARK_PARAM);
 
       // we already have the field sort values
       sreq.params.remove(ResponseBuilder.FIELD_SORT_VALUES);

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/ResponseBuilder.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/ResponseBuilder.java?rev=1556036&r1=1556035&r2=1556036&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/ResponseBuilder.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/ResponseBuilder.java Mon Jan  6 21:15:45 2014
@@ -18,7 +18,6 @@
 package org.apache.solr.handler.component;
 
 import org.apache.lucene.search.Query;
-import org.apache.lucene.search.ScoreDoc;
 import org.apache.lucene.search.grouping.SearchGroup;
 import org.apache.lucene.search.grouping.TopGroups;
 import org.apache.lucene.util.BytesRef;
@@ -30,6 +29,7 @@ import org.apache.solr.common.util.Simpl
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.request.SolrRequestInfo;
 import org.apache.solr.response.SolrQueryResponse;
+import org.apache.solr.search.CursorMark;
 import org.apache.solr.search.DocListAndSet;
 import org.apache.solr.search.QParser;
 import org.apache.solr.search.SolrIndexSearcher;
@@ -70,9 +70,8 @@ public class ResponseBuilder
   private List<Query> filters = null;
   private SortSpec sortSpec = null;
   private GroupingSpecification groupingSpec;
-  //used for handling deep paging
-  private ScoreDoc scoreDoc;
-
+  private CursorMark cursorMark;
+  private CursorMark nextCursorMark;
 
   private DocListAndSet results = null;
   private NamedList<Object> debugInfo = null;
@@ -395,7 +394,7 @@ public class ResponseBuilder
             .setLen(getSortSpec().getCount())
             .setFlags(getFieldFlags())
             .setNeedDocSet(isNeedDocSet())
-            .setScoreDoc(getScoreDoc()); //Issue 1726
+            .setCursorMark(getCursorMark());
     return cmd;
   }
 
@@ -407,6 +406,10 @@ public class ResponseBuilder
     if (result.isPartialResults()) {
       rsp.getResponseHeader().add("partialResults", Boolean.TRUE);
     }
+    if (null != cursorMark) {
+      assert null != result.getNextCursorMark() : "using cursor but no next cursor set";
+      this.setNextCursorMark(result.getNextCursorMark());
+    }
   }
   
   public long getNumberDocumentsFound() {
@@ -416,13 +419,17 @@ public class ResponseBuilder
     return _responseDocs.getNumFound();
   }
 
-  public ScoreDoc getScoreDoc()
-  {
-    return scoreDoc;
+  public CursorMark getCursorMark() {
+    return cursorMark;
   }
-  
-  public void setScoreDoc(ScoreDoc scoreDoc)
-  {
-    this.scoreDoc = scoreDoc;
+  public void setCursorMark(CursorMark cursorMark) {
+    this.cursorMark = cursorMark;
+  }
+
+  public CursorMark getNextCursorMark() {
+    return nextCursorMark;
+  }
+  public void setNextCursorMark(CursorMark nextCursorMark) {
+    this.nextCursorMark = nextCursorMark;
   }
 }

Added: lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/CursorMark.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/CursorMark.java?rev=1556036&view=auto
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/CursorMark.java (added)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/CursorMark.java Mon Jan  6 21:15:45 2014
@@ -0,0 +1,309 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.search;
+
+import org.apache.lucene.index.AtomicReaderContext;
+import org.apache.lucene.search.FieldComparator;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.FieldDoc;
+import org.apache.lucene.search.Scorer;
+import org.apache.lucene.search.Sort;
+import org.apache.lucene.search.SortField;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.util.BytesRef;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.SolrException.ErrorCode;
+import static org.apache.solr.common.params.CursorMarkParams.*;
+import org.apache.solr.common.util.Base64;
+import org.apache.solr.common.util.JavaBinCodec;
+import org.apache.solr.schema.IndexSchema;
+import org.apache.solr.schema.FieldType;
+import org.apache.solr.schema.SchemaField;
+import org.apache.solr.search.PostFilter;
+import org.apache.solr.search.ExtendedQueryBase;
+import org.apache.solr.search.DelegatingCollector;
+import org.apache.commons.lang.StringUtils;
+
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.io.ByteArrayOutputStream;
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+
+/**
+ * An object that encapsulates the basic information about the current Mark Point of a 
+ * "Cursor" based request.  <code>CursorMark</code> objects track the sort values of 
+ * the last document returned to a user, so that {@link SolrIndexSearcher} can then 
+ * be asked to find all documents "after" the values represented by this 
+ * <code>CursorMark</code>.
+ *
+ */
+public final class CursorMark {
+
+  /**
+   * Used for validation and (un)marshalling of sort values
+   */
+  private final SortSpec sortSpec;
+
+  /**
+   * The raw, unmarshalled, sort values (that corrispond with the SortField's in the 
+   * SortSpec) for knowing which docs this cursor should "search after".  If this 
+   * list is null, then we have no specific values to "search after" and we 
+   * should start from the very begining of the sorted list of documents matching 
+   * the query.
+   */
+  private List<Object> values = null;
+
+  /**
+   * for serializing this CursorMark as a String
+   */
+  private final JavaBinCodec codec = new JavaBinCodec();
+
+  /**
+   * Generates an empty CursorMark bound for use with the 
+   * specified schema and {@link SortSpec}.
+   *
+   * @param schema used for basic validation
+   * @param sortSpec bound to this totem (un)marshalling serialized values
+   */
+  public CursorMark(IndexSchema schema, SortSpec sortSpec) {
+
+    final SchemaField uniqueKey = schema.getUniqueKeyField();
+    if (null == uniqueKey) {
+      throw new SolrException(ErrorCode.BAD_REQUEST,
+                              "Cursor functionality is not available unless the IndexSchema defines a uniqueKey field");
+    }
+
+    final Sort sort = sortSpec.getSort();
+    if (null == sort) {
+      // pure score, by definition we don't include the mandatyr uniqueKey tie breaker
+      throw new SolrException(ErrorCode.BAD_REQUEST,
+                              "Cursor functionality requires a sort containing a uniqueKey field tie breaker");
+    }
+    
+    if (!sortSpec.getSchemaFields().contains(uniqueKey)) {
+      throw new SolrException(ErrorCode.BAD_REQUEST,
+                              "Cursor functionality requires a sort containing a uniqueKey field tie breaker");
+    }
+
+    if (0 != sortSpec.getOffset()) {
+      throw new SolrException(ErrorCode.BAD_REQUEST,
+                              "Cursor functionality requires start=0");
+    }
+
+    for (SortField sf : sort.getSort()) {
+      if (sf.getType().equals(SortField.Type.DOC)) {
+        throw new SolrException(ErrorCode.BAD_REQUEST,
+                                "Cursor functionality can not be used with internal doc ordering sort: _docid_");
+      }
+    }
+
+    if (sort.getSort().length != sortSpec.getSchemaFields().size()) {
+        throw new SolrException(ErrorCode.SERVER_ERROR,
+                                "Cursor SortSpec failure: sort length != SchemaFields: " 
+                                + sort.getSort().length + " != " + 
+                                sortSpec.getSchemaFields().size());
+    }
+
+    this.sortSpec = sortSpec;
+    this.values = null;
+  }
+
+  /**
+   * Generates an empty CursorMark bound for use with the same {@link SortSpec}
+   * as the specified existing CursorMark.
+   *
+   * @param previous Existing CursorMark whose SortSpec will be reused in the new CursorMark.
+   * @see #createNext
+   */
+  private CursorMark(CursorMark previous) {
+    this.sortSpec = previous.sortSpec;
+    this.values = null;
+  }
+
+  /**
+   * Generates an new CursorMark bound for use with the same {@link SortSpec}
+   * as the current CursorMark but using the new SortValues.
+   *
+   */
+  public CursorMark createNext(List<Object> nextSortValues) {
+    final CursorMark next = new CursorMark(this);
+    next.setSortValues(nextSortValues);
+    return next;
+  }
+
+
+  /**
+   * Sets the (raw, unmarshalled) sort values (which must conform to the existing 
+   * sortSpec) to populate this object.  If null, then there is nothing to 
+   * "search after" and the "first page" of results should be returned.
+   */
+  public void setSortValues(List<Object> input) {
+    if (null == input) {
+      this.values = null;
+    } else {
+      assert input.size() == sortSpec.getSort().getSort().length;
+      // defensive copy
+      this.values = new ArrayList<Object>(input);
+    }
+  }
+
+  /**
+   * Returns a copy of the (raw, unmarshalled) sort values used by this object, or 
+   * null if first page of docs should be returned (ie: no sort after)
+   */
+  public List<Object> getSortValues() {
+    // defensive copy
+    return null == this.values ? null : new ArrayList<Object>(this.values);
+  }
+
+  /**
+   * Returns the SortSpec used by this object.
+   */
+  public SortSpec getSortSpec() {
+    return this.sortSpec;
+  }
+
+  /**
+   * Parses the serialized version of a CursorMark from a client 
+   * (which must conform to the existing sortSpec) and populates this object.
+   *
+   * @see #getSerializedTotem
+   */
+  public void parseSerializedTotem(final String serialized) {
+    if (CURSOR_MARK_START.equals(serialized)) {
+      values = null;
+      return;
+    }
+    final SortField[] sortFields = sortSpec.getSort().getSort();
+    final List<SchemaField> schemaFields = sortSpec.getSchemaFields();
+
+    List<Object> pieces = null;
+    try {
+      final byte[] rawData = Base64.base64ToByteArray(serialized);
+      ByteArrayInputStream in = new ByteArrayInputStream(rawData);
+      try {
+        pieces = (List<Object>) codec.unmarshal(in);
+      } finally {
+        in.close();
+      }
+    } catch (Exception ex) {
+      throw new SolrException(ErrorCode.BAD_REQUEST,
+                              "Unable to parse '"+CURSOR_MARK_PARAM+"' after totem: " + 
+                              "value must either be '"+CURSOR_MARK_START+"' or the " + 
+                              "'"+CURSOR_MARK_NEXT+"' returned by a previous search: "
+                              + serialized, ex);
+    }
+    assert null != pieces : "pieces wasn't parsed?";
+
+    if (sortFields.length != pieces.size()) {
+      throw new SolrException(ErrorCode.BAD_REQUEST,
+                              CURSOR_MARK_PARAM+" does not work with current sort (wrong size): " + serialized);
+    }
+
+
+    this.values = new ArrayList<Object>(sortFields.length);
+
+    final BytesRef tmpBytes = new BytesRef();
+    for (int i = 0; i < sortFields.length; i++) {
+
+      SortField curSort = sortFields[i];
+      SchemaField curField = schemaFields.get(i);
+      Object rawValue = pieces.get(i);
+
+      if (null != curField) {
+        FieldType curType = curField.getType();
+        rawValue = curType.unmarshalSortValue(rawValue);
+      } 
+
+      this.values.add(rawValue);
+    }
+  }
+  
+  /**
+   * Generates a Base64 encoded serialized representation of the sort values 
+   * encapsulated by this object, for use in cursor requests.
+   *
+   * @see #parseSerializedTotem
+   */
+  public String getSerializedTotem() {
+    if (null == this.values) {
+      return CURSOR_MARK_START;
+    }
+
+    final List<SchemaField> schemaFields = sortSpec.getSchemaFields();
+    final ArrayList<Object> marshalledValues = new ArrayList<Object>(values.size()+1);
+    for (int i = 0; i < schemaFields.size(); i++) {
+      SchemaField fld = schemaFields.get(i);
+      Object safeValue = values.get(i);
+      if (null != fld) {
+        FieldType type = fld.getType();
+        safeValue = type.marshalSortValue(safeValue);
+      }
+      marshalledValues.add(safeValue);
+    }
+
+    // TODO: we could also encode info about the SortSpec for error checking:
+    // the type/name/dir from the SortFields (or a hashCode to act as a checksum) 
+    // could help provide more validation beyond just the number of clauses.
+
+    try {
+      ByteArrayOutputStream out = new ByteArrayOutputStream(256);
+      try {
+        codec.marshal(marshalledValues, out);
+        byte[] rawData = out.toByteArray();
+        return Base64.byteArrayToBase64(rawData, 0, rawData.length);
+      } finally {
+        out.close();
+      }
+    } catch (Exception ex) {
+      throw new SolrException(ErrorCode.SERVER_ERROR,
+                              "Unable to format search after totem", ex);
+      
+    }
+  }
+
+  /**
+   * Returns a synthetically constructed {@link FieldDoc} whose {@link FieldDoc#fields} 
+   * match the values of this object.  
+   * <p>
+   * Important Notes:
+   * </p>
+   * <ul>
+   *  <li>{@link FieldDoc#doc} will always be set to {@link Integer#MAX_VALUE} so 
+   *    that the tie breaking logic used by <code>IndexSearcher</code> won't select 
+   *    the same doc again based on the internal lucene docId when the Solr 
+   *    <code>uniqueKey</code> value is the same.
+   *  </li>
+   *  <li>{@link FieldDoc#score} will always be set to 0.0F since it is not used
+   *    when applying <code>searchAfter</code> logic. (Even if the sort values themselves 
+   *    contain scores which are used in the sort)
+   *  </li>
+   * </ul>
+   *
+   * @return a {@link FieldDoc} to "search after" or null if the initial 
+   *         page of results is requested.
+   */
+  public FieldDoc getSearchAfterFieldDoc() {
+    if (null == values) return null;
+
+    return new FieldDoc(Integer.MAX_VALUE, 0.0F, values.toArray());
+  }
+
+}

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/QParser.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/QParser.java?rev=1556036&r1=1556035&r2=1556036&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/QParser.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/QParser.java Mon Jan  6 21:15:45 2014
@@ -17,7 +17,6 @@
 package org.apache.solr.search;
 
 import org.apache.lucene.search.Query;
-import org.apache.lucene.search.ScoreDoc; //Issue 1726
 import org.apache.lucene.search.Sort;
 import org.apache.solr.common.params.CommonParams;
 import org.apache.solr.common.params.MapSolrParams;
@@ -209,37 +208,6 @@ public abstract class QParser {
   }
 
   /**
-   * use common params to look up pageScore and pageDoc in global params
-   * @return the ScoreDoc
-   */
-  public ScoreDoc getPaging() throws SyntaxError
-  {
-    return null;
-
-    /*** This is not ready for prime-time... see SOLR-1726
-
-    String pageScoreS = null;
-    String pageDocS = null;
-
-    pageScoreS = params.get(CommonParams.PAGESCORE);
-    pageDocS = params.get(CommonParams.PAGEDOC);
-
-    if (pageScoreS == null || pageDocS == null)
-      return null;
-
-    int pageDoc = pageDocS != null ? Integer.parseInt(pageDocS) : -1;
-    float pageScore = pageScoreS != null ? new Float(pageScoreS) : -1;
-    if(pageDoc != -1 && pageScore != -1){
-      return new ScoreDoc(pageDoc, pageScore);
-    }
-    else {
-      return null;
-    }
-
-    ***/
-  }
-  
-  /**
    * @param useGlobalParams look up sort, start, rows in global params if not in local params
    * @return the sort specification
    */

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java?rev=1556036&r1=1556035&r2=1556036&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java Mon Jan  6 21:15:45 2014
@@ -70,6 +70,7 @@ import org.apache.lucene.search.Filter;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.MatchAllDocsQuery;
 import org.apache.lucene.search.Query;
+import org.apache.lucene.search.FieldDoc;
 import org.apache.lucene.search.ScoreDoc;
 import org.apache.lucene.search.Scorer;
 import org.apache.lucene.search.Sort;
@@ -77,6 +78,7 @@ import org.apache.lucene.search.SortFiel
 import org.apache.lucene.search.TermQuery;
 import org.apache.lucene.search.TimeLimitingCollector;
 import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.search.TopFieldDocs;
 import org.apache.lucene.search.TopDocsCollector;
 import org.apache.lucene.search.TopFieldCollector;
 import org.apache.lucene.search.TopScoreDocCollector;
@@ -1346,6 +1348,7 @@ public class SolrIndexSearcher extends I
         key = null;  // we won't be caching the result
       }
     }
+    cmd.setSupersetMaxDoc(supersetMaxDoc);
 
 
     // OK, so now we need to generate an answer.
@@ -1368,7 +1371,6 @@ public class SolrIndexSearcher extends I
       }
     }
 
-    // disable useFilterCache optimization temporarily
     if (useFilterCache) {
       // now actually use the filter cache.
       // for large filters that match few documents, this may be
@@ -1381,11 +1383,9 @@ public class SolrIndexSearcher extends I
       // todo: there could be a sortDocSet that could take a list of
       // the filters instead of anding them first...
       // perhaps there should be a multi-docset-iterator
-      superset = sortDocSet(out.docSet,cmd.getSort(),supersetMaxDoc);
-      out.docList = superset.subset(cmd.getOffset(),cmd.getLen());
+      sortDocSet(qr, cmd);
     } else {
       // do it the normal way...
-      cmd.setSupersetMaxDoc(supersetMaxDoc);
       if ((flags & GET_DOCSET)!=0) {
         // this currently conflates returning the docset for the base query vs
         // the base query and all filters.
@@ -1394,11 +1394,28 @@ public class SolrIndexSearcher extends I
         if (qDocSet!=null && filterCache!=null && !qr.isPartialResults()) filterCache.put(cmd.getQuery(),qDocSet);
       } else {
         getDocListNC(qr,cmd);
-        //Parameters: cmd.getQuery(),theFilt,cmd.getSort(),0,supersetMaxDoc,cmd.getFlags(),cmd.getTimeAllowed(),responseHeader);
       }
+      assert null != out.docList : "docList is null";
+    }
 
+    if (null == cmd.getCursorMark()) {
+      // Kludge...
+      // we can't use DocSlice.subset, even though it should be an identity op
+      // because it gets confused by situations where there are lots of matches, but
+      // less docs in the slice then were requested, (due to the cursor)
+      // so we have to short circuit the call.
+      // None of which is really a problem since we can't use caching with
+      // cursors anyway, but it still looks weird to have to special case this
+      // behavior based on this condition - hence the long explanation.
       superset = out.docList;
       out.docList = superset.subset(cmd.getOffset(),cmd.getLen());
+    } else {
+      // sanity check our cursor assumptions
+      assert null == superset : "cursor: superset isn't null";
+      assert 0 == cmd.getOffset() : "cursor: command offset mismatch";
+      assert 0 == out.docList.offset() : "cursor: docList offset mismatch";
+      assert cmd.getLen() >= supersetMaxDoc : "cursor: superset len mismatch: " +
+        cmd.getLen() + " vs " + supersetMaxDoc;
     }
 
     // lastly, put the superset in the cache if the size is less than or equal
@@ -1408,7 +1425,76 @@ public class SolrIndexSearcher extends I
     }
   }
 
+  /**
+   * Helper method for extracting the {@link FieldDoc} sort values from a 
+   * {@link TopFieldDocs} when available and making the appropriate call to 
+   * {@link QueryResult#setNextCursorMark} when applicable.
+   *
+   * @param qr <code>QueryResult</code> to modify
+   * @param qc <code>QueryCommand</code> for context of method
+   * @param topDocs May or may not be a <code>TopFieldDocs</code> 
+   */
+  private void populateNextCursorMarkFromTopDocs(QueryResult qr, QueryCommand qc, 
+                                                 TopDocs topDocs) {
+    // TODO: would be nice to rename & generalize this method for non-cursor cases...
+    // ...would be handy to reuse the ScoreDoc/FieldDoc sort vals directly in distrib sort
+    // ...but that has non-trivial queryResultCache implications
+    // See: SOLR-5595
+    
+    if (null == qc.getCursorMark()) {
+      // nothing to do, short circuit out
+      return;
+    }
 
+    final CursorMark lastCursorMark = qc.getCursorMark();
+    
+    // if we have a cursor, then we have a sort that at minimum involves uniqueKey..
+    // so we must have a TopFieldDocs containing FieldDoc[]
+    assert topDocs instanceof TopFieldDocs : "TopFieldDocs cursor constraint violated";
+    final TopFieldDocs topFieldDocs = (TopFieldDocs) topDocs;
+    final ScoreDoc[] scoreDocs = topFieldDocs.scoreDocs;
+
+    if (0 == scoreDocs.length) {
+      // no docs on this page, re-use existing cursor mark
+      qr.setNextCursorMark(lastCursorMark);
+    } else {
+      ScoreDoc lastDoc = scoreDocs[scoreDocs.length-1];
+      assert lastDoc instanceof FieldDoc : "FieldDoc cursor constraint violated";
+      
+      List<Object> lastFields = Arrays.<Object>asList(((FieldDoc)lastDoc).fields);
+      CursorMark nextCursorMark = lastCursorMark.createNext(lastFields);
+      assert null != nextCursorMark : "null nextCursorMark";
+      qr.setNextCursorMark(nextCursorMark);
+    }
+  }
+
+  /**
+   * Helper method for inspecting QueryCommand and creating the appropriate 
+   * {@link TopDocsCollector}
+   *
+   * @param len the number of docs to return
+   * @param cmd The Command whose properties should determine the type of 
+   *        TopDocsCollector to use.
+   */
+  private TopDocsCollector buildTopDocsCollector(int len, QueryCommand cmd) throws IOException {
+    
+    if (null == cmd.getSort()) {
+      assert null == cmd.getCursorMark() : "have cursor but no sort";
+      return TopScoreDocCollector.create(len, true);
+    } else {
+      // we have a sort
+      final boolean needScores = (cmd.getFlags() & GET_SCORES) != 0;
+      final Sort weightedSort = weightSort(cmd.getSort());
+      final CursorMark cursor = cmd.getCursorMark();
+
+      // :TODO: make fillFields it's own QueryCommand flag? ...
+      // ... see comments in populateNextCursorMarkFromTopDocs for cache issues (SOLR-5595)
+      final boolean fillFields = (null != cursor);
+      final FieldDoc searchAfter = (null != cursor ? cursor.getSearchAfterFieldDoc() : null);
+      return TopFieldCollector.create(weightedSort, len, searchAfter,
+                                      fillFields, needScores, needScores, true); 
+    }
+  }
 
   private void getDocListNC(QueryResult qr,QueryCommand cmd) throws IOException {
     final long timeAllowed = cmd.getTimeAllowed();
@@ -1503,18 +1589,10 @@ public class SolrIndexSearcher extends I
       scores = new float[nDocsReturned];
       totalHits = numHits[0];
       maxScore = totalHits>0 ? topscore[0] : 0.0f;
+      // no docs on this page, so cursor doesn't change
+      qr.setNextCursorMark(cmd.getCursorMark());
     } else {
-      TopDocsCollector topCollector;
-      if (cmd.getSort() == null) {
-        if(cmd.getScoreDoc() != null) {
-          topCollector = TopScoreDocCollector.create(len, cmd.getScoreDoc(), true); //create the Collector with InOrderPagingCollector
-        } else {
-          topCollector = TopScoreDocCollector.create(len, true);
-        }
-
-      } else {
-        topCollector = TopFieldCollector.create(weightSort(cmd.getSort()), len, false, needScores, needScores, true);
-      }
+      final TopDocsCollector topCollector = buildTopDocsCollector(len, cmd);
       Collector collector = topCollector;
       if (terminateEarly) {
         collector = new EarlyTerminatingCollector(collector, cmd.len);
@@ -1539,6 +1617,8 @@ public class SolrIndexSearcher extends I
 
       totalHits = topCollector.getTotalHits();
       TopDocs topDocs = topCollector.topDocs(0, len);
+      populateNextCursorMarkFromTopDocs(qr, cmd, topDocs);
+
       maxScore = totalHits>0 ? topDocs.getMaxScore() : 0.0f;
       nDocsReturned = topDocs.scoreDocs.length;
       ids = new int[nDocsReturned];
@@ -1639,16 +1719,11 @@ public class SolrIndexSearcher extends I
       scores = new float[nDocsReturned];
       totalHits = set.size();
       maxScore = totalHits>0 ? topscore[0] : 0.0f;
+      // no docs on this page, so cursor doesn't change
+      qr.setNextCursorMark(cmd.getCursorMark());
     } else {
 
-      TopDocsCollector topCollector;
-
-      if (cmd.getSort() == null) {
-        topCollector = TopScoreDocCollector.create(len, true);
-      } else {
-        topCollector = TopFieldCollector.create(weightSort(cmd.getSort()), len, false, needScores, needScores, true);
-      }
-
+      final TopDocsCollector topCollector = buildTopDocsCollector(len, cmd);
       DocSetCollector setCollector = new DocSetDelegateCollector(maxDoc>>6, maxDoc, topCollector);
       Collector collector = setCollector;
       if (terminateEarly) {
@@ -1678,6 +1753,7 @@ public class SolrIndexSearcher extends I
       assert(totalHits == set.size());
 
       TopDocs topDocs = topCollector.topDocs(0, len);
+      populateNextCursorMarkFromTopDocs(qr, cmd, topDocs);
       maxScore = totalHits>0 ? topDocs.getMaxScore() : 0.0f;
       nDocsReturned = topDocs.scoreDocs.length;
 
@@ -1926,16 +2002,21 @@ public class SolrIndexSearcher extends I
     return qr.getDocListAndSet();
   }
 
-  protected DocList sortDocSet(DocSet set, Sort sort, int nDocs) throws IOException {
+  protected void sortDocSet(QueryResult qr, QueryCommand cmd) throws IOException {
+    DocSet set = qr.getDocListAndSet().docSet;
+    int nDocs = cmd.getSupersetMaxDoc();
     if (nDocs == 0) {
       // SOLR-2923
-      return new DocSlice(0, 0, new int[0], null, 0, 0f);
+      qr.getDocListAndSet().docList = new DocSlice(0, 0, new int[0], null, set.size(), 0f);
+      qr.setNextCursorMark(cmd.getCursorMark());
+      return;
     }
 
+
     // bit of a hack to tell if a set is sorted - do it better in the future.
     boolean inOrder = set instanceof BitDocSet || set instanceof SortedIntDocSet;
 
-    TopDocsCollector topCollector = TopFieldCollector.create(weightSort(sort), nDocs, false, false, false, inOrder);
+    TopDocsCollector topCollector = buildTopDocsCollector(nDocs, cmd);
 
     DocIterator iter = set.iterator();
     int base=0;
@@ -1964,7 +2045,8 @@ public class SolrIndexSearcher extends I
       ids[i] = scoreDoc.doc;
     }
 
-    return new DocSlice(0,nDocsReturned,ids,null,topDocs.totalHits,0.0f);
+    qr.getDocListAndSet().docList = new DocSlice(0,nDocsReturned,ids,null,topDocs.totalHits,0.0f);
+    populateNextCursorMarkFromTopDocs(qr, cmd, topDocs);
   }
 
 
@@ -2188,20 +2270,27 @@ public class SolrIndexSearcher extends I
     private int supersetMaxDoc;
     private int flags;
     private long timeAllowed = -1;
-    //Issue 1726 start
-    private ScoreDoc scoreDoc;
+    private CursorMark cursorMark;
     
-    public ScoreDoc getScoreDoc()
-    {
-      return scoreDoc;
+    public CursorMark getCursorMark() {
+      return cursorMark;
     }
-    public void setScoreDoc(ScoreDoc scoreDoc)
-    {
-      this.scoreDoc = scoreDoc;
+    public QueryCommand setCursorMark(CursorMark cursorMark) {
+      this.cursorMark = cursorMark;
+      if (null != cursorMark) {
+        // If we're using a cursor then we can't allow queryResult caching because the 
+        // cache keys don't know anything about the collector used.
+        //
+        // in theory, we could enhance the cache keys to be aware of the searchAfter
+        // FieldDoc but then there would still be complexity around things like the cache
+        // window size that would need to be worked out
+        //
+        // we *can* however allow the use of checking the filterCache for non-score based
+        // sorts, because that still runs our paging collector over the entire DocSet
+        this.flags |= (NO_CHECK_QCACHE | NO_SET_QCACHE);
+      }
+      return this;
     }
-    //Issue 1726 end
-
-    // public List<Grouping.Command> groupCommands;
 
     public Query getQuery() { return query; }
     public QueryCommand setQuery(Query query) {
@@ -2310,6 +2399,7 @@ public class SolrIndexSearcher extends I
   public static class QueryResult {
     private boolean partialResults;
     private DocListAndSet docListAndSet;
+    private CursorMark nextCursorMark;
 
     public Object groupedResults;   // TODO: currently for testing
     
@@ -2334,6 +2424,13 @@ public class SolrIndexSearcher extends I
 
     public void setDocListAndSet( DocListAndSet listSet ) { docListAndSet = listSet; }
     public DocListAndSet getDocListAndSet() { return docListAndSet; }
+
+    public void setNextCursorMark(CursorMark next) {
+      this.nextCursorMark = next;
+    }
+    public CursorMark getNextCursorMark() {
+      return nextCursorMark;
+    }
   }
 
 }

Added: lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/schema-sorts.xml
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/schema-sorts.xml?rev=1556036&view=auto
==============================================================================
--- lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/schema-sorts.xml (added)
+++ lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/schema-sorts.xml Mon Jan  6 21:15:45 2014
@@ -0,0 +1,139 @@
+<?xml version="1.0" ?>
+<!--
+Licensed to the Apache Software Foundation (ASF) under one or more
+contributor license agreements.  See the NOTICE file distributed with
+this work for additional information regarding copyright ownership.
+The ASF licenses this file to You under the Apache License, Version 2.0
+(the "License"); you may not use this file except in compliance with
+the License.  You may obtain a copy of the License at
+
+http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+-->
+
+<!-- 
+
+NOTE: Tests expect every field in this schema to be sortable.
+
+-->
+
+<schema name="sortable" version="1.5">
+  <uniqueKey>id</uniqueKey>
+
+  <fields>
+    <field name="id" type="int" />
+    <field name="_version_" type="long" />
+
+    <field name="str" type="str" />
+    <field name="str_last" type="str_last" />
+    <field name="str_first" type="str_first" />
+    <field name="str_dv_last" type="str_dv_last" />
+    <field name="str_dv_first" type="str_dv_first" />
+
+    <field name="bin" type="bin" />
+    <field name="bin_last" type="bin_last" />
+    <field name="bin_first" type="bin_first" />
+    <field name="bin_dv_last" type="bin_dv_last" />
+    <field name="bin_dv_first" type="bin_dv_first" />
+
+    <field name="int" type="int" />
+    <field name="int_last" type="int_last" />
+    <field name="int_first" type="int_first" />
+    <field name="int_dv_last" type="int_dv_last" />
+    <field name="int_dv_first" type="int_dv_first" />
+
+    <field name="long" type="long" />
+    <field name="long_last" type="long_last" />
+    <field name="long_first" type="long_first" />
+    <field name="long_dv_last" type="long_dv_last" />
+    <field name="long_dv_first" type="long_dv_first" />
+
+    <field name="float" type="float" />
+    <field name="float_last" type="float_last" />
+    <field name="float_first" type="float_first" />
+    <field name="float_dv_last" type="float_dv_last" />
+    <field name="float_dv_first" type="float_dv_first" />
+
+    <field name="double" type="double" />
+    <field name="double_last" type="double_last" />
+    <field name="double_first" type="double_first" />
+    <field name="double_dv_last" type="double_dv_last" />
+    <field name="double_dv_first" type="double_dv_first" />
+
+    <!-- ensure function sorts don't mistakenly get interpreted as field sorts
+         https://issues.apache.org/jira/browse/SOLR-5354?focusedCommentId=13835891&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-13835891
+    -->
+    <dynamicField name="*" type="str" multiValued="true" />
+  </fields>
+
+  <copyField source="str" dest="str_last" />
+  <copyField source="str" dest="str_first" />
+  <copyField source="str" dest="str_dv_last" />
+  <copyField source="str" dest="str_dv_first" />
+  <copyField source="bin" dest="bin_last" />
+  <copyField source="bin" dest="bin_first" />
+  <copyField source="bin" dest="bin_dv_last" />
+  <copyField source="bin" dest="bin_dv_first" />
+  <copyField source="int" dest="int_last" />
+  <copyField source="int" dest="int_first" />
+  <copyField source="int" dest="int_dv_last" />
+  <copyField source="int" dest="int_dv_first" />
+  <copyField source="long" dest="long_last" />
+  <copyField source="long" dest="long_first" />
+  <copyField source="long" dest="long_dv_last" />
+  <copyField source="long" dest="long_dv_first" />
+  <copyField source="float" dest="float_last" />
+  <copyField source="float" dest="float_first" />
+  <copyField source="float" dest="float_dv_last" />
+  <copyField source="float" dest="float_dv_first" />
+  <copyField source="double" dest="double_last" />
+  <copyField source="double" dest="double_first" />
+  <copyField source="double" dest="double_dv_last" />
+  <copyField source="double" dest="double_dv_first" />
+
+  <types>
+
+    <fieldtype name="str" class="solr.StrField" stored="true" indexed="true" />
+    <fieldtype name="str_last" class="solr.StrField" stored="true" indexed="true" sortMissingLast="true"/>
+    <fieldtype name="str_first" class="solr.StrField" stored="true" indexed="true" sortMissingFirst="true"/>
+    <fieldtype name="str_dv_last" class="solr.StrField" stored="true" indexed="false" docValues="true" sortMissingLast="true"/>
+    <fieldtype name="str_dv_first" class="solr.StrField" stored="true" indexed="false" docValues="true" sortMissingFirst="true"/>
+
+    <fieldtype name="bin" class="solr.SortableBinaryField" stored="true" indexed="true" />
+    <fieldtype name="bin_last" class="solr.SortableBinaryField" stored="true" indexed="true" sortMissingLast="true"/>
+    <fieldtype name="bin_first" class="solr.SortableBinaryField" stored="true" indexed="true" sortMissingFirst="true"/>
+    <fieldtype name="bin_dv_last" class="solr.SortableBinaryField" stored="true" indexed="false" docValues="true" sortMissingLast="true"/>
+    <fieldtype name="bin_dv_first" class="solr.SortableBinaryField" stored="true" indexed="false" docValues="true" sortMissingFirst="true"/>
+
+    <fieldtype name="int" class="solr.TrieIntField" stored="true" indexed="true" />
+    <fieldtype name="int_last" class="solr.TrieIntField" stored="true" indexed="true" sortMissingLast="true"/>
+    <fieldtype name="int_first" class="solr.TrieIntField" stored="true" indexed="true" sortMissingFirst="true"/>
+    <fieldtype name="int_dv_last" class="solr.TrieIntField" stored="true" indexed="false" docValues="true" sortMissingLast="true"/>
+    <fieldtype name="int_dv_first" class="solr.TrieIntField" stored="true" indexed="false" docValues="true" sortMissingFirst="true"/>
+
+    <fieldtype name="long" class="solr.TrieLongField" stored="true" indexed="true" />
+    <fieldtype name="long_last" class="solr.TrieLongField" stored="true" indexed="true" sortMissingLast="true"/>
+    <fieldtype name="long_first" class="solr.TrieLongField" stored="true" indexed="true" sortMissingFirst="true"/>
+    <fieldtype name="long_dv_last" class="solr.TrieLongField" stored="true" indexed="false" docValues="true" sortMissingLast="true"/>
+    <fieldtype name="long_dv_first" class="solr.TrieLongField" stored="true" indexed="false" docValues="true" sortMissingFirst="true"/>
+
+    <fieldtype name="float" class="solr.TrieFloatField" stored="true" indexed="true" />
+    <fieldtype name="float_last" class="solr.TrieFloatField" stored="true" indexed="true" sortMissingLast="true"/>
+    <fieldtype name="float_first" class="solr.TrieFloatField" stored="true" indexed="true" sortMissingFirst="true"/>
+    <fieldtype name="float_dv_last" class="solr.TrieFloatField" stored="true" indexed="false" docValues="true" sortMissingLast="true"/>
+    <fieldtype name="float_dv_first" class="solr.TrieFloatField" stored="true" indexed="false" docValues="true" sortMissingFirst="true"/>
+
+    <fieldtype name="double" class="solr.TrieDoubleField" stored="true" indexed="true" />
+    <fieldtype name="double_last" class="solr.TrieDoubleField" stored="true" indexed="true" sortMissingLast="true"/>
+    <fieldtype name="double_first" class="solr.TrieDoubleField" stored="true" indexed="true" sortMissingFirst="true"/>
+    <fieldtype name="double_dv_last" class="solr.TrieDoubleField" stored="true" indexed="false" docValues="true" sortMissingLast="true"/>
+    <fieldtype name="double_dv_first" class="solr.TrieDoubleField" stored="true" indexed="false" docValues="true" sortMissingFirst="true"/>
+
+  </types>
+
+</schema>

Added: lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/solrconfig-deeppaging.xml
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/solrconfig-deeppaging.xml?rev=1556036&view=auto
==============================================================================
--- lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/solrconfig-deeppaging.xml (added)
+++ lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/solrconfig-deeppaging.xml Mon Jan  6 21:15:45 2014
@@ -0,0 +1,59 @@
+<?xml version="1.0" ?>
+
+<!--
+ Licensed to the Apache Software Foundation (ASF) under one or more
+ contributor license agreements.  See the NOTICE file distributed with
+ this work for additional information regarding copyright ownership.
+ The ASF licenses this file to You under the Apache License, Version 2.0
+ (the "License"); you may not use this file except in compliance with
+ the License.  You may obtain a copy of the License at
+
+     http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+
+<config>
+
+  <dataDir>${solr.data.dir:}</dataDir>
+
+  <directoryFactory name="DirectoryFactory"
+                    class="${solr.directoryFactory:solr.NRTCachingDirectoryFactory}"/>
+
+  <luceneMatchVersion>${tests.luceneMatchVersion:LUCENE_CURRENT}</luceneMatchVersion>
+
+  <xi:include href="solrconfig.snippet.randomindexconfig.xml" xmlns:xi="http://www.w3.org/2001/XInclude"/>
+
+  <updateHandler class="solr.DirectUpdateHandler2">
+    <updateLog>
+      <str name="dir">${solr.ulog.dir:}</str>
+    </updateLog>
+  </updateHandler>
+  
+  <!-- deep paging better play nice with caching -->
+  <query>
+    <!-- no wautowarming, it screws up our ability to sanity check cache stats in tests -->
+    <filterCache class="solr.FastLRUCache" size="50" initialSize="50" autowarmCount="0"/>
+    <queryResultCache class="solr.LRUCache" size="50" initialSize="50" autowarmCount="0"/>
+    <queryResultWindowSize>50</queryResultWindowSize>
+    <queryResultMaxDocsCached>500</queryResultMaxDocsCached>
+    <!-- randomized so we excersize cursors using various paths in SolrIndexSearcher -->
+    <useFilterForSortedQuery>${solr.test.useFilterForSortedQuery}</useFilterForSortedQuery>
+  </query>
+
+
+  <requestHandler name="/get" class="solr.RealTimeGetHandler">
+    <lst name="defaults">
+      <str name="omitHeader">true</str>
+    </lst>
+  </requestHandler>
+  <requestHandler name="/replication" class="solr.ReplicationHandler" startup="lazy" /> 
+  <requestHandler name="/select" class="solr.SearchHandler" default="true" />
+  <requestHandler name="/update" class="solr.UpdateRequestHandler"  />
+  <requestHandler name="/admin/" class="solr.admin.AdminHandlers"  />
+</config>
+

Modified: lucene/dev/trunk/solr/core/src/test/org/apache/solr/BasicFunctionalityTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/BasicFunctionalityTest.java?rev=1556036&r1=1556035&r2=1556036&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/BasicFunctionalityTest.java (original)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/BasicFunctionalityTest.java Mon Jan  6 21:15:45 2014
@@ -947,51 +947,6 @@ public class BasicFunctionalityTest exte
     }
   }
 
-  @Ignore("See SOLR-1726")
-  @Test
-  public void testDeepPaging() throws Exception {
-    for (int i = 0; i < 1000; i++){
-      assertU(adoc("id", String.valueOf(i),  "foo_t", English.intToEnglish(i)));
-    }
-    assertU(commit());
-    SolrQueryRequest goldReq = null;
-    try {
-      goldReq = req("q", "foo_t:one", "rows", "50", "fl", "docid, score");
-      SolrQueryResponse gold = h.queryAndResponse("standard", goldReq);
-      ResultContext response = (ResultContext) gold.getValues().get("response");
-      assertQ("page: " + 0 + " failed",
-          req("q", "foo_t:one", "rows", "10", CommonParams.QT, "standard", "fl", "[docid], score"),
-          "*[count(//doc)=10]");
-      //ugh, what a painful way to get the document
-      DocIterator iterator = response.docs.subset(9, 1).iterator();
-      int lastDoc = iterator.nextDoc();
-      float lastScore = iterator.score();
-      for (int i = 1; i < 5; i++){
-        //page through some results
-        DocList subset = response.docs.subset(i * 10, 1);
-        iterator = subset.iterator();
-        int compareDoc = iterator.nextDoc();
-        float compareScore = iterator.score();
-        assertQ("page: " + i + " failed",
-            req("q", "foo_t:one", CommonParams.QT, "standard", "fl", "[docid], score",
-                "start", String.valueOf(i * 10), "rows", "1",  //only get one doc, and then compare it to gold
-                CommonParams.PAGEDOC, String.valueOf(lastDoc), CommonParams.PAGESCORE, String.valueOf(lastScore)),
-            "*[count(//doc)=1]",
-            "//float[@name='score'][.='" + compareScore + "']",
-            "//int[@name='[docid]'][.='" + compareDoc + "']"
-        );
-        lastScore = compareScore;
-        lastDoc = compareDoc;
-
-      }
-    } finally {
-      if (goldReq != null ) {
-        goldReq.close();
-      }
-    }
-  }
-
-
 //   /** this doesn't work, but if it did, this is how we'd test it. */
 //   public void testOverwriteFalse() {