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

[1/3] lucene-solr:branch_6x: SOLR-9717: Refactor '/export' to not hardcode the JSON output and to use an API

Repository: lucene-solr
Updated Branches:
  refs/heads/branch_6x 0f0526f9b -> e6dfb76c9


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e6dfb76c/solr/solrj/src/java/org/apache/solr/common/PushWriter.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/common/PushWriter.java b/solr/solrj/src/java/org/apache/solr/common/PushWriter.java
new file mode 100644
index 0000000..ddfac3c
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/common/PushWriter.java
@@ -0,0 +1,42 @@
+/*
+ * 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.common;
+
+
+import java.io.Closeable;
+import java.io.IOException;
+
+/**This is an interface to stream data out using a push API
+ *
+ */
+public interface PushWriter extends Closeable {
+
+  /**Write a Map. The map is opened in the beginning of the method
+   * and closed at the end. All map entries MUST be written before this
+   * method returns
+   */
+  void writeMap(MapWriter mw) throws IOException;
+
+  /**Write an array. The array is opened at the beginning of this method
+   * and closed at the end. All array entries must be returned before this
+   * method returns
+   *
+   */
+  void writeIterator(IteratorWriter iw) throws IOException;
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e6dfb76c/solr/test-framework/src/java/org/apache/solr/util/TestHarness.java
----------------------------------------------------------------------
diff --git a/solr/test-framework/src/java/org/apache/solr/util/TestHarness.java b/solr/test-framework/src/java/org/apache/solr/util/TestHarness.java
index b293ecb..becffd5 100644
--- a/solr/test-framework/src/java/org/apache/solr/util/TestHarness.java
+++ b/solr/test-framework/src/java/org/apache/solr/util/TestHarness.java
@@ -16,6 +16,7 @@
  */
 package org.apache.solr.util;
 
+import java.io.ByteArrayOutputStream;
 import java.io.IOException;
 import java.io.StringWriter;
 import java.nio.file.Path;
@@ -30,12 +31,22 @@ import org.apache.solr.common.SolrException;
 import org.apache.solr.common.params.CommonParams;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.NamedList.NamedListEntry;
-import org.apache.solr.core.*;
+import org.apache.solr.core.CloudConfig;
+import org.apache.solr.core.CoreContainer;
+import org.apache.solr.core.CoreDescriptor;
+import org.apache.solr.core.CorePropertiesLocator;
+import org.apache.solr.core.CoresLocator;
+import org.apache.solr.core.NodeConfig;
+import org.apache.solr.core.SolrConfig;
+import org.apache.solr.core.SolrCore;
+import org.apache.solr.core.SolrResourceLoader;
+import org.apache.solr.core.SolrXmlConfig;
 import org.apache.solr.handler.UpdateRequestHandler;
 import org.apache.solr.request.LocalSolrQueryRequest;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.request.SolrRequestHandler;
 import org.apache.solr.request.SolrRequestInfo;
+import org.apache.solr.response.BinaryQueryResponseWriter;
 import org.apache.solr.response.QueryResponseWriter;
 import org.apache.solr.response.SolrQueryResponse;
 import org.apache.solr.schema.IndexSchema;
@@ -311,10 +322,18 @@ public class TestHarness extends BaseTestHarness {
       if (rsp.getException() != null) {
         throw rsp.getException();
       }
-      StringWriter sw = new StringWriter(32000);
       QueryResponseWriter responseWriter = core.getQueryResponseWriter(req);
-      responseWriter.write(sw,req,rsp);
-      return sw.toString();
+      if (responseWriter instanceof BinaryQueryResponseWriter) {
+        ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream(32000);
+        BinaryQueryResponseWriter writer = (BinaryQueryResponseWriter) responseWriter;
+        writer.write(byteArrayOutputStream, req, rsp);
+        return new String(byteArrayOutputStream.toByteArray(), "UTF-8");
+      } else {
+        StringWriter sw = new StringWriter(32000);
+        responseWriter.write(sw,req,rsp);
+        return sw.toString();
+      }
+
     } finally {
       req.close();
       SolrRequestInfo.clearRequestInfo();


[2/3] lucene-solr:branch_6x: SOLR-9717: Refactor '/export' to not hardcode the JSON output and to use an API

Posted by no...@apache.org.
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e6dfb76c/solr/core/src/java/org/apache/solr/response/SortingResponseWriter.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/response/SortingResponseWriter.java b/solr/core/src/java/org/apache/solr/response/SortingResponseWriter.java
deleted file mode 100644
index 0e4a4a4..0000000
--- a/solr/core/src/java/org/apache/solr/response/SortingResponseWriter.java
+++ /dev/null
@@ -1,1718 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.solr.response;
-
-import java.io.IOException;
-import java.io.PrintWriter;
-import java.io.Writer;
-import java.lang.invoke.MethodHandles;
-import java.util.Date;
-import java.util.List;
-import java.util.ArrayList;
-
-import org.apache.lucene.index.DocValues;
-import org.apache.lucene.index.LeafReader;
-import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.index.MultiDocValues;
-import org.apache.lucene.index.NumericDocValues;
-import org.apache.lucene.index.SortedDocValues;
-import org.apache.lucene.index.SortedSetDocValues;
-import org.apache.lucene.search.DocIdSetIterator;
-import org.apache.lucene.search.Sort;
-import org.apache.lucene.search.SortField;
-import org.apache.lucene.util.ArrayUtil;
-import org.apache.lucene.util.BitSetIterator;
-import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.CharsRefBuilder;
-import org.apache.lucene.util.FixedBitSet;
-import org.apache.lucene.util.LongValues;
-import org.apache.solr.common.SolrException;
-import org.apache.solr.common.params.SolrParams;
-import org.apache.solr.common.util.NamedList;
-import org.apache.solr.request.SolrQueryRequest;
-import org.apache.solr.request.SolrRequestInfo;
-import org.apache.solr.schema.BoolField;
-import org.apache.solr.schema.FieldType;
-import org.apache.solr.schema.IndexSchema;
-import org.apache.solr.schema.SchemaField;
-import org.apache.solr.schema.StrField;
-import org.apache.solr.schema.TrieDateField;
-import org.apache.solr.schema.TrieDoubleField;
-import org.apache.solr.schema.TrieFloatField;
-import org.apache.solr.schema.TrieIntField;
-import org.apache.solr.schema.TrieLongField;
-import org.apache.solr.search.SolrIndexSearcher;
-import org.apache.solr.search.SortSpec;
-import org.apache.solr.search.SyntaxError;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-
-public class SortingResponseWriter implements QueryResponseWriter {
-
-  private static final Logger logger = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-
-  public void init(NamedList args) {
-    /* NOOP */
-  }
-
-  public String getContentType(SolrQueryRequest req, SolrQueryResponse res) {
-    return "application/json";
-  }
-
-  public void write(Writer writer, SolrQueryRequest req, SolrQueryResponse res) throws IOException {
-    Exception e1 = res.getException();
-    if(e1 != null) {
-      if(!(e1 instanceof IgnoreException)) {
-        writeException(e1, writer, false);
-      }
-      return;
-    }
-
-    SolrRequestInfo info = SolrRequestInfo.getRequestInfo();
-    SortSpec sortSpec = info.getResponseBuilder().getSortSpec();
-
-    if(sortSpec == null) {
-      writeException((new IOException(new SyntaxError("No sort criteria was provided."))), writer, true);
-      return;
-    }
-
-    SolrIndexSearcher searcher = req.getSearcher();
-    Sort sort = searcher.weightSort(sortSpec.getSort());
-
-    if(sort == null) {
-      writeException((new IOException(new SyntaxError("No sort criteria was provided."))), writer, true);
-      return;
-    }
-
-    if(sort != null && sort.needsScores()) {
-      writeException((new IOException(new SyntaxError("Scoring is not currently supported with xsort."))), writer, true);
-      return;
-    }
-
-    // There is a bailout in SolrIndexSearcher.getDocListNC when there are _no_ docs in the index at all.
-    // if (lastDocRequested <= 0) {
-    // That causes the totalHits and export entries in the context to _not_ get set.
-    // The only time that really matters is when we search against an _empty_ set. That's too obscure
-    // a condition to handle as part of this patch, if someone wants to pursue it it can be reproduced with:
-    // ant test  -Dtestcase=StreamingTest -Dtests.method=testAllValidExportTypes -Dtests.seed=10F13879D0D1D6AD -Dtests.slow=true -Dtests.locale=es-PA -Dtests.timezone=America/Bahia_Banderas -Dtests.asserts=true -Dtests.file.encoding=ISO-8859-1
-    // You'll have to uncomment the if below to hit the null pointer exception.
-    // This is such an unusual case (i.e. an empty index) that catching this concdition here is probably OK.
-    // This came to light in the very artifical case of indexing a single doc to Cloud.
-    int totalHits = 0;
-    FixedBitSet[] sets = null;
-    if (req.getContext().get("totalHits") != null) {
-      totalHits = ((Integer)req.getContext().get("totalHits")).intValue();
-      sets = (FixedBitSet[]) req.getContext().get("export");
-      if (sets == null) {
-        writeException((new IOException(new SyntaxError("xport RankQuery is required for xsort: rq={!xport}"))), writer, true);
-        return;
-      }
-    }
-    SolrParams params = req.getParams();
-    String fl = params.get("fl");
-
-    String[] fields = null;
-
-    if(fl == null) {
-      writeException((new IOException(new SyntaxError("export field list (fl) must be specified."))), writer, true);
-      return;
-    } else  {
-      fields = fl.split(",");
-
-      for(int i=0;i<fields.length; i++) {
-
-        fields[i] = fields[i].trim();
-
-        if(fields[i].equals("score")) {
-          writeException((new IOException(new SyntaxError("Scoring is not currently supported with xsort."))), writer, true);
-          return;
-        }
-      }
-    }
-
-    FieldWriter[] fieldWriters = null;
-
-    try {
-      fieldWriters = getFieldWriters(fields, req.getSearcher());
-    } catch (Exception e) {
-      writeException(e, writer, true);
-      return;
-    }
-
-    writer.write("{\"responseHeader\": {\"status\": 0}, \"response\":{\"numFound\":"+totalHits+", \"docs\":[");
-
-
-    //Write the data.
-    List<LeafReaderContext> leaves = req.getSearcher().getTopReaderContext().leaves();
-    SortDoc sortDoc = getSortDoc(req.getSearcher(), sort.getSort());
-    int count = 0;
-    int queueSize = 30000;
-    SortQueue queue = new SortQueue(queueSize, sortDoc);
-    SortDoc[] outDocs = new SortDoc[queueSize];
-
-    boolean commaNeeded = false;
-    while(count < totalHits) {
-      //long begin = System.nanoTime();
-      queue.reset();
-      SortDoc top = queue.top();
-      for(int i=0; i<leaves.size(); i++) {
-        sortDoc.setNextReader(leaves.get(i));
-        DocIdSetIterator it = new BitSetIterator(sets[i], 0); // cost is not useful here
-        int docId = -1;
-        while((docId = it.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
-          sortDoc.setValues(docId);
-          if(top.lessThan(sortDoc)) {
-            top.setValues(sortDoc);
-            top = queue.updateTop();
-          }
-        }
-      }
-
-      int outDocsIndex = -1;
-
-      for(int i=0; i<queueSize; i++) {
-        SortDoc s = queue.pop();
-        if(s.docId > -1) {
-          outDocs[++outDocsIndex] = s;
-        }
-      }
-
-     //long end = System.nanoTime();
-
-      count += (outDocsIndex+1);
-
-      try {
-        for(int i=outDocsIndex; i>=0; --i) {
-          SortDoc s = outDocs[i];
-          if(commaNeeded){writer.write(',');}
-          writer.write('{');
-          writeDoc(s, leaves, fieldWriters, sets, writer);
-          writer.write('}');
-          commaNeeded = true;
-          s.reset();
-        }
-      } catch(Throwable e) {
-        Throwable ex = e;
-        e.printStackTrace();
-        while(ex != null) {
-          String m = ex.getMessage();
-          if(m != null && m.contains("Broken pipe")) {
-            throw new IgnoreException();
-          }
-          ex = ex.getCause();
-        }
-
-        if(e instanceof IOException) {
-          throw ((IOException)e);
-        } else {
-          throw new IOException(e);
-        }
-      }
-    }
-
-    //System.out.println("Sort Time 2:"+Long.toString(total/1000000));
-    writer.write("]}}");
-    writer.flush();
-  }
-
-  public static class IgnoreException extends IOException {
-    public void printStackTrace(PrintWriter pw) {
-      pw.print("Early Client Disconnect");
-
-    }
-
-    public String getMessage() {
-      return "Early Client Disconnect";
-    }
-  }
-
-
-  protected void writeDoc(SortDoc sortDoc,
-                          List<LeafReaderContext> leaves,
-                          FieldWriter[] fieldWriters,
-                          FixedBitSet[] sets,
-                          Writer out) throws IOException{
-
-    int ord = sortDoc.ord;
-    FixedBitSet set = sets[ord];
-    set.clear(sortDoc.docId);
-    LeafReaderContext context = leaves.get(ord);
-    int fieldIndex = 0;
-    for(FieldWriter fieldWriter : fieldWriters) {
-      if(fieldWriter.write(sortDoc.docId, context.reader(), out, fieldIndex)){
-        ++fieldIndex;
-      }
-    }
-  }
-
-  protected void writeException(Exception e, Writer out, boolean log) throws IOException{
-    out.write("{\"responseHeader\": {\"status\": 400}, \"response\":{\"numFound\":0, \"docs\":[");
-    out.write("{\"EXCEPTION\":\"");
-    writeStr(e.getMessage(), out);
-    out.write("\"}");
-    out.write("]}}");
-    out.flush();
-    if(log) {
-      SolrException.log(logger, e);
-    }
-  }
-
-  protected FieldWriter[] getFieldWriters(String[] fields, SolrIndexSearcher searcher) throws IOException {
-    IndexSchema schema = searcher.getSchema();
-    FieldWriter[] writers = new FieldWriter[fields.length];
-    for(int i=0; i<fields.length; i++) {
-      String field = fields[i];
-      SchemaField schemaField = null;
-
-      try {
-        schemaField = schema.getField(field);
-      } catch (Exception e) {
-        throw new IOException(e);
-      }
-
-      if(!schemaField.hasDocValues()) {
-        throw new IOException(field+" must have DocValues to use this feature.");
-      }
-
-      boolean multiValued = schemaField.multiValued();
-      FieldType fieldType = schemaField.getType();
-      if(fieldType instanceof TrieIntField) {
-        if(multiValued) {
-          writers[i] = new MultiFieldWriter(field, fieldType, true);
-        } else {
-          writers[i] = new IntFieldWriter(field);
-        }
-      } else if (fieldType instanceof TrieLongField) {
-        if(multiValued) {
-          writers[i] = new MultiFieldWriter(field, fieldType, true);
-        } else {
-          writers[i] = new LongFieldWriter(field);
-        }
-      } else if (fieldType instanceof TrieFloatField) {
-        if(multiValued) {
-          writers[i] = new MultiFieldWriter(field, fieldType, true);
-        } else {
-          writers[i] = new FloatFieldWriter(field);
-        }
-      } else if(fieldType instanceof TrieDoubleField) {
-        if(multiValued) {
-          writers[i] = new MultiFieldWriter(field, fieldType, true);
-        } else {
-          writers[i] = new DoubleFieldWriter(field);
-        }
-      } else if(fieldType instanceof StrField) {
-        if(multiValued) {
-          writers[i] = new MultiFieldWriter(field, fieldType, false);
-        } else {
-          writers[i] = new StringFieldWriter(field, fieldType);
-        }
-      } else if (fieldType instanceof TrieDateField) {
-        if (multiValued) {
-          writers[i] = new MultiFieldWriter(field, fieldType, false);
-        } else {
-          writers[i] = new DateFieldWriter(field);
-        }
-      } else if(fieldType instanceof BoolField) {
-        if(multiValued) {
-          writers[i] = new MultiFieldWriter(field, fieldType, true);
-        } else {
-          writers[i] = new BoolFieldWriter(field, fieldType);
-        }
-      }
-      else {
-        throw new IOException("Export fields must either be one of the following types: int,float,long,double,string,date,boolean");
-      }
-    }
-    return writers;
-  }
-
-  private SortDoc getSortDoc(SolrIndexSearcher searcher, SortField[] sortFields) throws IOException {
-    SortValue[] sortValues = new SortValue[sortFields.length];
-    IndexSchema schema = searcher.getSchema();
-    for(int i=0; i<sortFields.length; ++i) {
-      SortField sf = sortFields[i];
-      String field = sf.getField();
-      boolean reverse = sf.getReverse();
-      SchemaField schemaField = schema.getField(field);
-      FieldType ft = schemaField.getType();
-
-      if(!schemaField.hasDocValues()) {
-        throw new IOException(field+" must have DocValues to use this feature.");
-      }
-
-      if(ft instanceof TrieIntField) {
-        if(reverse) {
-          sortValues[i] = new IntValue(field, new IntDesc());
-        } else {
-          sortValues[i] = new IntValue(field, new IntAsc());
-        }
-      } else if(ft instanceof TrieFloatField) {
-        if(reverse) {
-          sortValues[i] = new FloatValue(field, new FloatDesc());
-        } else {
-          sortValues[i] = new FloatValue(field, new FloatAsc());
-        }
-      } else if(ft instanceof TrieDoubleField) {
-        if(reverse) {
-          sortValues[i] = new DoubleValue(field, new DoubleDesc());
-        } else {
-          sortValues[i] = new DoubleValue(field, new DoubleAsc());
-        }
-      } else if(ft instanceof TrieLongField) {
-        if(reverse) {
-          sortValues[i] = new LongValue(field, new LongDesc());
-        } else {
-          sortValues[i] = new LongValue(field, new LongAsc());
-        }
-      } else if(ft instanceof StrField) {
-        LeafReader reader = searcher.getSlowAtomicReader();
-        SortedDocValues vals =  reader.getSortedDocValues(field);
-        if(reverse) {
-          sortValues[i] = new StringValue(vals, field, new IntDesc());
-        } else {
-          sortValues[i] = new StringValue(vals, field, new IntAsc());
-        }
-      } else if (ft instanceof TrieDateField) {
-        if (reverse) {
-          sortValues[i] = new LongValue(field, new LongDesc());
-        } else {
-          sortValues[i] = new LongValue(field, new LongAsc());
-        }
-      } else if (ft instanceof BoolField) {
-        // This is a bit of a hack, but since the boolean field stores ByteRefs, just like Strings
-        // _and_ since "F" happens to sort before "T" (thus false sorts "less" than true)
-        // we can just use the existing StringValue here.
-        LeafReader reader = searcher.getSlowAtomicReader();
-        SortedDocValues vals =  reader.getSortedDocValues(field);
-        if(reverse) {
-          sortValues[i] = new StringValue(vals, field, new IntDesc());
-        } else {
-          sortValues[i] = new StringValue(vals, field, new IntAsc());
-        }
-      } else {
-        throw new IOException("Sort fields must be one of the following types: int,float,long,double,string,date,boolean");
-      }
-    }
-
-    if(sortValues.length == 1) {
-      return new SingleValueSortDoc(sortValues[0]);
-    } else if(sortValues.length == 2) {
-      return new DoubleValueSortDoc(sortValues[0], sortValues[1]);
-    } else if(sortValues.length == 3) {
-      return new TripleValueSortDoc(sortValues[0], sortValues[1], sortValues[2]);
-    } else if(sortValues.length == 4) {
-      return new QuadValueSortDoc(sortValues[0], sortValues[1], sortValues[2], sortValues[3]);
-    } else {
-      throw new IOException("A max of 4 sorts can be specified");
-    }
-  }
-
-  class SortQueue extends PriorityQueue<SortDoc> {
-
-    private SortDoc proto;
-    private Object[] cache;
-
-    public SortQueue(int len, SortDoc proto) {
-      super(len);
-      this.proto = proto;
-    }
-
-    protected boolean lessThan(SortDoc t1, SortDoc t2) {
-      return t1.lessThan(t2);
-    }
-
-    private void populate() {
-      Object[] heap = getHeapArray();
-      cache = new SortDoc[heap.length];
-      for(int i=1; i<heap.length; i++) {
-        cache[i] = heap[i]  = proto.copy();
-      }
-      size = maxSize;
-    }
-
-    private void reset() {
-      Object[] heap = getHeapArray();
-      if(cache != null) {
-        System.arraycopy(cache, 1, heap, 1, heap.length-1);
-        size = maxSize;
-      } else {
-        populate();
-      }
-    }
-  }
-
-  class SortDoc {
-
-    protected int docId = -1;
-    protected int ord = -1;
-    protected int docBase = -1;
-
-    private SortValue[] sortValues;
-
-    public SortDoc() {
-
-    }
-
-    public void setNextReader(LeafReaderContext context) throws IOException {
-      this.ord = context.ord;
-      for(SortValue value : sortValues) {
-        value.setNextReader(context);
-      }
-    }
-
-    public void reset() {
-      this.docId = -1;
-    }
-
-    public void setValues(int docId) throws IOException {
-      this.docId = docId;
-      for(SortValue sortValue : sortValues) {
-        sortValue.setCurrentValue(docId);
-      }
-    }
-
-    public void setValues(SortDoc sortDoc) throws IOException {
-      this.docId = sortDoc.docId;
-      this.ord = sortDoc.ord;
-      SortValue[] vals = sortDoc.sortValues;
-      for(int i=0; i<vals.length; i++) {
-        sortValues[i].setCurrentValue(vals[i]);
-      }
-    }
-
-    public SortDoc(SortValue[] sortValues) {
-      this.sortValues = sortValues;
-    }
-
-    public SortDoc copy() {
-      SortValue[] svs = new SortValue[sortValues.length];
-      for(int i=0; i<sortValues.length; i++) {
-        svs[i] = sortValues[i].copy();
-      }
-
-      return new SortDoc(svs);
-    }
-
-    public boolean lessThan(Object o) {
-      if(docId == -1) {
-        return true;
-      }
-
-      SortDoc sd = (SortDoc)o;
-      SortValue[] sortValues1 = sd.sortValues;
-      for(int i=0; i<sortValues.length; i++) {
-        int comp = sortValues[i].compareTo(sortValues1[i]);
-        if(comp < 0) {
-          return true;
-        } if(comp > 0) {
-          return false;
-        }
-      }
-      return docId+docBase < sd.docId+sd.docBase;
-    }
-
-    public String toString() {
-      return "";
-    }
-  }
-
-  class SingleValueSortDoc extends SortDoc {
-
-    protected SortValue value1;
-
-    public void setNextReader(LeafReaderContext context) throws IOException {
-      this.ord = context.ord;
-      value1.setNextReader(context);
-    }
-
-    public void reset() {
-      this.docId = -1;
-      this.value1.reset();
-    }
-
-    public void setValues(int docId) throws IOException {
-      this.docId = docId;
-      value1.setCurrentValue(docId);
-    }
-
-    public void setValues(SortDoc sortDoc) throws IOException {
-      this.docId = sortDoc.docId;
-      this.ord = sortDoc.ord;
-      value1.setCurrentValue(((SingleValueSortDoc)sortDoc).value1);
-    }
-
-    public SingleValueSortDoc(SortValue value1) {
-      super();
-      this.value1 = value1;
-    }
-
-    public SortDoc copy() {
-      return new SingleValueSortDoc(value1.copy());
-    }
-
-    public boolean lessThan(Object o) {
-      SingleValueSortDoc sd = (SingleValueSortDoc)o;
-      int comp = value1.compareTo(sd.value1);
-      if(comp == -1) {
-        return true;
-      } else if (comp == 1) {
-        return false;
-      } else {
-        return docId+docBase > sd.docId+sd.docBase;
-      }
-    }
-
-    public int compareTo(Object o) {
-      SingleValueSortDoc sd = (SingleValueSortDoc)o;
-      return value1.compareTo(sd.value1);
-    }
-
-    public String toString() {
-      return docId+":"+value1.toString();
-    }
-  }
-
-  class DoubleValueSortDoc extends SingleValueSortDoc {
-
-    protected SortValue value2;
-
-    public void setNextReader(LeafReaderContext context) throws IOException {
-      this.ord = context.ord;
-      value1.setNextReader(context);
-      value2.setNextReader(context);
-    }
-
-    public void reset() {
-      this.docId = -1;
-      value1.reset();
-      value2.reset();
-    }
-
-    public void setValues(int docId) throws IOException {
-      this.docId = docId;
-      value1.setCurrentValue(docId);
-      value2.setCurrentValue(docId);
-    }
-
-    public void setValues(SortDoc sortDoc) throws IOException {
-      this.docId = sortDoc.docId;
-      this.ord = sortDoc.ord;
-      value1.setCurrentValue(((DoubleValueSortDoc)sortDoc).value1);
-      value2.setCurrentValue(((DoubleValueSortDoc)sortDoc).value2);
-    }
-
-    public DoubleValueSortDoc(SortValue value1, SortValue value2) {
-      super(value1);
-      this.value2 = value2;
-    }
-
-    public SortDoc copy() {
-      return new DoubleValueSortDoc(value1.copy(), value2.copy());
-    }
-
-    public boolean lessThan(Object o) {
-      DoubleValueSortDoc sd = (DoubleValueSortDoc)o;
-      int comp = value1.compareTo(sd.value1);
-      if(comp == -1) {
-        return true;
-      } else if (comp == 1) {
-        return false;
-      } else {
-        comp = value2.compareTo(sd.value2);
-        if(comp == -1) {
-          return true;
-        } else if (comp == 1) {
-          return false;
-        } else {
-          return docId+docBase > sd.docId+sd.docBase;
-        }
-      }
-    }
-
-    public int compareTo(Object o) {
-      DoubleValueSortDoc sd = (DoubleValueSortDoc)o;
-      int comp = value1.compareTo(sd.value1);
-      if(comp == 0) {
-        return value2.compareTo(sd.value2);
-      } else {
-        return comp;
-      }
-    }
-  }
-
-  class TripleValueSortDoc extends DoubleValueSortDoc {
-
-    protected SortValue value3;
-
-    public void setNextReader(LeafReaderContext context) throws IOException {
-      this.ord = context.ord;
-      value1.setNextReader(context);
-      value2.setNextReader(context);
-      value3.setNextReader(context);
-    }
-
-    public void reset() {
-      this.docId = -1;
-      value1.reset();
-      value2.reset();
-      value3.reset();
-    }
-
-    public void setValues(int docId) throws IOException {
-      this.docId = docId;
-      value1.setCurrentValue(docId);
-      value2.setCurrentValue(docId);
-      value3.setCurrentValue(docId);
-    }
-
-    public void setValues(SortDoc sortDoc) throws IOException {
-      this.docId = sortDoc.docId;
-      this.ord = sortDoc.ord;
-      value1.setCurrentValue(((TripleValueSortDoc)sortDoc).value1);
-      value2.setCurrentValue(((TripleValueSortDoc)sortDoc).value2);
-      value3.setCurrentValue(((TripleValueSortDoc)sortDoc).value3);
-    }
-
-    public TripleValueSortDoc(SortValue value1, SortValue value2, SortValue value3) {
-      super(value1, value2);
-      this.value3 = value3;
-    }
-
-    public SortDoc copy() {
-      return new TripleValueSortDoc(value1.copy(), value2.copy(), value3.copy());
-    }
-
-    public boolean lessThan(Object o) {
-
-      TripleValueSortDoc sd = (TripleValueSortDoc)o;
-      int comp = value1.compareTo(sd.value1);
-      if(comp == -1) {
-        return true;
-      } else if (comp == 1) {
-        return false;
-      } else {
-        comp = value2.compareTo(sd.value2);
-        if(comp == -1) {
-          return true;
-        } else if (comp == 1) {
-          return false;
-        } else {
-          comp = value3.compareTo(sd.value3);
-          if(comp == -1) {
-            return true;
-          } else if (comp == 1) {
-            return false;
-          } else {
-            return docId+docBase > sd.docId+sd.docBase;
-          }
-        }
-      }
-    }
-
-    public int compareTo(Object o) {
-
-      TripleValueSortDoc sd = (TripleValueSortDoc)o;
-      int comp = value1.compareTo(sd.value1);
-      if(comp == 0) {
-        comp = value2.compareTo(sd.value2);
-        if(comp == 0) {
-          return value3.compareTo(sd.value3);
-        } else {
-          return comp;
-        }
-      } else {
-        return comp;
-      }
-    }
-  }
-
-  class QuadValueSortDoc extends TripleValueSortDoc {
-
-    protected SortValue value4;
-
-    public void setNextReader(LeafReaderContext context) throws IOException {
-      this.ord = context.ord;
-      value1.setNextReader(context);
-      value2.setNextReader(context);
-      value3.setNextReader(context);
-      value4.setNextReader(context);
-    }
-
-    public void reset() {
-      this.docId = -1;
-      value1.reset();
-      value2.reset();
-      value3.reset();
-      value4.reset();
-    }
-
-    public void setValues(int docId) throws IOException {
-      this.docId = docId;
-      value1.setCurrentValue(docId);
-      value2.setCurrentValue(docId);
-      value3.setCurrentValue(docId);
-      value4.setCurrentValue(docId);
-    }
-
-    public void setValues(SortDoc sortDoc) throws IOException {
-      this.docId = sortDoc.docId;
-      this.ord = sortDoc.ord;
-      value1.setCurrentValue(((QuadValueSortDoc)sortDoc).value1);
-      value2.setCurrentValue(((QuadValueSortDoc)sortDoc).value2);
-      value3.setCurrentValue(((QuadValueSortDoc)sortDoc).value3);
-      value4.setCurrentValue(((QuadValueSortDoc)sortDoc).value4);
-    }
-
-    public QuadValueSortDoc(SortValue value1, SortValue value2, SortValue value3, SortValue value4) {
-      super(value1, value2, value3);
-      this.value4 = value4;
-    }
-
-    public SortDoc copy() {
-      return new QuadValueSortDoc(value1.copy(), value2.copy(), value3.copy(), value4.copy());
-    }
-
-    public boolean lessThan(Object o) {
-
-      QuadValueSortDoc sd = (QuadValueSortDoc)o;
-      int comp = value1.compareTo(sd.value1);
-      if(comp == -1) {
-        return true;
-      } else if (comp == 1) {
-        return false;
-      } else {
-        comp = value2.compareTo(sd.value2);
-        if(comp == -1) {
-          return true;
-        } else if (comp == 1) {
-          return false;
-        } else {
-          comp = value3.compareTo(sd.value3);
-          if(comp == -1) {
-            return true;
-          } else if (comp == 1) {
-            return false;
-          } else {
-            comp = value4.compareTo(sd.value4);
-            if(comp == -1) {
-              return true;
-            } else if (comp == 1) {
-              return false;
-            } else {
-              return docId+docBase > sd.docId+sd.docBase;
-            }
-          }
-        }
-      }
-    }
-
-    public int compareTo(Object o) {
-      QuadValueSortDoc sd = (QuadValueSortDoc)o;
-      int comp = value1.compareTo(sd.value1);
-      if(comp == 0) {
-        comp = value2.compareTo(sd.value2);
-        if(comp == 0) {
-          comp = value3.compareTo(sd.value3);
-          if(comp == 0) {
-            return value4.compareTo(sd.value4);
-          } else {
-            return comp;
-          }
-        } else {
-          return comp;
-        }
-      } else {
-        return comp;
-      }
-    }
-  }
-
-  public interface SortValue extends Comparable<SortValue> {
-    public void setCurrentValue(int docId) throws IOException;
-    public void setNextReader(LeafReaderContext context) throws IOException;
-    public void setCurrentValue(SortValue value);
-    public void reset();
-    public SortValue copy();
-  }
-
-  class IntValue implements SortValue {
-
-    protected NumericDocValues vals;
-    protected String field;
-    protected int currentValue;
-    protected IntComp comp;
-
-    public IntValue copy() {
-      return new IntValue(field, comp);
-    }
-
-    public IntValue(String field, IntComp comp) {
-      this.field = field;
-      this.comp = comp;
-      this.currentValue = comp.resetValue();
-    }
-
-    public void setNextReader(LeafReaderContext context) throws IOException {
-      this.vals = DocValues.getNumeric(context.reader(), field);
-    }
-
-    public void setCurrentValue(int docId) {
-      currentValue = (int)vals.get(docId);
-    }
-
-    public int compareTo(SortValue o) {
-      IntValue iv = (IntValue)o;
-      return comp.compare(currentValue, iv.currentValue);
-    }
-
-    public void setCurrentValue (SortValue value) {
-      currentValue = ((IntValue)value).currentValue;
-    }
-
-    public void reset() {
-      currentValue = comp.resetValue();
-    }
-  }
-
-  public interface IntComp {
-    public int compare(int a, int b);
-    public int resetValue();
-  }
-
-  class IntDesc implements IntComp {
-
-    public int resetValue() {
-      return Integer.MIN_VALUE;
-    }
-
-    public int compare(int a, int b) {
-      if(a < b) {
-        return -1;
-      } else if (a > b) {
-        return 1;
-      } else {
-        return 0;
-      }
-    }
-  }
-
-  class IntAsc implements IntComp {
-
-    public int resetValue() {
-      return Integer.MAX_VALUE;
-    }
-
-    public int compare(int a, int b) {
-      if(a < b) {
-        return 1;
-      } else if (a > b) {
-        return -1;
-      } else {
-        return 0;
-      }
-    }
-  }
-
-  class LongValue implements SortValue {
-
-    protected NumericDocValues vals;
-    protected String field;
-    protected long currentValue;
-    protected LongComp comp;
-
-    public LongValue(String field, LongComp comp) {
-      this.field = field;
-      this.comp = comp;
-      this.currentValue = comp.resetValue();
-    }
-
-    public LongValue copy() {
-      return new LongValue(field, comp);
-    }
-
-    public void setNextReader(LeafReaderContext context) throws IOException {
-      this.vals = DocValues.getNumeric(context.reader(), field);
-    }
-
-    public void setCurrentValue(int docId) {
-      currentValue = vals.get(docId);
-    }
-
-    public void setCurrentValue(SortValue sv) {
-      LongValue lv = (LongValue)sv;
-      this.currentValue = lv.currentValue;
-    }
-
-    public int compareTo(SortValue o) {
-      LongValue l = (LongValue)o;
-      return comp.compare(currentValue, l.currentValue);
-    }
-
-    public void reset() {
-      this.currentValue = comp.resetValue();
-    }
-  }
-
-  interface LongComp {
-    public int compare(long a, long b);
-    public long resetValue();
-  }
-
-  class LongDesc implements LongComp {
-
-    public long resetValue() {
-      return Long.MIN_VALUE;
-    }
-
-    public int compare(long a, long b) {
-      if(a < b) {
-        return -1;
-      } else if (a > b) {
-        return 1;
-      } else {
-        return 0;
-      }
-    }
-  }
-
-  class LongAsc implements LongComp {
-
-    public long resetValue() {
-      return Long.MAX_VALUE;
-    }
-
-    public int compare(long a, long b) {
-      if(a < b) {
-        return 1;
-      } else if (a > b) {
-        return -1;
-      } else {
-        return 0;
-      }
-    }
-  }
-
-  class FloatValue implements SortValue {
-
-    protected NumericDocValues vals;
-    protected String field;
-    protected float currentValue;
-    protected FloatComp comp;
-
-    public FloatValue(String field, FloatComp comp) {
-      this.field = field;
-      this.comp = comp;
-      this.currentValue = comp.resetValue();
-    }
-
-    public FloatValue copy() {
-      return new FloatValue(field, comp);
-    }
-
-    public void setNextReader(LeafReaderContext context) throws IOException {
-      this.vals = DocValues.getNumeric(context.reader(), field);
-    }
-
-    public void setCurrentValue(int docId) {
-      currentValue = Float.intBitsToFloat((int)vals.get(docId));
-    }
-
-    public void setCurrentValue(SortValue sv) {
-      FloatValue fv = (FloatValue)sv;
-      this.currentValue = fv.currentValue;
-    }
-
-    public void reset() {
-      this.currentValue = comp.resetValue();
-    }
-
-    public int compareTo(SortValue o) {
-      FloatValue fv = (FloatValue)o;
-      return comp.compare(currentValue, fv.currentValue);
-    }
-  }
-
-  interface FloatComp {
-    public int compare(float a, float b);
-    public float resetValue();
-  }
-
-  public class FloatDesc implements FloatComp {
-    public float resetValue() {
-      return -Float.MAX_VALUE;
-    }
-
-    public int compare(float a, float b) {
-      if(a < b) {
-        return -1;
-      } else if (a > b) {
-        return 1;
-      } else {
-        return 0;
-      }
-    }
-  }
-
-  public class FloatAsc implements FloatComp {
-    public float resetValue() {
-      return Float.MAX_VALUE;
-    }
-
-    public int compare(float a, float b) {
-      if(a < b) {
-        return 1;
-      } else if (a > b) {
-        return -1;
-      } else {
-        return 0;
-      }
-    }
-  }
-
-  class DoubleValue implements SortValue {
-
-    protected NumericDocValues vals;
-    protected String field;
-    protected double currentValue;
-    protected DoubleComp comp;
-
-    public DoubleValue(String field, DoubleComp comp) {
-      this.field = field;
-      this.comp = comp;
-      this.currentValue = comp.resetValue();
-    }
-
-    public DoubleValue copy() {
-      return new DoubleValue(field, comp);
-    }
-
-    public void setNextReader(LeafReaderContext context) throws IOException {
-      this.vals = DocValues.getNumeric(context.reader(), field);
-    }
-
-    public void setCurrentValue(int docId) {
-      currentValue = Double.longBitsToDouble(vals.get(docId));
-    }
-
-    public void setCurrentValue(SortValue sv) {
-      DoubleValue dv = (DoubleValue)sv;
-      this.currentValue = dv.currentValue;
-    }
-
-    public void reset() {
-      this.currentValue = comp.resetValue();
-    }
-
-    public int compareTo(SortValue o) {
-      DoubleValue dv = (DoubleValue)o;
-      return comp.compare(currentValue, dv.currentValue);
-    }
-  }
-
-  interface DoubleComp {
-    public int compare(double a, double b);
-    public double resetValue();
-  }
-
-  public class DoubleDesc implements DoubleComp {
-    public double resetValue() {
-      return -Double.MAX_VALUE;
-    }
-
-    public int compare(double a, double b) {
-      if(a < b) {
-        return -1;
-      } else if (a > b) {
-        return 1;
-      } else {
-        return 0;
-      }
-    }
-  }
-
-  public class DoubleAsc implements DoubleComp {
-    public double resetValue() {
-      return Double.MAX_VALUE;
-    }
-
-    public int compare(double a, double b) {
-      if(a < b) {
-        return 1;
-      } else if (a > b) {
-        return -1;
-      } else {
-        return 0;
-      }
-    }
-  }
-
-  class StringValue implements SortValue {
-
-    protected SortedDocValues vals;
-    protected SortedDocValues segmentVals[];
-
-    protected MultiDocValues.OrdinalMap ordinalMap;
-    protected LongValues globalOrds;
-    protected SortedDocValues currentVals;
-
-    protected String field;
-    protected int segment;
-    protected int currentOrd;
-    protected IntComp comp;
-
-    public StringValue(SortedDocValues vals, String field, IntComp comp)  {
-      this.vals = vals;
-      if(vals instanceof  MultiDocValues.MultiSortedDocValues) {
-        this.segmentVals = ((MultiDocValues.MultiSortedDocValues) vals).values;
-        this.ordinalMap = ((MultiDocValues.MultiSortedDocValues) vals).mapping;
-      }
-      this.field = field;
-      this.comp = comp;
-      this.currentOrd = comp.resetValue();
-    }
-
-    public StringValue copy() {
-      return new StringValue(vals, field, comp);
-    }
-
-    public void setCurrentValue(int docId) {
-      int ord = currentVals.getOrd(docId);
-
-      if(ord < 0) {
-        currentOrd = -1;
-      } else {
-        if(globalOrds != null) {
-          currentOrd = (int)globalOrds.get(ord);
-        } else {
-          currentOrd = ord;
-        }
-      }
-    }
-
-    public void setCurrentValue(SortValue sv) {
-      StringValue v = (StringValue)sv;
-      this.currentOrd = v.currentOrd;
-    }
-
-    public void setNextReader(LeafReaderContext context) {
-      segment = context.ord;
-      if(ordinalMap != null) {
-        globalOrds = ordinalMap.getGlobalOrds(segment);
-        currentVals = segmentVals[segment];
-      } else {
-        currentVals = vals;
-      }
-    }
-
-    public void reset() {
-      this.currentOrd = comp.resetValue();
-    }
-
-    public int compareTo(SortValue o) {
-      StringValue sv = (StringValue)o;
-      return comp.compare(currentOrd, sv.currentOrd);
-    }
-
-    public String toString() {
-      return Integer.toString(this.currentOrd);
-    }
-  }
-
-  protected abstract class FieldWriter {
-    public abstract boolean write(int docId, LeafReader reader, Writer out, int fieldIndex) throws IOException;
-  }
-
-  class IntFieldWriter extends FieldWriter {
-    private String field;
-
-    public IntFieldWriter(String field) {
-      this.field = field;
-    }
-
-    public boolean write(int docId, LeafReader reader, Writer out, int fieldIndex) throws IOException {
-      NumericDocValues vals = DocValues.getNumeric(reader, this.field);
-      int val = (int)vals.get(docId);
-      if(fieldIndex>0) {
-        out.write(',');
-      }
-      out.write('"');
-      out.write(this.field);
-      out.write('"');
-      out.write(':');
-      out.write(Integer.toString(val));
-      return true;
-    }
-  }
-
-  class MultiFieldWriter extends FieldWriter {
-    private String field;
-    private FieldType fieldType;
-    private boolean numeric;
-    private CharsRefBuilder cref = new CharsRefBuilder();
-
-    public MultiFieldWriter(String field, FieldType fieldType, boolean numeric) {
-      this.field = field;
-      this.fieldType = fieldType;
-      this.numeric = numeric;
-    }
-    public boolean write(int docId, LeafReader reader, Writer out, int fieldIndex) throws IOException {
-      SortedSetDocValues vals = DocValues.getSortedSet(reader, this.field);
-      vals.setDocument(docId);
-      List<Long> ords = new ArrayList();
-      long o = -1;
-      while((o = vals.nextOrd()) != SortedSetDocValues.NO_MORE_ORDS) {
-        ords.add(o);
-      }
-
-      if(ords.size()== 0) {
-        return false;
-      }
-
-
-      if(fieldIndex>0) {
-        out.write(',');
-      }
-      out.write('"');
-      out.write(this.field);
-      out.write('"');
-      out.write(':');
-      out.write('[');
-      int v = 0;
-      for(long ord : ords) {
-        BytesRef ref = vals.lookupOrd(ord);
-        fieldType.indexedToReadable(ref, cref);
-        if(v > 0) {
-          out.write(',');
-        }
-
-        if(!numeric) {
-          out.write('"');
-        }
-
-        writeStr(cref.toString(), out);
-
-        if(!numeric) {
-          out.write('"');
-        }
-        ++v;
-      }
-      out.write("]");
-      return true;
-    }
-  }
-
-  class LongFieldWriter extends FieldWriter {
-    private String field;
-
-    public LongFieldWriter(String field) {
-      this.field = field;
-    }
-
-    public boolean write(int docId, LeafReader reader, Writer out, int fieldIndex) throws IOException {
-      NumericDocValues vals = DocValues.getNumeric(reader, this.field);
-      long val = vals.get(docId);
-      if(fieldIndex > 0) {
-        out.write(',');
-      }
-      out.write('"');
-      out.write(this.field);
-      out.write('"');
-      out.write(':');
-      out.write(Long.toString(val));
-      return true;
-    }
-  }
-
-  class DateFieldWriter extends FieldWriter {
-    private String field;
-
-    public DateFieldWriter(String field) {
-      this.field = field;
-    }
-
-    public boolean write(int docId, LeafReader reader, Writer out, int fieldIndex) throws IOException {
-      NumericDocValues vals = DocValues.getNumeric(reader, this.field);
-      long val = vals.get(docId);
-
-      if (fieldIndex > 0) {
-        out.write(',');
-      }
-      out.write('"');
-      out.write(this.field);
-      out.write('"');
-      out.write(':');
-      out.write('"');
-      writeStr(new Date(val).toInstant().toString(), out);
-      out.write('"');
-      return true;
-    }
-  }
-
-  class BoolFieldWriter extends FieldWriter {
-    private String field;
-    private FieldType fieldType;
-    private CharsRefBuilder cref = new CharsRefBuilder();
-
-    public BoolFieldWriter(String field, FieldType fieldType) {
-      this.field = field;
-      this.fieldType = fieldType;
-    }
-
-    public boolean write(int docId, LeafReader reader, Writer out, int fieldIndex) throws IOException {
-      SortedDocValues vals = DocValues.getSorted(reader, this.field);
-      int ord = vals.getOrd(docId);
-      if(ord == -1) {
-        return false;
-      }
-
-      BytesRef ref = vals.lookupOrd(ord);
-      fieldType.indexedToReadable(ref, cref);
-
-      if (fieldIndex > 0) {
-        out.write(',');
-      }
-      out.write('"');
-      out.write(this.field);
-      out.write('"');
-      out.write(':');
-      //out.write('"');
-      writeStr(cref.toString(), out);
-      //out.write('"');
-      return true;
-    }
-  }
-
-  class FloatFieldWriter extends FieldWriter {
-    private String field;
-
-    public FloatFieldWriter(String field) {
-      this.field = field;
-    }
-
-    public boolean write(int docId, LeafReader reader, Writer out, int fieldIndex) throws IOException {
-      NumericDocValues vals = DocValues.getNumeric(reader, this.field);
-      int val = (int)vals.get(docId);
-      if(fieldIndex > 0) {
-        out.write(',');
-      }
-      out.write('"');
-      out.write(this.field);
-      out.write('"');
-      out.write(':');
-      out.write(Float.toString(Float.intBitsToFloat(val)));
-      return true;
-    }
-  }
-
-  class DoubleFieldWriter extends FieldWriter {
-    private String field;
-
-    public DoubleFieldWriter(String field) {
-      this.field = field;
-    }
-
-    public boolean write(int docId, LeafReader reader, Writer out, int fieldIndex) throws IOException {
-      NumericDocValues vals = DocValues.getNumeric(reader, this.field);
-      if(fieldIndex > 0) {
-        out.write(',');
-      }
-      long val = vals.get(docId);
-      out.write('"');
-      out.write(this.field);
-      out.write('"');
-      out.write(':');
-      out.write(Double.toString(Double.longBitsToDouble(val)));
-      return true;
-    }
-  }
-
-  class StringFieldWriter extends FieldWriter {
-    private String field;
-    private FieldType fieldType;
-    private CharsRefBuilder cref = new CharsRefBuilder();
-
-    public StringFieldWriter(String field, FieldType fieldType) {
-      this.field = field;
-      this.fieldType = fieldType;
-    }
-
-    public boolean write(int docId, LeafReader reader, Writer out, int fieldIndex) throws IOException {
-      SortedDocValues vals = DocValues.getSorted(reader, this.field);
-      int ord = vals.getOrd(docId);
-      if(ord == -1) {
-        return false;
-      }
-
-      BytesRef ref = vals.lookupOrd(ord);
-      fieldType.indexedToReadable(ref, cref);
-      if(fieldIndex > 0) {
-        out.write(',');
-      }
-      out.write('"');
-      out.write(this.field);
-      out.write('"');
-      out.write(":");
-      out.write('"');
-      writeStr(cref.toString(), out);
-      out.write('"');
-      return true;
-    }
-  }
-
-  private void writeStr(String val, Writer writer) throws IOException {
-    for (int i=0; i<val.length(); i++) {
-      char ch = val.charAt(i);
-      if ((ch > '#' && ch != '\\' && ch < '\u2028') || ch == ' ') { // fast path
-        writer.write(ch);
-        continue;
-      }
-      switch(ch) {
-        case '"':
-        case '\\':
-          writer.write('\\');
-          writer.write(ch);
-          break;
-        case '\r': writer.write('\\'); writer.write('r'); break;
-        case '\n': writer.write('\\'); writer.write('n'); break;
-        case '\t': writer.write('\\'); writer.write('t'); break;
-        case '\b': writer.write('\\'); writer.write('b'); break;
-        case '\f': writer.write('\\'); writer.write('f'); break;
-        case '\u2028': // fallthrough
-        case '\u2029':
-          unicodeEscape(writer,ch);
-          break;
-        // case '/':
-        default: {
-          if (ch <= 0x1F) {
-            unicodeEscape(writer,ch);
-          } else {
-            writer.write(ch);
-          }
-        }
-      }
-    }
-  }
-
-  private static char[] hexdigits = {'0','1','2','3','4','5','6','7','8','9','a','b','c','d','e','f'};
-  protected static void unicodeEscape(Appendable out, int ch) throws IOException {
-    out.append('\\');
-    out.append('u');
-    out.append(hexdigits[(ch>>>12)     ]);
-    out.append(hexdigits[(ch>>>8) & 0xf]);
-    out.append(hexdigits[(ch>>>4) & 0xf]);
-    out.append(hexdigits[(ch)     & 0xf]);
-  }
-
-  public abstract class PriorityQueue<T> {
-    protected int size = 0;
-    protected final int maxSize;
-    private final T[] heap;
-
-    public PriorityQueue(int maxSize) {
-      this(maxSize, true);
-    }
-
-    public PriorityQueue(int maxSize, boolean prepopulate) {
-      final int heapSize;
-      if (0 == maxSize) {
-        // We allocate 1 extra to avoid if statement in top()
-        heapSize = 2;
-      } else {
-        if (maxSize > ArrayUtil.MAX_ARRAY_LENGTH) {
-          // Don't wrap heapSize to -1, in this case, which
-          // causes a confusing NegativeArraySizeException.
-          // Note that very likely this will simply then hit
-          // an OOME, but at least that's more indicative to
-          // caller that this values is too big.  We don't +1
-          // in this case, but it's very unlikely in practice
-          // one will actually insert this many objects into
-          // the PQ:
-          // Throw exception to prevent confusing OOME:
-          throw new IllegalArgumentException("maxSize must be <= " + ArrayUtil.MAX_ARRAY_LENGTH + "; got: " + maxSize);
-        } else {
-          // NOTE: we add +1 because all access to heap is
-          // 1-based not 0-based.  heap[0] is unused.
-          heapSize = maxSize + 1;
-        }
-      }
-      // T is unbounded type, so this unchecked cast works always:
-      @SuppressWarnings("unchecked") final T[] h = (T[]) new Object[heapSize];
-      this.heap = h;
-      this.maxSize = maxSize;
-
-      if (prepopulate) {
-        // If sentinel objects are supported, populate the queue with them
-        T sentinel = getSentinelObject();
-        if (sentinel != null) {
-          heap[1] = sentinel;
-          for (int i = 2; i < heap.length; i++) {
-            heap[i] = getSentinelObject();
-          }
-          size = maxSize;
-        }
-      }
-    }
-
-    /** Determines the ordering of objects in this priority queue.  Subclasses
-     *  must define this one method.
-     *  @return <code>true</code> iff parameter <tt>a</tt> is less than parameter <tt>b</tt>.
-     */
-    protected abstract boolean lessThan(T a, T b);
-
-
-    protected T getSentinelObject() {
-      return null;
-    }
-
-    /**
-     * Adds an Object to a PriorityQueue in log(size) time. If one tries to add
-     * more objects than maxSize from initialize an
-     *
-     * @return the new 'top' element in the queue.
-     */
-    public final T add(T element) {
-      size++;
-      heap[size] = element;
-      upHeap();
-      return heap[1];
-    }
-
-    /**
-     * Adds an Object to a PriorityQueue in log(size) time.
-     * It returns the object (if any) that was
-     * dropped off the heap because it was full. This can be
-     * the given parameter (in case it is smaller than the
-     * full heap's minimum, and couldn't be added), or another
-     * object that was previously the smallest value in the
-     * heap and now has been replaced by a larger one, or null
-     * if the queue wasn't yet full with maxSize elements.
-     */
-    public T insertWithOverflow(T element) {
-      if (size < maxSize) {
-        add(element);
-        return null;
-      } else if (size > 0 && !lessThan(element, heap[1])) {
-        T ret = heap[1];
-        heap[1] = element;
-        updateTop();
-        return ret;
-      } else {
-        return element;
-      }
-    }
-
-    /** Returns the least element of the PriorityQueue in constant time. */
-    public final T top() {
-      // We don't need to check size here: if maxSize is 0,
-      // then heap is length 2 array with both entries null.
-      // If size is 0 then heap[1] is already null.
-      return heap[1];
-    }
-
-    /** Removes and returns the least element of the PriorityQueue in log(size)
-     time. */
-    public final T pop() {
-      if (size > 0) {
-        T result = heap[1];       // save first value
-        heap[1] = heap[size];     // move last to first
-        heap[size] = null;        // permit GC of objects
-        size--;
-        downHeap();               // adjust heap
-        return result;
-      } else {
-        return null;
-      }
-    }
-
-    /**
-     * Should be called when the Object at top changes values. Still log(n) worst
-     * case, but it's at least twice as fast to
-     *
-     * <pre class="prettyprint">
-     * pq.top().change();
-     * pq.updateTop();
-     * </pre>
-     *
-     * instead of
-     *
-     * <pre class="prettyprint">
-     * o = pq.pop();
-     * o.change();
-     * pq.push(o);
-     * </pre>
-     *
-     * @return the new 'top' element.
-     */
-    public final T updateTop() {
-      downHeap();
-      return heap[1];
-    }
-
-    /** Returns the number of elements currently stored in the PriorityQueue. */
-    public final int size() {
-      return size;
-    }
-
-    /** Removes all entries from the PriorityQueue. */
-    public final void clear() {
-      for (int i = 0; i <= size; i++) {
-        heap[i] = null;
-      }
-      size = 0;
-    }
-
-    private final void upHeap() {
-      int i = size;
-      T node = heap[i];          // save bottom node
-      int j = i >>> 1;
-      while (j > 0 && lessThan(node, heap[j])) {
-        heap[i] = heap[j];       // shift parents down
-        i = j;
-        j = j >>> 1;
-      }
-      heap[i] = node;            // install saved node
-    }
-
-    private final void downHeap() {
-      int i = 1;
-      T node = heap[i];          // save top node
-      int j = i << 1;            // find smaller child
-      int k = j + 1;
-      if (k <= size && lessThan(heap[k], heap[j])) {
-        j = k;
-      }
-      while (j <= size && lessThan(heap[j], node)) {
-        heap[i] = heap[j];       // shift up child
-        i = j;
-        j = i << 1;
-        k = j + 1;
-        if (k <= size && lessThan(heap[k], heap[j])) {
-          j = k;
-        }
-      }
-      heap[i] = node;            // install saved node
-    }
-
-    /** This method returns the internal heap array as Object[].
-     * @lucene.internal
-     */
-    public final Object[] getHeapArray() {
-      return (Object[]) heap;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e6dfb76c/solr/core/src/java/org/apache/solr/response/TextResponseWriter.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/response/TextResponseWriter.java b/solr/core/src/java/org/apache/solr/response/TextResponseWriter.java
index 255d338..c4c2994 100644
--- a/solr/core/src/java/org/apache/solr/response/TextResponseWriter.java
+++ b/solr/core/src/java/org/apache/solr/response/TextResponseWriter.java
@@ -31,9 +31,12 @@ import org.apache.lucene.document.Document;
 import org.apache.lucene.index.IndexableField;
 import org.apache.lucene.util.BytesRef;
 import org.apache.solr.common.EnumFieldValue;
+import org.apache.solr.common.IteratorWriter;
 import org.apache.solr.common.MapSerializable;
+import org.apache.solr.common.PushWriter;
 import org.apache.solr.common.SolrDocument;
 import org.apache.solr.common.SolrDocumentList;
+import org.apache.solr.common.MapWriter;
 import org.apache.solr.common.params.CommonParams;
 import org.apache.solr.common.util.Base64;
 import org.apache.solr.common.util.NamedList;
@@ -48,7 +51,7 @@ import org.apache.solr.util.FastWriter;
  *
  *
  */
-public abstract class TextResponseWriter {
+public abstract class TextResponseWriter implements PushWriter {
 
   // indent up to 40 spaces
   static final char[] indentChars = new char[81];
@@ -138,19 +141,19 @@ public abstract class TextResponseWriter {
         writeStr(name, f.stringValue(), true);
       }
     } else if (val instanceof Number) {
-      writeNumber(name, (Number)val);
+      writeNumber(name, (Number) val);
     } else if (val instanceof Boolean) {
-      writeBool(name, (Boolean)val);
+      writeBool(name, (Boolean) val);
     } else if (val instanceof Date) {
-      writeDate(name,(Date)val);
+      writeDate(name, (Date) val);
     } else if (val instanceof Document) {
       SolrDocument doc = DocsStreamer.getDoc((Document) val, schema);
-      writeSolrDocument(name, doc,returnFields, 0 );
+      writeSolrDocument(name, doc, returnFields, 0);
     } else if (val instanceof SolrDocument) {
-      writeSolrDocument(name, (SolrDocument)val,returnFields, 0);
+      writeSolrDocument(name, (SolrDocument) val, returnFields, 0);
     } else if (val instanceof ResultContext) {
       // requires access to IndexReader
-      writeDocuments(name, (ResultContext)val);
+      writeDocuments(name, (ResultContext) val);
     } else if (val instanceof DocList) {
       // Should not happen normally
       ResultContext ctx = new BasicResultContext((DocList)val, returnFields, null, null, req);
@@ -168,6 +171,8 @@ public abstract class TextResponseWriter {
       writeNamedList(name, (NamedList)val);
     } else if (val instanceof Path) {
       writeStr(name, ((Path) val).toAbsolutePath().toString(), true);
+    } else if (val instanceof IteratorWriter) {
+      writeIterator((IteratorWriter) val);
     } else if (val instanceof Iterable) {
       writeArray(name,((Iterable)val).iterator());
     } else if (val instanceof Object[]) {
@@ -184,6 +189,8 @@ public abstract class TextResponseWriter {
       writeStr(name, val.toString(), true);
     } else if (val instanceof WriteableValue) {
       ((WriteableValue)val).write(name, this);
+    } else if (val instanceof MapWriter) {
+      writeMap((MapWriter) val);
     } else if (val instanceof MapSerializable) {
       //todo find a better way to reuse the map more efficiently
       writeMap(name, ((MapSerializable) val).toMap(new LinkedHashMap<>()), false, true);
@@ -192,6 +199,15 @@ public abstract class TextResponseWriter {
       writeStr(name, val.getClass().getName() + ':' + val.toString(), true);
     }
   }
+  @Override
+  public void writeMap(MapWriter mw) throws IOException {
+    //todo
+  }
+
+  @Override
+  public void writeIterator(IteratorWriter iw) throws IOException {
+    /*todo*/
+  }
 
   protected void writeBool(String name , Boolean val) throws IOException {
     writeBool(name, val.toString());

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e6dfb76c/solr/core/src/resources/ImplicitPlugins.json
----------------------------------------------------------------------
diff --git a/solr/core/src/resources/ImplicitPlugins.json b/solr/core/src/resources/ImplicitPlugins.json
index 8bf2106..34e5c07 100644
--- a/solr/core/src/resources/ImplicitPlugins.json
+++ b/solr/core/src/resources/ImplicitPlugins.json
@@ -92,14 +92,16 @@
       "useParams":"_ADMIN_FILE"
     },
     "/export": {
-      "class": "solr.SearchHandler",
+      "class": "solr.ExportHandler",
       "useParams":"_EXPORT",
       "components": [
         "query"
       ],
+      "defaults": {
+        "wt": "json"
+      },
       "invariants": {
         "rq": "{!xport}",
-        "wt": "xsort",
         "distrib": false
       }
     },

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e6dfb76c/solr/core/src/test/org/apache/solr/core/SolrCoreTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/core/SolrCoreTest.java b/solr/core/src/test/org/apache/solr/core/SolrCoreTest.java
index 2799078..004039c 100644
--- a/solr/core/src/test/org/apache/solr/core/SolrCoreTest.java
+++ b/solr/core/src/test/org/apache/solr/core/SolrCoreTest.java
@@ -97,7 +97,7 @@ public class SolrCoreTest extends SolrTestCaseJ4 {
       ++ihCount; assertEquals(pathToClassMap.get("/admin/system"), "solr.SystemInfoHandler");
       ++ihCount; assertEquals(pathToClassMap.get("/admin/threads"), "solr.ThreadDumpHandler");
       ++ihCount; assertEquals(pathToClassMap.get("/config"), "solr.SolrConfigHandler");
-      ++ihCount; assertEquals(pathToClassMap.get("/export"), "solr.SearchHandler");
+      ++ihCount; assertEquals(pathToClassMap.get("/export"), "solr.ExportHandler");
       ++ihCount; assertEquals(pathToClassMap.get("/terms"), "solr.SearchHandler");
       ++ihCount; assertEquals(pathToClassMap.get("/get"), "solr.RealTimeGetHandler");
       ++ihCount; assertEquals(pathToClassMap.get(ReplicationHandler.PATH), "solr.ReplicationHandler");

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e6dfb76c/solr/core/src/test/org/apache/solr/response/JSONWriterTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/response/JSONWriterTest.java b/solr/core/src/test/org/apache/solr/response/JSONWriterTest.java
index b096a09..2dce08e 100644
--- a/solr/core/src/test/org/apache/solr/response/JSONWriterTest.java
+++ b/solr/core/src/test/org/apache/solr/response/JSONWriterTest.java
@@ -184,12 +184,15 @@ public class JSONWriterTest extends SolrTestCaseJ4 {
     methodsExpectedNotOverriden.add("writeArrayOpener");
     methodsExpectedNotOverriden.add("writeArraySeparator");
     methodsExpectedNotOverriden.add("writeArrayCloser");
+    methodsExpectedNotOverriden.add("public void org.apache.solr.response.JSONWriter.writeMap(org.apache.solr.common.MapWriter) throws java.io.IOException");
+    methodsExpectedNotOverriden.add("public void org.apache.solr.response.JSONWriter.writeIterator(org.apache.solr.common.IteratorWriter) throws java.io.IOException");
 
     final Class<?> subClass = ArrayOfNamedValuePairJSONWriter.class;
     final Class<?> superClass = subClass.getSuperclass();
 
     for (final Method superClassMethod : superClass.getDeclaredMethods()) {
       final String methodName = superClassMethod.getName();
+      final String methodFullName = superClassMethod.toString();
       if (!methodName.startsWith("write")) continue;
 
       final int modifiers = superClassMethod.getModifiers();
@@ -197,7 +200,8 @@ public class JSONWriterTest extends SolrTestCaseJ4 {
       if (Modifier.isStatic(modifiers)) continue;
       if (Modifier.isPrivate(modifiers)) continue;
 
-      final boolean expectOverriden = !methodsExpectedNotOverriden.contains(methodName);
+      final boolean expectOverriden = !methodsExpectedNotOverriden.contains(methodName)
+          && !methodsExpectedNotOverriden.contains(methodFullName);
 
       try {
         final Method subClassMethod = subClass.getDeclaredMethod(
@@ -215,7 +219,7 @@ public class JSONWriterTest extends SolrTestCaseJ4 {
         if (expectOverriden) {
           fail(subClass + " needs to override '" + superClassMethod + "'");
         } else {
-          assertTrue(methodName+" not found in remaining "+methodsExpectedNotOverriden, methodsExpectedNotOverriden.remove(methodName));
+          assertTrue(methodName+" not found in remaining "+methodsExpectedNotOverriden, methodsExpectedNotOverriden.remove(methodName)|| methodsExpectedNotOverriden.remove(methodFullName));
         }
       }
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e6dfb76c/solr/core/src/test/org/apache/solr/response/TestExportWriter.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/response/TestExportWriter.java b/solr/core/src/test/org/apache/solr/response/TestExportWriter.java
new file mode 100644
index 0000000..5a303e9
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/response/TestExportWriter.java
@@ -0,0 +1,195 @@
+/*
+ * 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.response;
+
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.common.util.Utils;
+import org.junit.*;
+import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
+
+@SuppressCodecs({"Lucene3x", "Lucene40","Lucene41","Lucene42","Lucene45"})
+public class TestExportWriter extends SolrTestCaseJ4 {
+  @BeforeClass
+  public static void beforeClass() throws Exception {
+    System.setProperty("export.test", "true");
+    initCore("solrconfig-sortingresponse.xml","schema-sortingresponse.xml");
+    createIndex();
+  }
+
+  public static void createIndex() {
+    assertU(adoc("id","1",
+                 "floatdv","2.1",
+                 "intdv", "1",
+                 "stringdv", "hello world",
+                 "longdv", "323223232323",
+                 "doubledv","2344.345",
+                 "intdv_m","100",
+                 "intdv_m","250",
+                 "floatdv_m", "123.321",
+                 "floatdv_m", "345.123",
+                 "doubledv_m", "3444.222",
+                 "doubledv_m", "23232.2",
+                 "longdv_m", "43434343434",
+                 "longdv_m", "343332",
+                 "stringdv_m", "manchester \"city\"",
+                 "stringdv_m", "liverpool",
+                 "stringdv_m", "Everton"));
+
+    assertU(adoc("id","7",
+        "floatdv","2.1",
+        "intdv", "7",
+        "longdv", "323223232323",
+        "doubledv","2344.345",
+        "floatdv_m", "123.321",
+        "floatdv_m", "345.123",
+        "doubledv_m", "3444.222",
+        "doubledv_m", "23232.2",
+        "longdv_m", "43434343434",
+        "longdv_m", "343332"));
+
+    assertU(commit());
+    assertU(adoc("id","2", "floatdv","2.1", "intdv", "2", "stringdv", "hello world", "longdv", "323223232323","doubledv","2344.344"));
+    assertU(commit());
+    assertU(adoc("id","3",
+        "floatdv","2.1",
+        "intdv", "3",
+        "stringdv", "chello world",
+        "longdv", "323223232323",
+        "doubledv","2344.346",
+        "intdv_m","100",
+        "intdv_m","250",
+        "floatdv_m", "123.321",
+        "floatdv_m", "345.123",
+        "doubledv_m", "3444.222",
+        "doubledv_m", "23232.2",
+        "longdv_m", "43434343434",
+        "longdv_m", "343332",
+        "stringdv_m", "manchester \"city\"",
+        "stringdv_m", "liverpool",
+        "stringdv_m", "everton"));
+    assertU(commit());
+    assertU(adoc("id","8",
+        "floatdv","2.1",
+        "intdv", "10000000",
+        "stringdv", "chello \"world\"",
+        "longdv", "323223232323",
+        "doubledv","2344.346",
+        "intdv_m","100",
+        "intdv_m","250",
+        "floatdv_m", "123.321",
+        "floatdv_m", "345.123",
+        "doubledv_m", "3444.222",
+        "doubledv_m", "23232.2",
+        "longdv_m", "43434343434",
+        "longdv_m", "343332",
+        "stringdv_m", "manchester \"city\"",
+        "stringdv_m", "liverpool",
+        "stringdv_m", "everton"));
+    assertU(commit());
+
+
+  }
+
+  @Test
+  public void testSortingOutput() throws Exception {
+
+    //Test single value DocValue output
+    String s =  h.query(req("q", "id:1", "qt", "/export", "fl", "floatdv,intdv,stringdv,longdv,doubledv", "sort", "intdv asc"));
+
+    assertJsonEquals(s, "{\"responseHeader\": {\"status\": 0}, \"response\":{\"numFound\":1, \"docs\":[{\"floatdv\":2.1,\"intdv\":1,\"stringdv\":\"hello world\",\"longdv\":323223232323,\"doubledv\":2344.345}]}}");
+
+    //Test null value string:
+    s = h.query(req("q", "id:7", "qt", "/export", "fl", "floatdv,intdv,stringdv,longdv,doubledv", "sort", "intdv asc"));
+
+    assertJsonEquals(s, "{\"responseHeader\": {\"status\": 0}, \"response\":{\"numFound\":1, \"docs\":[{\"floatdv\":2.1,\"intdv\":7,\"longdv\":323223232323,\"doubledv\":2344.345}]}}");
+
+    //Test multiValue docValues output
+    s = h.query(req("q", "id:1", "qt", "/export", "fl", "intdv_m,floatdv_m,doubledv_m,longdv_m,stringdv_m", "sort", "intdv asc"));
+    assertJsonEquals(s, "{\"responseHeader\": {\"status\": 0}, \"response\":{\"numFound\":1, \"docs\":[{\"intdv_m\":[100,250],\"floatdv_m\":[123.321,345.123],\"doubledv_m\":[3444.222,23232.2],\"longdv_m\":[343332,43434343434],\"stringdv_m\":[\"Everton\",\"liverpool\",\"manchester \\\"city\\\"\"]}]}}");
+
+    //Test multiValues docValues output with nulls
+    s =  h.query(req("q", "id:7", "qt", "/export", "fl", "intdv_m,floatdv_m,doubledv_m,longdv_m,stringdv_m", "sort", "intdv asc"));
+    assertJsonEquals(s, "{\"responseHeader\": {\"status\": 0}, \"response\":{\"numFound\":1, \"docs\":[{\"floatdv_m\":[123.321,345.123],\"doubledv_m\":[3444.222,23232.2],\"longdv_m\":[343332,43434343434]}]}}");
+
+    //Test single sort param is working
+    s =  h.query(req("q", "id:(1 2)", "qt", "/export", "fl", "intdv", "sort", "intdv desc"));
+    assertJsonEquals(s, "{\"responseHeader\": {\"status\": 0}, \"response\":{\"numFound\":2, \"docs\":[{\"intdv\":2},{\"intdv\":1}]}}");
+
+    s =  h.query(req("q", "id:(1 2)", "qt", "/export", "fl", "intdv", "sort", "intdv asc"));
+    assertJsonEquals(s, "{\"responseHeader\": {\"status\": 0}, \"response\":{\"numFound\":2, \"docs\":[{\"intdv\":1},{\"intdv\":2}]}}");
+
+    // Test sort on String will null value. Null value should sort last on desc and first on asc.
+
+    s =  h.query(req("q", "id:(1 7)", "qt", "/export", "fl", "intdv", "sort", "stringdv desc"));
+    assertJsonEquals(s, "{\"responseHeader\": {\"status\": 0}, \"response\":{\"numFound\":2, \"docs\":[{\"intdv\":1},{\"intdv\":7}]}}");
+
+    s =  h.query(req("q", "id:(1 7)", "qt", "/export", "fl", "intdv", "sort", "stringdv asc"));
+    assertJsonEquals(s, "{\"responseHeader\": {\"status\": 0}, \"response\":{\"numFound\":2, \"docs\":[{\"intdv\":7},{\"intdv\":1}]}}");
+
+
+    //Test multi-sort params
+    s =  h.query(req("q", "id:(1 2)", "qt", "/export", "fl", "intdv", "sort", "floatdv asc,intdv desc"));
+    assertJsonEquals(s, "{\"responseHeader\": {\"status\": 0}, \"response\":{\"numFound\":2, \"docs\":[{\"intdv\":2},{\"intdv\":1}]}}");
+
+    s =  h.query(req("q", "id:(1 2)", "qt", "/export", "fl", "intdv", "sort", "floatdv desc,intdv asc"));
+    assertJsonEquals(s, "{\"responseHeader\": {\"status\": 0}, \"response\":{\"numFound\":2, \"docs\":[{\"intdv\":1},{\"intdv\":2}]}}");
+
+    //Test three sort fields
+    s =  h.query(req("q", "id:(1 2 3)", "qt", "/export", "fl", "intdv", "sort", "floatdv asc,stringdv asc,intdv desc"));
+    assertJsonEquals(s, "{\"responseHeader\": {\"status\": 0}, \"response\":{\"numFound\":3, \"docs\":[{\"intdv\":3},{\"intdv\":2},{\"intdv\":1}]}}");
+
+    //Test three sort fields
+    s =  h.query(req("q", "id:(1 2 3)", "qt", "/export", "fl", "intdv", "sort", "floatdv asc,stringdv desc,intdv asc"));
+    assertJsonEquals(s, "{\"responseHeader\": {\"status\": 0}, \"response\":{\"numFound\":3, \"docs\":[{\"intdv\":1},{\"intdv\":2},{\"intdv\":3}]}}");
+
+    //Test four sort fields
+    s =  h.query(req("q", "id:(1 2 3)", "qt", "/export", "fl", "intdv", "sort", "floatdv asc,floatdv desc,floatdv asc,intdv desc"));
+    assertJsonEquals(s, "{\"responseHeader\": {\"status\": 0}, \"response\":{\"numFound\":3, \"docs\":[{\"intdv\":3},{\"intdv\":2},{\"intdv\":1}]}}");
+
+    s =  h.query(req("q", "id:(1 2 3)", "qt", "/export", "fl", "intdv", "sort", "doubledv desc"));
+    assertJsonEquals(s, "{\"responseHeader\": {\"status\": 0}, \"response\":{\"numFound\":3, \"docs\":[{\"intdv\":3},{\"intdv\":1},{\"intdv\":2}]}}");
+
+    s =  h.query(req("q", "intdv:[2 TO 1000]", "qt", "/export", "fl", "intdv", "sort", "doubledv desc"));
+    assertJsonEquals(s, "{\"responseHeader\": {\"status\": 0}, \"response\":{\"numFound\":3, \"docs\":[{\"intdv\":3},{\"intdv\":7},{\"intdv\":2}]}}");
+
+    s =  h.query(req("q", "stringdv:blah", "qt", "/export", "fl", "intdv", "sort", "doubledv desc"));
+    assertJsonEquals(s, "{\"responseHeader\": {\"status\": 0}, \"response\":{\"numFound\":0, \"docs\":[]}}");
+
+    s =  h.query(req("q", "id:8", "qt", "/export", "fl", "stringdv", "sort", "intdv asc"));
+    assertJsonEquals(s, "{\"responseHeader\": {\"status\": 0}, \"response\":{\"numFound\":1, \"docs\":[{\"stringdv\":\"chello \\\"world\\\"\"}]}}");
+  }
+
+  private void assertJsonEquals(String actual, String expected) {
+    assertEquals(Utils.toJSONString(Utils.fromJSONString(expected)), Utils.toJSONString(Utils.fromJSONString(actual)));
+  }
+
+  @Test
+  public void testExportRequiredParams() throws Exception {
+
+    //Test whether missing required parameters returns expected errors.
+
+    //String s =  h.query(req("q", "id:1", "qt", "/export", "fl", "floatdv,intdv,stringdv,longdv,doubledv", "sort", "intdv asc"));
+    String s;
+    s = h.query(req("qt", "/export"));
+    assertTrue("Should have had a sort error", s.contains("No sort criteria"));
+    s = h.query(req("sort", "intdv asc", "qt", "/export"));
+    assertTrue("Should have had fl error", s.contains("export field list (fl) must be specified"));
+    s = h.query(req("sort", "intdv asc", "qt", "/export", "fl", "stringdv"));
+    // Interesting you don't even need to specify a "q" parameter.
+    
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e6dfb76c/solr/core/src/test/org/apache/solr/response/TestPushWriter.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/response/TestPushWriter.java b/solr/core/src/test/org/apache/solr/response/TestPushWriter.java
new file mode 100644
index 0000000..5d42645
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/response/TestPushWriter.java
@@ -0,0 +1,76 @@
+/*
+ * 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.response;
+
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.OutputStreamWriter;
+import java.lang.invoke.MethodHandles;
+import java.util.Map;
+
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.common.IteratorWriter;
+import org.apache.solr.common.MapWriter;
+import org.apache.solr.common.PushWriter;
+import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.common.util.Utils;
+import org.apache.solr.request.LocalSolrQueryRequest;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static java.util.Collections.singletonMap;
+
+public class TestPushWriter extends SolrTestCaseJ4 {
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+
+  public void testStandardResponse() throws IOException {
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    OutputStreamWriter osw = new OutputStreamWriter(baos, UTF_8);
+    PushWriter pw = new JSONWriter(osw,
+        new LocalSolrQueryRequest(null, new ModifiableSolrParams()), new SolrQueryResponse());
+    writeData(pw);
+    osw.flush();
+    log.info(new String(baos.toByteArray(), "UTF-8"));
+    Object m = Utils.fromJSON(baos.toByteArray());
+    checkValues((Map) m);
+  }
+
+  protected void checkValues(Map m) {
+    assertEquals(0, ((Number)Utils.getObjectByPath(m, true, "responseHeader/status")).intValue());
+    assertEquals(10, ((Number)Utils.getObjectByPath(m, true, "response/numFound")).intValue());
+    assertEquals(1, ((Number)Utils.getObjectByPath(m, true, "response/docs[0]/id")).intValue());
+    assertEquals(2, ((Number)Utils.getObjectByPath(m, true, "response/docs[1]/id")).intValue());
+    assertEquals(3, ((Number)Utils.getObjectByPath(m, true, "response/docs[2]/id")).intValue());
+  }
+
+  protected void writeData(PushWriter pw) throws IOException {
+    pw.writeMap(m -> {
+      m.put("responseHeader", singletonMap("status", 0))
+          .put("response", (MapWriter) m1 -> {
+            m1.put("numFound", 10)
+                .put("docs", (IteratorWriter) w -> {
+                  w.add((MapWriter) m3 -> m3.put("id", 1))
+                      .add(singletonMap("id", 2))
+                      .add(singletonMap("id", 3));
+                }); }); });
+    pw.close();
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e6dfb76c/solr/core/src/test/org/apache/solr/response/TestSortingResponseWriter.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/response/TestSortingResponseWriter.java b/solr/core/src/test/org/apache/solr/response/TestSortingResponseWriter.java
deleted file mode 100644
index 4b18133..0000000
--- a/solr/core/src/test/org/apache/solr/response/TestSortingResponseWriter.java
+++ /dev/null
@@ -1,190 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.solr.response;
-
-import org.apache.solr.SolrTestCaseJ4;
-import org.junit.*;
-import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
-
-@SuppressCodecs({"Lucene3x", "Lucene40","Lucene41","Lucene42","Lucene45"})
-public class TestSortingResponseWriter extends SolrTestCaseJ4 {
-  @BeforeClass
-  public static void beforeClass() throws Exception {
-    System.setProperty("export.test", "true");
-    initCore("solrconfig-sortingresponse.xml","schema-sortingresponse.xml");
-    createIndex();
-  }
-
-  public static void createIndex() {
-    assertU(adoc("id","1",
-                 "floatdv","2.1",
-                 "intdv", "1",
-                 "stringdv", "hello world",
-                 "longdv", "323223232323",
-                 "doubledv","2344.345",
-                 "intdv_m","100",
-                 "intdv_m","250",
-                 "floatdv_m", "123.321",
-                 "floatdv_m", "345.123",
-                 "doubledv_m", "3444.222",
-                 "doubledv_m", "23232.2",
-                 "longdv_m", "43434343434",
-                 "longdv_m", "343332",
-                 "stringdv_m", "manchester \"city\"",
-                 "stringdv_m", "liverpool",
-                 "stringdv_m", "Everton"));
-
-    assertU(adoc("id","7",
-        "floatdv","2.1",
-        "intdv", "7",
-        "longdv", "323223232323",
-        "doubledv","2344.345",
-        "floatdv_m", "123.321",
-        "floatdv_m", "345.123",
-        "doubledv_m", "3444.222",
-        "doubledv_m", "23232.2",
-        "longdv_m", "43434343434",
-        "longdv_m", "343332"));
-
-    assertU(commit());
-    assertU(adoc("id","2", "floatdv","2.1", "intdv", "2", "stringdv", "hello world", "longdv", "323223232323","doubledv","2344.344"));
-    assertU(commit());
-    assertU(adoc("id","3",
-        "floatdv","2.1",
-        "intdv", "3",
-        "stringdv", "chello world",
-        "longdv", "323223232323",
-        "doubledv","2344.346",
-        "intdv_m","100",
-        "intdv_m","250",
-        "floatdv_m", "123.321",
-        "floatdv_m", "345.123",
-        "doubledv_m", "3444.222",
-        "doubledv_m", "23232.2",
-        "longdv_m", "43434343434",
-        "longdv_m", "343332",
-        "stringdv_m", "manchester \"city\"",
-        "stringdv_m", "liverpool",
-        "stringdv_m", "everton"));
-    assertU(commit());
-    assertU(adoc("id","8",
-        "floatdv","2.1",
-        "intdv", "10000000",
-        "stringdv", "chello \"world\"",
-        "longdv", "323223232323",
-        "doubledv","2344.346",
-        "intdv_m","100",
-        "intdv_m","250",
-        "floatdv_m", "123.321",
-        "floatdv_m", "345.123",
-        "doubledv_m", "3444.222",
-        "doubledv_m", "23232.2",
-        "longdv_m", "43434343434",
-        "longdv_m", "343332",
-        "stringdv_m", "manchester \"city\"",
-        "stringdv_m", "liverpool",
-        "stringdv_m", "everton"));
-    assertU(commit());
-
-
-  }
-
-  @Test
-  public void testSortingOutput() throws Exception {
-
-    //Test single value DocValue output
-    String s =  h.query(req("q", "id:1", "qt", "/export", "fl", "floatdv,intdv,stringdv,longdv,doubledv", "sort", "intdv asc"));
-
-    assertEquals(s, "{\"responseHeader\": {\"status\": 0}, \"response\":{\"numFound\":1, \"docs\":[{\"floatdv\":2.1,\"intdv\":1,\"stringdv\":\"hello world\",\"longdv\":323223232323,\"doubledv\":2344.345}]}}");
-
-    //Test null value string:
-    s = h.query(req("q", "id:7", "qt", "/export", "fl", "floatdv,intdv,stringdv,longdv,doubledv", "sort", "intdv asc"));
-
-    assertEquals(s, "{\"responseHeader\": {\"status\": 0}, \"response\":{\"numFound\":1, \"docs\":[{\"floatdv\":2.1,\"intdv\":7,\"longdv\":323223232323,\"doubledv\":2344.345}]}}");
-
-    //Test multiValue docValues output
-    s = h.query(req("q", "id:1", "qt", "/export", "fl", "intdv_m,floatdv_m,doubledv_m,longdv_m,stringdv_m", "sort", "intdv asc"));
-    assertEquals(s, "{\"responseHeader\": {\"status\": 0}, \"response\":{\"numFound\":1, \"docs\":[{\"intdv_m\":[100,250],\"floatdv_m\":[123.321,345.123],\"doubledv_m\":[3444.222,23232.2],\"longdv_m\":[343332,43434343434],\"stringdv_m\":[\"Everton\",\"liverpool\",\"manchester \\\"city\\\"\"]}]}}");
-
-    //Test multiValues docValues output with nulls
-    s =  h.query(req("q", "id:7", "qt", "/export", "fl", "intdv_m,floatdv_m,doubledv_m,longdv_m,stringdv_m", "sort", "intdv asc"));
-    assertEquals(s, "{\"responseHeader\": {\"status\": 0}, \"response\":{\"numFound\":1, \"docs\":[{\"floatdv_m\":[123.321,345.123],\"doubledv_m\":[3444.222,23232.2],\"longdv_m\":[343332,43434343434]}]}}");
-
-    //Test single sort param is working
-    s =  h.query(req("q", "id:(1 2)", "qt", "/export", "fl", "intdv", "sort", "intdv desc"));
-    assertEquals(s, "{\"responseHeader\": {\"status\": 0}, \"response\":{\"numFound\":2, \"docs\":[{\"intdv\":2},{\"intdv\":1}]}}");
-
-    s =  h.query(req("q", "id:(1 2)", "qt", "/export", "fl", "intdv", "sort", "intdv asc"));
-    assertEquals(s, "{\"responseHeader\": {\"status\": 0}, \"response\":{\"numFound\":2, \"docs\":[{\"intdv\":1},{\"intdv\":2}]}}");
-
-    // Test sort on String will null value. Null value should sort last on desc and first on asc.
-
-    s =  h.query(req("q", "id:(1 7)", "qt", "/export", "fl", "intdv", "sort", "stringdv desc"));
-    assertEquals(s, "{\"responseHeader\": {\"status\": 0}, \"response\":{\"numFound\":2, \"docs\":[{\"intdv\":1},{\"intdv\":7}]}}");
-
-    s =  h.query(req("q", "id:(1 7)", "qt", "/export", "fl", "intdv", "sort", "stringdv asc"));
-    assertEquals(s, "{\"responseHeader\": {\"status\": 0}, \"response\":{\"numFound\":2, \"docs\":[{\"intdv\":7},{\"intdv\":1}]}}");
-
-
-    //Test multi-sort params
-    s =  h.query(req("q", "id:(1 2)", "qt", "/export", "fl", "intdv", "sort", "floatdv asc,intdv desc"));
-    assertEquals(s, "{\"responseHeader\": {\"status\": 0}, \"response\":{\"numFound\":2, \"docs\":[{\"intdv\":2},{\"intdv\":1}]}}");
-
-    s =  h.query(req("q", "id:(1 2)", "qt", "/export", "fl", "intdv", "sort", "floatdv desc,intdv asc"));
-    assertEquals(s, "{\"responseHeader\": {\"status\": 0}, \"response\":{\"numFound\":2, \"docs\":[{\"intdv\":1},{\"intdv\":2}]}}");
-
-    //Test three sort fields
-    s =  h.query(req("q", "id:(1 2 3)", "qt", "/export", "fl", "intdv", "sort", "floatdv asc,stringdv asc,intdv desc"));
-    assertEquals(s, "{\"responseHeader\": {\"status\": 0}, \"response\":{\"numFound\":3, \"docs\":[{\"intdv\":3},{\"intdv\":2},{\"intdv\":1}]}}");
-
-    //Test three sort fields
-    s =  h.query(req("q", "id:(1 2 3)", "qt", "/export", "fl", "intdv", "sort", "floatdv asc,stringdv desc,intdv asc"));
-    assertEquals(s, "{\"responseHeader\": {\"status\": 0}, \"response\":{\"numFound\":3, \"docs\":[{\"intdv\":1},{\"intdv\":2},{\"intdv\":3}]}}");
-
-    //Test four sort fields
-    s =  h.query(req("q", "id:(1 2 3)", "qt", "/export", "fl", "intdv", "sort", "floatdv asc,floatdv desc,floatdv asc,intdv desc"));
-    assertEquals(s, "{\"responseHeader\": {\"status\": 0}, \"response\":{\"numFound\":3, \"docs\":[{\"intdv\":3},{\"intdv\":2},{\"intdv\":1}]}}");
-
-    s =  h.query(req("q", "id:(1 2 3)", "qt", "/export", "fl", "intdv", "sort", "doubledv desc"));
-    assertEquals(s, "{\"responseHeader\": {\"status\": 0}, \"response\":{\"numFound\":3, \"docs\":[{\"intdv\":3},{\"intdv\":1},{\"intdv\":2}]}}");
-
-    s =  h.query(req("q", "intdv:[2 TO 1000]", "qt", "/export", "fl", "intdv", "sort", "doubledv desc"));
-    assertEquals(s, "{\"responseHeader\": {\"status\": 0}, \"response\":{\"numFound\":3, \"docs\":[{\"intdv\":3},{\"intdv\":7},{\"intdv\":2}]}}");
-
-    s =  h.query(req("q", "stringdv:blah", "qt", "/export", "fl", "intdv", "sort", "doubledv desc"));
-    assertEquals(s, "{\"responseHeader\": {\"status\": 0}, \"response\":{\"numFound\":0, \"docs\":[]}}");
-
-    s =  h.query(req("q", "id:8", "qt", "/export", "fl", "stringdv", "sort", "intdv asc"));
-    assertEquals(s, "{\"responseHeader\": {\"status\": 0}, \"response\":{\"numFound\":1, \"docs\":[{\"stringdv\":\"chello \\\"world\\\"\"}]}}");
-  }
-
-  @Test
-  public void testExportRequiredParams() throws Exception {
-
-    //Test whether missing required parameters returns expected errors.
-
-    //String s =  h.query(req("q", "id:1", "qt", "/export", "fl", "floatdv,intdv,stringdv,longdv,doubledv", "sort", "intdv asc"));
-    String s;
-    s = h.query(req("qt", "/export"));
-    assertTrue("Should have had a sort error", s.contains("No sort criteria"));
-    s = h.query(req("sort", "intdv asc", "qt", "/export"));
-    assertTrue("Should have had fl error", s.contains("export field list (fl) must be specified"));
-    s = h.query(req("sort", "intdv asc", "qt", "/export", "fl", "stringdv"));
-    // Interesting you don't even need to specify a "q" parameter.
-    
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e6dfb76c/solr/solrj/src/java/org/apache/solr/common/IteratorWriter.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/common/IteratorWriter.java b/solr/solrj/src/java/org/apache/solr/common/IteratorWriter.java
new file mode 100644
index 0000000..1294784
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/common/IteratorWriter.java
@@ -0,0 +1,63 @@
+/*
+ * 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.common;
+
+
+import java.io.IOException;
+
+/**
+ * Interface to help do push writing to an array
+ */
+public interface IteratorWriter {
+  /**
+   * @param w after this method returns , the EntryWriter Object is invalid
+   *          Do not hold a reference to this object
+   */
+  void writeIter(ItemWriter w) throws IOException;
+
+  interface ItemWriter {
+    ItemWriter add(Object o) throws IOException;
+
+    default ItemWriter add(int v) throws IOException {
+      add((Integer) v);
+      return this;
+    }
+
+
+    default ItemWriter add(long v) throws IOException {
+      add((Long) v);
+      return this;
+    }
+
+
+    default ItemWriter add(float v) throws IOException {
+      add((Float) v);
+      return this;
+    }
+
+    default ItemWriter add(double v) throws IOException {
+      add((Double) v);
+      return this;
+    }
+
+    default ItemWriter add(boolean v) throws IOException {
+      add((Boolean) v);
+      return this;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e6dfb76c/solr/solrj/src/java/org/apache/solr/common/MapWriter.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/common/MapWriter.java b/solr/solrj/src/java/org/apache/solr/common/MapWriter.java
new file mode 100644
index 0000000..d5dad2a
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/common/MapWriter.java
@@ -0,0 +1,81 @@
+/*
+ * 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.common;
+
+
+import java.io.IOException;
+import java.util.Map;
+
+/**
+ * Use this class if the Map size is not known
+ */
+public interface MapWriter extends MapSerializable {
+
+  @Override
+  default Map toMap(Map<String, Object> map) {
+    try {
+      writeMap(new EntryWriter() {
+        @Override
+        public EntryWriter put(String k, Object v) throws IOException {
+          map.put(k, v);
+          return this;
+        }
+
+      });
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+    return map;
+  }
+
+  void writeMap(EntryWriter ew) throws IOException;
+
+  interface EntryWriter {
+    /**Writes a key value into the map
+     * @param k The key
+     * @param v The value can be any supported object
+     */
+    EntryWriter put(String k, Object v) throws IOException;
+
+    default EntryWriter put(String k, int v) throws IOException {
+      put(k, (Integer) v);
+      return this;
+    }
+
+
+    default EntryWriter put(String k, long v) throws IOException {
+      put(k, (Long) v);
+      return this;
+    }
+
+
+    default EntryWriter put(String k, float v) throws IOException {
+      put(k, (Float) v);
+      return this;
+    }
+
+    default EntryWriter put(String k, double v) throws IOException{
+      put(k, (Double) v);
+      return this;
+    }
+    default EntryWriter put(String k, boolean v) throws IOException{
+      put(k, (Boolean) v);
+      return this;
+    }
+  }
+}


[3/3] lucene-solr:branch_6x: SOLR-9717: Refactor '/export' to not hardcode the JSON output and to use an API

Posted by no...@apache.org.
SOLR-9717: Refactor '/export' to not hardcode the JSON output and to use an API


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

Branch: refs/heads/branch_6x
Commit: e6dfb76c98048d3dc3aec85b9aaeb2a6e9f1d6a8
Parents: 0f0526f
Author: Noble Paul <no...@apache.org>
Authored: Tue Nov 8 13:45:59 2016 +0530
Committer: Noble Paul <no...@apache.org>
Committed: Tue Nov 8 13:45:59 2016 +0530

----------------------------------------------------------------------
 solr/CHANGES.txt                                |    2 +
 .../src/java/org/apache/solr/core/SolrCore.java |   18 +-
 .../org/apache/solr/handler/ExportHandler.java  |   48 +
 .../org/apache/solr/handler/ExportWriter.java   | 1635 +++++++++++++++++
 .../solr/response/JSONResponseWriter.java       |   56 +
 .../solr/response/SortingResponseWriter.java    | 1718 ------------------
 .../solr/response/TextResponseWriter.java       |   30 +-
 solr/core/src/resources/ImplicitPlugins.json    |    6 +-
 .../test/org/apache/solr/core/SolrCoreTest.java |    2 +-
 .../apache/solr/response/JSONWriterTest.java    |    8 +-
 .../apache/solr/response/TestExportWriter.java  |  195 ++
 .../apache/solr/response/TestPushWriter.java    |   76 +
 .../response/TestSortingResponseWriter.java     |  190 --
 .../org/apache/solr/common/IteratorWriter.java  |   63 +
 .../java/org/apache/solr/common/MapWriter.java  |   81 +
 .../java/org/apache/solr/common/PushWriter.java |   42 +
 .../java/org/apache/solr/util/TestHarness.java  |   27 +-
 17 files changed, 2269 insertions(+), 1928 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e6dfb76c/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 82f54be..6e579eb 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -87,6 +87,8 @@ Other Changes
 * SOLR-9720: Refactor Responsewriters to remove dependencies on TupleStream,
   Tuple, Explanation (noble)
 
+* SOLR-9717: Refactor '/export' to not hardcode the JSON output and to use an API (noble)
+
 ==================  6.3.0 ==================
 
 Consult the LUCENE_CHANGES.txt file for additional, low level, changes in this release.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e6dfb76c/solr/core/src/java/org/apache/solr/core/SolrCore.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/SolrCore.java b/solr/core/src/java/org/apache/solr/core/SolrCore.java
index 1148b45..c9468e0 100644
--- a/solr/core/src/java/org/apache/solr/core/SolrCore.java
+++ b/solr/core/src/java/org/apache/solr/core/SolrCore.java
@@ -110,7 +110,6 @@ import org.apache.solr.response.RubyResponseWriter;
 import org.apache.solr.response.SchemaXmlResponseWriter;
 import org.apache.solr.response.SmileResponseWriter;
 import org.apache.solr.response.SolrQueryResponse;
-import org.apache.solr.response.SortingResponseWriter;
 import org.apache.solr.response.XMLResponseWriter;
 import org.apache.solr.response.transform.TransformerFactory;
 import org.apache.solr.rest.ManagedResourceStorage;
@@ -2314,7 +2313,6 @@ public final class SolrCore implements SolrInfoMBean, Closeable {
     m.put("raw", new RawResponseWriter());
     m.put(CommonParams.JAVABIN, new BinaryResponseWriter());
     m.put("csv", new CSVResponseWriter());
-    m.put("xsort", new SortingResponseWriter());
     m.put("schema.xml", new SchemaXmlResponseWriter());
     m.put("smile", new SmileResponseWriter());
     m.put(ReplicationHandler.FILE_STREAM, getFileStreamWriter());
@@ -2332,12 +2330,21 @@ public final class SolrCore implements SolrInfoMBean, Closeable {
       @Override
       public void write(OutputStream out, SolrQueryRequest req, SolrQueryResponse response) throws IOException {
         RawWriter rawWriter = (RawWriter) response.getValues().get(ReplicationHandler.FILE_STREAM);
-        if(rawWriter!=null) rawWriter.write(out);
+        if (rawWriter != null) {
+          rawWriter.write(out);
+          if (rawWriter instanceof Closeable) ((Closeable) rawWriter).close();
+        }
+
       }
 
       @Override
       public String getContentType(SolrQueryRequest request, SolrQueryResponse response) {
-        return BinaryResponseParser.BINARY_CONTENT_TYPE;
+        RawWriter rawWriter = (RawWriter) response.getValues().get(ReplicationHandler.FILE_STREAM);
+        if (rawWriter != null) {
+          return rawWriter.getContentType();
+        } else {
+          return BinaryResponseParser.BINARY_CONTENT_TYPE;
+        }
       }
     };
   }
@@ -2347,6 +2354,9 @@ public final class SolrCore implements SolrInfoMBean, Closeable {
   }
 
   public interface RawWriter {
+    default String getContentType() {
+      return BinaryResponseParser.BINARY_CONTENT_TYPE;
+    }
     void write(OutputStream os) throws IOException ;
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e6dfb76c/solr/core/src/java/org/apache/solr/handler/ExportHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/ExportHandler.java b/solr/core/src/java/org/apache/solr/handler/ExportHandler.java
new file mode 100644
index 0000000..9c75ef0
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/handler/ExportHandler.java
@@ -0,0 +1,48 @@
+/*
+ * 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.handler;
+
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.solr.common.params.CommonParams;
+import org.apache.solr.common.params.MapSolrParams;
+import org.apache.solr.common.params.SolrParams;
+import org.apache.solr.handler.component.SearchHandler;
+import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.response.SolrQueryResponse;
+
+import static org.apache.solr.common.params.CommonParams.JSON;
+
+public class ExportHandler extends SearchHandler {
+  @Override
+  public void handleRequestBody(SolrQueryRequest req, SolrQueryResponse rsp) throws Exception {
+    try {
+      super.handleRequestBody(req, rsp);
+    } catch (Exception e) {
+      rsp.setException(e);
+    }
+    String wt = req.getParams().get(CommonParams.WT, JSON);
+    if("xsort".equals(wt)) wt = JSON;
+    Map<String, String> map = new HashMap<>(1);
+    map.put(CommonParams.WT, ReplicationHandler.FILE_STREAM);
+    req.setParams(SolrParams.wrapDefaults(new MapSolrParams(map),req.getParams()));
+    rsp.add(ReplicationHandler.FILE_STREAM, new ExportWriter(req, rsp, wt));
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e6dfb76c/solr/core/src/java/org/apache/solr/handler/ExportWriter.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/ExportWriter.java b/solr/core/src/java/org/apache/solr/handler/ExportWriter.java
new file mode 100644
index 0000000..85180b5
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/handler/ExportWriter.java
@@ -0,0 +1,1635 @@
+/*
+ * 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.handler;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.io.OutputStreamWriter;
+import java.io.PrintWriter;
+import java.lang.invoke.MethodHandles;
+import java.nio.charset.StandardCharsets;
+import java.util.Date;
+import java.util.List;
+
+import org.apache.lucene.index.DocValues;
+import org.apache.lucene.index.IndexableField;
+import org.apache.lucene.index.LeafReader;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.MultiDocValues;
+import org.apache.lucene.index.NumericDocValues;
+import org.apache.lucene.index.SortedDocValues;
+import org.apache.lucene.index.SortedSetDocValues;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.Sort;
+import org.apache.lucene.search.SortField;
+import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.BitSetIterator;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.CharsRefBuilder;
+import org.apache.lucene.util.FixedBitSet;
+import org.apache.lucene.util.LongValues;
+import org.apache.solr.client.solrj.impl.BinaryResponseParser;
+import org.apache.solr.common.IteratorWriter;
+import org.apache.solr.common.MapWriter;
+import org.apache.solr.common.MapWriter.EntryWriter;
+import org.apache.solr.common.PushWriter;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.params.SolrParams;
+import org.apache.solr.core.SolrCore;
+import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.request.SolrRequestInfo;
+import org.apache.solr.response.JSONResponseWriter;
+import org.apache.solr.response.SolrQueryResponse;
+import org.apache.solr.schema.BoolField;
+import org.apache.solr.schema.FieldType;
+import org.apache.solr.schema.IndexSchema;
+import org.apache.solr.schema.SchemaField;
+import org.apache.solr.schema.StrField;
+import org.apache.solr.schema.TrieDateField;
+import org.apache.solr.schema.TrieDoubleField;
+import org.apache.solr.schema.TrieFloatField;
+import org.apache.solr.schema.TrieIntField;
+import org.apache.solr.schema.TrieLongField;
+import org.apache.solr.search.SolrIndexSearcher;
+import org.apache.solr.search.SortSpec;
+import org.apache.solr.search.SyntaxError;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static java.util.Collections.singletonList;
+import static java.util.Collections.singletonMap;
+import static org.apache.solr.common.util.Utils.makeMap;
+
+public class ExportWriter implements SolrCore.RawWriter, Closeable {
+  private static final Logger logger = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+  private OutputStreamWriter respWriter;
+  final SolrQueryRequest req;
+  final SolrQueryResponse res;
+  FieldWriter[] fieldWriters;
+  int totalHits = 0;
+  FixedBitSet[] sets = null;
+  PushWriter writer;
+  private String wt;
+
+
+  ExportWriter(SolrQueryRequest req, SolrQueryResponse res, String wt) {
+    this.req = req;
+    this.res = res;
+    this.wt = wt;
+
+  }
+
+  @Override
+  public String getContentType() {
+    if ("javabin".equals(wt)) {
+      return BinaryResponseParser.BINARY_CONTENT_TYPE;
+    } else return "json";
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (writer != null) writer.close();
+    if (respWriter != null) {
+      respWriter.flush();
+      respWriter.close();
+    }
+
+  }
+
+  protected void writeException(Exception e, PushWriter w, boolean log) throws IOException {
+    w.writeMap(mw -> {
+      mw.put("responseHeader", singletonMap("status", 400))
+          .put("response", makeMap(
+              "numFound", 0,
+              "docs", singletonList(singletonMap("EXCEPTION", e.getMessage()))));
+    });
+    if (log) {
+      SolrException.log(logger, e);
+    }
+  }
+
+  public void write(OutputStream os) throws IOException {
+    respWriter = new OutputStreamWriter(os, StandardCharsets.UTF_8);
+    writer = JSONResponseWriter.getPushWriter(respWriter, req, res);
+    Exception exception = res.getException();
+    if (exception != null) {
+      if (!(exception instanceof IgnoreException)) {
+        writeException(exception, writer, false);
+      }
+      return;
+    }
+
+    SolrRequestInfo info = SolrRequestInfo.getRequestInfo();
+    SortSpec sortSpec = info.getResponseBuilder().getSortSpec();
+
+    if(sortSpec == null) {
+      writeException((new IOException(new SyntaxError("No sort criteria was provided."))), writer, true);
+      return;
+    }
+
+    SolrIndexSearcher searcher = req.getSearcher();
+    Sort sort = searcher.weightSort(sortSpec.getSort());
+
+    if(sort == null) {
+      writeException((new IOException(new SyntaxError("No sort criteria was provided."))), writer, true);
+      return;
+    }
+
+    if(sort != null && sort.needsScores()) {
+      writeException((new IOException(new SyntaxError("Scoring is not currently supported with xsort."))), writer, true);
+      return;
+    }
+
+    // There is a bailout in SolrIndexSearcher.getDocListNC when there are _no_ docs in the index at all.
+    // if (lastDocRequested <= 0) {
+    // That causes the totalHits and export entries in the context to _not_ get set.
+    // The only time that really matters is when we search against an _empty_ set. That's too obscure
+    // a condition to handle as part of this patch, if someone wants to pursue it it can be reproduced with:
+    // ant test  -Dtestcase=StreamingTest -Dtests.method=testAllValidExportTypes -Dtests.seed=10F13879D0D1D6AD -Dtests.slow=true -Dtests.locale=es-PA -Dtests.timezone=America/Bahia_Banderas -Dtests.asserts=true -Dtests.file.encoding=ISO-8859-1
+    // You'll have to uncomment the if below to hit the null pointer exception.
+    // This is such an unusual case (i.e. an empty index) that catching this concdition here is probably OK.
+    // This came to light in the very artifical case of indexing a single doc to Cloud.
+    if (req.getContext().get("totalHits") != null) {
+      totalHits = ((Integer)req.getContext().get("totalHits")).intValue();
+      sets = (FixedBitSet[]) req.getContext().get("export");
+      if (sets == null) {
+        writeException((new IOException(new SyntaxError("xport RankQuery is required for xsort: rq={!xport}"))), writer, true);
+        return;
+      }
+    }
+    SolrParams params = req.getParams();
+    String fl = params.get("fl");
+
+    String[] fields = null;
+
+    if(fl == null) {
+      writeException((new IOException(new SyntaxError("export field list (fl) must be specified."))), writer, true);
+      return;
+    } else  {
+      fields = fl.split(",");
+
+      for(int i=0;i<fields.length; i++) {
+
+        fields[i] = fields[i].trim();
+
+        if(fields[i].equals("score")) {
+          writeException((new IOException(new SyntaxError("Scoring is not currently supported with xsort."))), writer, true);
+          return;
+        }
+      }
+    }
+
+    try {
+      fieldWriters = getFieldWriters(fields, req.getSearcher());
+    } catch (Exception e) {
+      writeException(e, writer, true);
+      return;
+    }
+
+    writer.writeMap(m -> {
+      m.put("responseHeader", singletonMap("status", 0));
+      m.put("response", (MapWriter) mw -> {
+        mw.put("numFound", totalHits);
+        mw.put("docs", (IteratorWriter) iw -> writeDocs(req, iw, sort));
+      });
+    });
+
+  }
+
+  protected void writeDocs(SolrQueryRequest req, IteratorWriter.ItemWriter writer, Sort sort) throws IOException {
+    //Write the data.
+    List<LeafReaderContext> leaves = req.getSearcher().getTopReaderContext().leaves();
+    SortDoc sortDoc = getSortDoc(req.getSearcher(), sort.getSort());
+    int count = 0;
+    int queueSize = 30000;
+    SortQueue queue = new SortQueue(queueSize, sortDoc);
+    SortDoc[] outDocs = new SortDoc[queueSize];
+
+    while(count < totalHits) {
+      //long begin = System.nanoTime();
+      queue.reset();
+      SortDoc top = queue.top();
+      for(int i=0; i<leaves.size(); i++) {
+        sortDoc.setNextReader(leaves.get(i));
+        DocIdSetIterator it = new BitSetIterator(sets[i], 0); // cost is not useful here
+        int docId = -1;
+        while((docId = it.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
+          sortDoc.setValues(docId);
+          if(top.lessThan(sortDoc)) {
+            top.setValues(sortDoc);
+            top = queue.updateTop();
+          }
+        }
+      }
+
+      int outDocsIndex = -1;
+
+      for(int i=0; i<queueSize; i++) {
+        SortDoc s = queue.pop();
+        if(s.docId > -1) {
+          outDocs[++outDocsIndex] = s;
+        }
+      }
+
+     //long end = System.nanoTime();
+
+      count += (outDocsIndex+1);
+
+      try {
+        for(int i=outDocsIndex; i>=0; --i) {
+          SortDoc s = outDocs[i];
+          writer.add((MapWriter) ew -> {
+            writeDoc(s, leaves, ew);
+            s.reset();
+          });
+        }
+      } catch(Throwable e) {
+        Throwable ex = e;
+        e.printStackTrace();
+        while(ex != null) {
+          String m = ex.getMessage();
+          if(m != null && m.contains("Broken pipe")) {
+            throw new IgnoreException();
+          }
+          ex = ex.getCause();
+        }
+
+        if(e instanceof IOException) {
+          throw ((IOException)e);
+        } else {
+          throw new IOException(e);
+        }
+      }
+    }
+  }
+
+  protected void writeDoc(SortDoc sortDoc,
+                          List<LeafReaderContext> leaves,
+                          EntryWriter ew) throws IOException {
+
+    int ord = sortDoc.ord;
+    FixedBitSet set = sets[ord];
+    set.clear(sortDoc.docId);
+    LeafReaderContext context = leaves.get(ord);
+    int fieldIndex = 0;
+    for (FieldWriter fieldWriter : fieldWriters) {
+      if (fieldWriter.write(sortDoc.docId, context.reader(), ew, fieldIndex)) {
+        ++fieldIndex;
+      }
+    }
+  }
+
+  protected FieldWriter[] getFieldWriters(String[] fields, SolrIndexSearcher searcher) throws IOException {
+    IndexSchema schema = searcher.getSchema();
+    FieldWriter[] writers = new FieldWriter[fields.length];
+    for(int i=0; i<fields.length; i++) {
+      String field = fields[i];
+      SchemaField schemaField = null;
+
+      try {
+        schemaField = schema.getField(field);
+      } catch (Exception e) {
+        throw new IOException(e);
+      }
+
+      if(!schemaField.hasDocValues()) {
+        throw new IOException(field+" must have DocValues to use this feature.");
+      }
+
+      boolean multiValued = schemaField.multiValued();
+      FieldType fieldType = schemaField.getType();
+      if (fieldType instanceof TrieIntField) {
+        if (multiValued) {
+          writers[i] = new MultiFieldWriter(field, fieldType, schemaField, true);
+        } else {
+          writers[i] = new IntFieldWriter(field);
+        }
+      } else if (fieldType instanceof TrieLongField) {
+        if (multiValued) {
+          writers[i] = new MultiFieldWriter(field, fieldType, schemaField, true);
+        } else {
+          writers[i] = new LongFieldWriter(field);
+        }
+      } else if (fieldType instanceof TrieFloatField) {
+        if (multiValued) {
+          writers[i] = new MultiFieldWriter(field, fieldType, schemaField, true);
+        } else {
+          writers[i] = new FloatFieldWriter(field);
+        }
+      } else if (fieldType instanceof TrieDoubleField) {
+        if (multiValued) {
+          writers[i] = new MultiFieldWriter(field, fieldType, schemaField, true);
+        } else {
+          writers[i] = new DoubleFieldWriter(field);
+        }
+      } else if (fieldType instanceof StrField) {
+        if (multiValued) {
+          writers[i] = new MultiFieldWriter(field, fieldType, schemaField, false);
+        } else {
+          writers[i] = new StringFieldWriter(field, fieldType);
+        }
+      } else if (fieldType instanceof TrieDateField) {
+        if (multiValued) {
+          writers[i] = new MultiFieldWriter(field, fieldType, schemaField, false);
+        } else {
+          writers[i] = new DateFieldWriter(field);
+        }
+      } else if (fieldType instanceof BoolField) {
+        if (multiValued) {
+          writers[i] = new MultiFieldWriter(field, fieldType, schemaField, true);
+        } else {
+          writers[i] = new BoolFieldWriter(field, fieldType);
+        }
+      } else {
+        throw new IOException("Export fields must either be one of the following types: int,float,long,double,string,date,boolean");
+      }
+    }
+    return writers;
+  }
+
+  private SortDoc getSortDoc(SolrIndexSearcher searcher, SortField[] sortFields) throws IOException {
+    SortValue[] sortValues = new SortValue[sortFields.length];
+    IndexSchema schema = searcher.getSchema();
+    for(int i=0; i<sortFields.length; ++i) {
+      SortField sf = sortFields[i];
+      String field = sf.getField();
+      boolean reverse = sf.getReverse();
+      SchemaField schemaField = schema.getField(field);
+      FieldType ft = schemaField.getType();
+
+      if(!schemaField.hasDocValues()) {
+        throw new IOException(field+" must have DocValues to use this feature.");
+      }
+
+      if(ft instanceof TrieIntField) {
+        if(reverse) {
+          sortValues[i] = new IntValue(field, new IntDesc());
+        } else {
+          sortValues[i] = new IntValue(field, new IntAsc());
+        }
+      } else if(ft instanceof TrieFloatField) {
+        if(reverse) {
+          sortValues[i] = new FloatValue(field, new FloatDesc());
+        } else {
+          sortValues[i] = new FloatValue(field, new FloatAsc());
+        }
+      } else if(ft instanceof TrieDoubleField) {
+        if(reverse) {
+          sortValues[i] = new DoubleValue(field, new DoubleDesc());
+        } else {
+          sortValues[i] = new DoubleValue(field, new DoubleAsc());
+        }
+      } else if(ft instanceof TrieLongField) {
+        if(reverse) {
+          sortValues[i] = new LongValue(field, new LongDesc());
+        } else {
+          sortValues[i] = new LongValue(field, new LongAsc());
+        }
+      } else if(ft instanceof StrField) {
+        LeafReader reader = searcher.getSlowAtomicReader();
+        SortedDocValues vals =  reader.getSortedDocValues(field);
+        if(reverse) {
+          sortValues[i] = new StringValue(vals, field, new IntDesc());
+        } else {
+          sortValues[i] = new StringValue(vals, field, new IntAsc());
+        }
+      } else if (ft instanceof TrieDateField) {
+        if (reverse) {
+          sortValues[i] = new LongValue(field, new LongDesc());
+        } else {
+          sortValues[i] = new LongValue(field, new LongAsc());
+        }
+      } else if (ft instanceof BoolField) {
+        // This is a bit of a hack, but since the boolean field stores ByteRefs, just like Strings
+        // _and_ since "F" happens to sort before "T" (thus false sorts "less" than true)
+        // we can just use the existing StringValue here.
+        LeafReader reader = searcher.getSlowAtomicReader();
+        SortedDocValues vals = reader.getSortedDocValues(field);
+        if (reverse) {
+          sortValues[i] = new StringValue(vals, field, new IntDesc());
+        } else {
+          sortValues[i] = new StringValue(vals, field, new IntAsc());
+        }
+      } else {
+        throw new IOException("Sort fields must be one of the following types: int,float,long,double,string,date,boolean");
+      }
+    }
+
+    if(sortValues.length == 1) {
+      return new SingleValueSortDoc(sortValues[0]);
+    } else if(sortValues.length == 2) {
+      return new DoubleValueSortDoc(sortValues[0], sortValues[1]);
+    } else if(sortValues.length == 3) {
+      return new TripleValueSortDoc(sortValues[0], sortValues[1], sortValues[2]);
+    } else if(sortValues.length == 4) {
+      return new QuadValueSortDoc(sortValues[0], sortValues[1], sortValues[2], sortValues[3]);
+    } else {
+      throw new IOException("A max of 4 sorts can be specified");
+    }
+  }
+
+  class SortQueue extends PriorityQueue<SortDoc> {
+
+    private SortDoc proto;
+    private Object[] cache;
+
+    public SortQueue(int len, SortDoc proto) {
+      super(len);
+      this.proto = proto;
+    }
+
+    protected boolean lessThan(SortDoc t1, SortDoc t2) {
+      return t1.lessThan(t2);
+    }
+
+    private void populate() {
+      Object[] heap = getHeapArray();
+      cache = new SortDoc[heap.length];
+      for (int i = 1; i < heap.length; i++) {
+        cache[i] = heap[i] = proto.copy();
+      }
+      size = maxSize;
+    }
+
+    private void reset() {
+      Object[] heap = getHeapArray();
+      if(cache != null) {
+        System.arraycopy(cache, 1, heap, 1, heap.length-1);
+        size = maxSize;
+      } else {
+        populate();
+      }
+    }
+  }
+
+  class SortDoc {
+
+    protected int docId = -1;
+    protected int ord = -1;
+    protected int docBase = -1;
+
+    private SortValue[] sortValues;
+
+    public SortDoc() {
+
+    }
+
+    public void setNextReader(LeafReaderContext context) throws IOException {
+      this.ord = context.ord;
+      for (SortValue value : sortValues) {
+        value.setNextReader(context);
+      }
+    }
+
+    public void reset() {
+      this.docId = -1;
+    }
+
+    public void setValues(int docId) throws IOException {
+      this.docId = docId;
+      for(SortValue sortValue : sortValues) {
+        sortValue.setCurrentValue(docId);
+      }
+    }
+
+    public void setValues(SortDoc sortDoc) throws IOException {
+      this.docId = sortDoc.docId;
+      this.ord = sortDoc.ord;
+      SortValue[] vals = sortDoc.sortValues;
+      for(int i=0; i<vals.length; i++) {
+        sortValues[i].setCurrentValue(vals[i]);
+      }
+    }
+
+    public SortDoc(SortValue[] sortValues) {
+      this.sortValues = sortValues;
+    }
+
+    public SortDoc copy() {
+      SortValue[] svs = new SortValue[sortValues.length];
+      for(int i=0; i<sortValues.length; i++) {
+        svs[i] = sortValues[i].copy();
+      }
+
+      return new SortDoc(svs);
+    }
+
+    public boolean lessThan(Object o) {
+      if(docId == -1) {
+        return true;
+      }
+
+      SortDoc sd = (SortDoc)o;
+      SortValue[] sortValues1 = sd.sortValues;
+      for(int i=0; i<sortValues.length; i++) {
+        int comp = sortValues[i].compareTo(sortValues1[i]);
+        if(comp < 0) {
+          return true;
+        } if(comp > 0) {
+          return false;
+        }
+      }
+      return docId+docBase < sd.docId+sd.docBase;
+    }
+
+    public String toString() {
+      return "";
+    }
+  }
+
+  class SingleValueSortDoc extends SortDoc {
+
+    protected SortValue value1;
+
+    public void setNextReader(LeafReaderContext context) throws IOException {
+      this.ord = context.ord;
+      value1.setNextReader(context);
+    }
+
+    public void reset() {
+      this.docId = -1;
+      this.value1.reset();
+    }
+
+    public void setValues(int docId) throws IOException {
+      this.docId = docId;
+      value1.setCurrentValue(docId);
+    }
+
+    public void setValues(SortDoc sortDoc) throws IOException {
+      this.docId = sortDoc.docId;
+      this.ord = sortDoc.ord;
+      value1.setCurrentValue(((SingleValueSortDoc)sortDoc).value1);
+    }
+
+    public SingleValueSortDoc(SortValue value1) {
+      super();
+      this.value1 = value1;
+    }
+
+    public SortDoc copy() {
+      return new SingleValueSortDoc(value1.copy());
+    }
+
+    public boolean lessThan(Object o) {
+      SingleValueSortDoc sd = (SingleValueSortDoc)o;
+      int comp = value1.compareTo(sd.value1);
+      if(comp == -1) {
+        return true;
+      } else if (comp == 1) {
+        return false;
+      } else {
+        return docId+docBase > sd.docId+sd.docBase;
+      }
+    }
+
+    public int compareTo(Object o) {
+      SingleValueSortDoc sd = (SingleValueSortDoc)o;
+      return value1.compareTo(sd.value1);
+    }
+
+    public String toString() {
+      return docId+":"+value1.toString();
+    }
+  }
+
+  class DoubleValueSortDoc extends SingleValueSortDoc {
+
+    protected SortValue value2;
+
+    public void setNextReader(LeafReaderContext context) throws IOException {
+      this.ord = context.ord;
+      value1.setNextReader(context);
+      value2.setNextReader(context);
+    }
+
+    public void reset() {
+      this.docId = -1;
+      value1.reset();
+      value2.reset();
+    }
+
+    public void setValues(int docId) throws IOException {
+      this.docId = docId;
+      value1.setCurrentValue(docId);
+      value2.setCurrentValue(docId);
+    }
+
+    public void setValues(SortDoc sortDoc) throws IOException {
+      this.docId = sortDoc.docId;
+      this.ord = sortDoc.ord;
+      value1.setCurrentValue(((DoubleValueSortDoc)sortDoc).value1);
+      value2.setCurrentValue(((DoubleValueSortDoc)sortDoc).value2);
+    }
+
+    public DoubleValueSortDoc(SortValue value1, SortValue value2) {
+      super(value1);
+      this.value2 = value2;
+    }
+
+    public SortDoc copy() {
+      return new DoubleValueSortDoc(value1.copy(), value2.copy());
+    }
+
+    public boolean lessThan(Object o) {
+      DoubleValueSortDoc sd = (DoubleValueSortDoc)o;
+      int comp = value1.compareTo(sd.value1);
+      if(comp == -1) {
+        return true;
+      } else if (comp == 1) {
+        return false;
+      } else {
+        comp = value2.compareTo(sd.value2);
+        if(comp == -1) {
+          return true;
+        } else if (comp == 1) {
+          return false;
+        } else {
+          return docId+docBase > sd.docId+sd.docBase;
+        }
+      }
+    }
+
+    public int compareTo(Object o) {
+      DoubleValueSortDoc sd = (DoubleValueSortDoc)o;
+      int comp = value1.compareTo(sd.value1);
+      if(comp == 0) {
+        return value2.compareTo(sd.value2);
+      } else {
+        return comp;
+      }
+    }
+  }
+
+  class TripleValueSortDoc extends DoubleValueSortDoc {
+
+    protected SortValue value3;
+
+    public void setNextReader(LeafReaderContext context) throws IOException {
+      this.ord = context.ord;
+      value1.setNextReader(context);
+      value2.setNextReader(context);
+      value3.setNextReader(context);
+    }
+
+    public void reset() {
+      this.docId = -1;
+      value1.reset();
+      value2.reset();
+      value3.reset();
+    }
+
+    public void setValues(int docId) throws IOException {
+      this.docId = docId;
+      value1.setCurrentValue(docId);
+      value2.setCurrentValue(docId);
+      value3.setCurrentValue(docId);
+    }
+
+    public void setValues(SortDoc sortDoc) throws IOException {
+      this.docId = sortDoc.docId;
+      this.ord = sortDoc.ord;
+      value1.setCurrentValue(((TripleValueSortDoc)sortDoc).value1);
+      value2.setCurrentValue(((TripleValueSortDoc)sortDoc).value2);
+      value3.setCurrentValue(((TripleValueSortDoc)sortDoc).value3);
+    }
+
+    public TripleValueSortDoc(SortValue value1, SortValue value2, SortValue value3) {
+      super(value1, value2);
+      this.value3 = value3;
+    }
+
+    public SortDoc copy() {
+      return new TripleValueSortDoc(value1.copy(), value2.copy(), value3.copy());
+    }
+
+    public boolean lessThan(Object o) {
+
+      TripleValueSortDoc sd = (TripleValueSortDoc)o;
+      int comp = value1.compareTo(sd.value1);
+      if(comp == -1) {
+        return true;
+      } else if (comp == 1) {
+        return false;
+      } else {
+        comp = value2.compareTo(sd.value2);
+        if(comp == -1) {
+          return true;
+        } else if (comp == 1) {
+          return false;
+        } else {
+          comp = value3.compareTo(sd.value3);
+          if(comp == -1) {
+            return true;
+          } else if (comp == 1) {
+            return false;
+          } else {
+            return docId+docBase > sd.docId+sd.docBase;
+          }
+        }
+      }
+    }
+
+    public int compareTo(Object o) {
+
+      TripleValueSortDoc sd = (TripleValueSortDoc)o;
+      int comp = value1.compareTo(sd.value1);
+      if(comp == 0) {
+        comp = value2.compareTo(sd.value2);
+        if(comp == 0) {
+          return value3.compareTo(sd.value3);
+        } else {
+          return comp;
+        }
+      } else {
+        return comp;
+      }
+    }
+  }
+
+  class QuadValueSortDoc extends TripleValueSortDoc {
+
+    protected SortValue value4;
+
+    public void setNextReader(LeafReaderContext context) throws IOException {
+      this.ord = context.ord;
+      value1.setNextReader(context);
+      value2.setNextReader(context);
+      value3.setNextReader(context);
+      value4.setNextReader(context);
+    }
+
+    public void reset() {
+      this.docId = -1;
+      value1.reset();
+      value2.reset();
+      value3.reset();
+      value4.reset();
+    }
+
+    public void setValues(int docId) throws IOException {
+      this.docId = docId;
+      value1.setCurrentValue(docId);
+      value2.setCurrentValue(docId);
+      value3.setCurrentValue(docId);
+      value4.setCurrentValue(docId);
+    }
+
+    public void setValues(SortDoc sortDoc) throws IOException {
+      this.docId = sortDoc.docId;
+      this.ord = sortDoc.ord;
+      value1.setCurrentValue(((QuadValueSortDoc)sortDoc).value1);
+      value2.setCurrentValue(((QuadValueSortDoc)sortDoc).value2);
+      value3.setCurrentValue(((QuadValueSortDoc)sortDoc).value3);
+      value4.setCurrentValue(((QuadValueSortDoc)sortDoc).value4);
+    }
+
+    public QuadValueSortDoc(SortValue value1, SortValue value2, SortValue value3, SortValue value4) {
+      super(value1, value2, value3);
+      this.value4 = value4;
+    }
+
+    public SortDoc copy() {
+      return new QuadValueSortDoc(value1.copy(), value2.copy(), value3.copy(), value4.copy());
+    }
+
+    public boolean lessThan(Object o) {
+
+      QuadValueSortDoc sd = (QuadValueSortDoc)o;
+      int comp = value1.compareTo(sd.value1);
+      if(comp == -1) {
+        return true;
+      } else if (comp == 1) {
+        return false;
+      } else {
+        comp = value2.compareTo(sd.value2);
+        if(comp == -1) {
+          return true;
+        } else if (comp == 1) {
+          return false;
+        } else {
+          comp = value3.compareTo(sd.value3);
+          if(comp == -1) {
+            return true;
+          } else if (comp == 1) {
+            return false;
+          } else {
+            comp = value4.compareTo(sd.value4);
+            if(comp == -1) {
+              return true;
+            } else if (comp == 1) {
+              return false;
+            } else {
+              return docId+docBase > sd.docId+sd.docBase;
+            }
+          }
+        }
+      }
+    }
+
+    public int compareTo(Object o) {
+      QuadValueSortDoc sd = (QuadValueSortDoc)o;
+      int comp = value1.compareTo(sd.value1);
+      if(comp == 0) {
+        comp = value2.compareTo(sd.value2);
+        if(comp == 0) {
+          comp = value3.compareTo(sd.value3);
+          if(comp == 0) {
+            return value4.compareTo(sd.value4);
+          } else {
+            return comp;
+          }
+        } else {
+          return comp;
+        }
+      } else {
+        return comp;
+      }
+    }
+  }
+
+  public interface SortValue extends Comparable<SortValue> {
+    public void setCurrentValue(int docId) throws IOException;
+    public void setNextReader(LeafReaderContext context) throws IOException;
+    public void setCurrentValue(SortValue value);
+    public void reset();
+    public SortValue copy();
+  }
+
+  class IntValue implements SortValue {
+
+    protected NumericDocValues vals;
+    protected String field;
+    protected int currentValue;
+    protected IntComp comp;
+
+    public IntValue copy() {
+      return new IntValue(field, comp);
+    }
+
+    public IntValue(String field, IntComp comp) {
+      this.field = field;
+      this.comp = comp;
+      this.currentValue = comp.resetValue();
+    }
+
+    public void setNextReader(LeafReaderContext context) throws IOException {
+      this.vals = DocValues.getNumeric(context.reader(), field);
+    }
+
+    public void setCurrentValue(int docId) {
+      currentValue = (int) vals.get(docId);
+    }
+
+    public int compareTo(SortValue o) {
+      IntValue iv = (IntValue)o;
+      return comp.compare(currentValue, iv.currentValue);
+    }
+
+    public void setCurrentValue (SortValue value) {
+      currentValue = ((IntValue)value).currentValue;
+    }
+
+    public void reset() {
+      currentValue = comp.resetValue();
+    }
+  }
+
+  public interface IntComp {
+    public int compare(int a, int b);
+    public int resetValue();
+  }
+
+  class IntDesc implements IntComp {
+
+    public int resetValue() {
+      return Integer.MIN_VALUE;
+    }
+
+    public int compare(int a, int b) {
+      if(a < b) {
+        return -1;
+      } else if (a > b) {
+        return 1;
+      } else {
+        return 0;
+      }
+    }
+  }
+
+  class IntAsc implements IntComp {
+
+    public int resetValue() {
+      return Integer.MAX_VALUE;
+    }
+
+    public int compare(int a, int b) {
+      if(a < b) {
+        return 1;
+      } else if (a > b) {
+        return -1;
+      } else {
+        return 0;
+      }
+    }
+  }
+
+  class LongValue implements SortValue {
+
+    protected NumericDocValues vals;
+    protected String field;
+    protected long currentValue;
+    protected LongComp comp;
+
+    public LongValue(String field, LongComp comp) {
+      this.field = field;
+      this.comp = comp;
+      this.currentValue = comp.resetValue();
+    }
+
+    public LongValue copy() {
+      return new LongValue(field, comp);
+    }
+
+    public void setNextReader(LeafReaderContext context) throws IOException {
+      this.vals = DocValues.getNumeric(context.reader(), field);
+    }
+
+    public void setCurrentValue(int docId) {
+      currentValue = vals.get(docId);
+    }
+
+    public void setCurrentValue(SortValue sv) {
+      LongValue lv = (LongValue)sv;
+      this.currentValue = lv.currentValue;
+    }
+
+    public int compareTo(SortValue o) {
+      LongValue l = (LongValue)o;
+      return comp.compare(currentValue, l.currentValue);
+    }
+
+    public void reset() {
+      this.currentValue = comp.resetValue();
+    }
+  }
+
+  interface LongComp {
+    public int compare(long a, long b);
+    public long resetValue();
+  }
+
+  class LongDesc implements LongComp {
+
+    public long resetValue() {
+      return Long.MIN_VALUE;
+    }
+
+    public int compare(long a, long b) {
+      if(a < b) {
+        return -1;
+      } else if (a > b) {
+        return 1;
+      } else {
+        return 0;
+      }
+    }
+  }
+
+  class LongAsc implements LongComp {
+
+    public long resetValue() {
+      return Long.MAX_VALUE;
+    }
+
+    public int compare(long a, long b) {
+      if(a < b) {
+        return 1;
+      } else if (a > b) {
+        return -1;
+      } else {
+        return 0;
+      }
+    }
+  }
+
+  class FloatValue implements SortValue {
+
+    protected NumericDocValues vals;
+    protected String field;
+    protected float currentValue;
+    protected FloatComp comp;
+
+    public FloatValue(String field, FloatComp comp) {
+      this.field = field;
+      this.comp = comp;
+      this.currentValue = comp.resetValue();
+    }
+
+    public FloatValue copy() {
+      return new FloatValue(field, comp);
+    }
+
+    public void setNextReader(LeafReaderContext context) throws IOException {
+      this.vals = DocValues.getNumeric(context.reader(), field);
+    }
+
+    public void setCurrentValue(int docId) {
+      currentValue = Float.intBitsToFloat((int) vals.get(docId));
+    }
+
+    public void setCurrentValue(SortValue sv) {
+      FloatValue fv = (FloatValue)sv;
+      this.currentValue = fv.currentValue;
+    }
+
+    public void reset() {
+      this.currentValue = comp.resetValue();
+    }
+
+    public int compareTo(SortValue o) {
+      FloatValue fv = (FloatValue)o;
+      return comp.compare(currentValue, fv.currentValue);
+    }
+  }
+
+  interface FloatComp {
+    public int compare(float a, float b);
+    public float resetValue();
+  }
+
+  public class FloatDesc implements FloatComp {
+    public float resetValue() {
+      return -Float.MAX_VALUE;
+    }
+
+    public int compare(float a, float b) {
+      if(a < b) {
+        return -1;
+      } else if (a > b) {
+        return 1;
+      } else {
+        return 0;
+      }
+    }
+  }
+
+  public class FloatAsc implements FloatComp {
+    public float resetValue() {
+      return Float.MAX_VALUE;
+    }
+
+    public int compare(float a, float b) {
+      if(a < b) {
+        return 1;
+      } else if (a > b) {
+        return -1;
+      } else {
+        return 0;
+      }
+    }
+  }
+
+  class DoubleValue implements SortValue {
+
+    protected NumericDocValues vals;
+    protected String field;
+    protected double currentValue;
+    protected DoubleComp comp;
+
+    public DoubleValue(String field, DoubleComp comp) {
+      this.field = field;
+      this.comp = comp;
+      this.currentValue = comp.resetValue();
+    }
+
+    public DoubleValue copy() {
+      return new DoubleValue(field, comp);
+    }
+
+    public void setNextReader(LeafReaderContext context) throws IOException {
+      this.vals = DocValues.getNumeric(context.reader(), field);
+    }
+
+    public void setCurrentValue(int docId) {
+      currentValue = Double.longBitsToDouble(vals.get(docId));
+    }
+
+    public void setCurrentValue(SortValue sv) {
+      DoubleValue dv = (DoubleValue)sv;
+      this.currentValue = dv.currentValue;
+    }
+
+    public void reset() {
+      this.currentValue = comp.resetValue();
+    }
+
+    public int compareTo(SortValue o) {
+      DoubleValue dv = (DoubleValue)o;
+      return comp.compare(currentValue, dv.currentValue);
+    }
+  }
+
+  interface DoubleComp {
+    public int compare(double a, double b);
+    public double resetValue();
+  }
+
+  public class DoubleDesc implements DoubleComp {
+    public double resetValue() {
+      return -Double.MAX_VALUE;
+    }
+
+    public int compare(double a, double b) {
+      if(a < b) {
+        return -1;
+      } else if (a > b) {
+        return 1;
+      } else {
+        return 0;
+      }
+    }
+  }
+
+  public class DoubleAsc implements DoubleComp {
+    public double resetValue() {
+      return Double.MAX_VALUE;
+    }
+
+    public int compare(double a, double b) {
+      if(a < b) {
+        return 1;
+      } else if (a > b) {
+        return -1;
+      } else {
+        return 0;
+      }
+    }
+  }
+
+  class StringValue implements SortValue {
+
+    protected SortedDocValues vals;
+    protected SortedDocValues segmentVals[];
+
+    protected MultiDocValues.OrdinalMap ordinalMap;
+    protected LongValues globalOrds;
+    protected SortedDocValues currentVals;
+
+    protected String field;
+    protected int segment;
+    protected int currentOrd;
+    protected IntComp comp;
+
+    public StringValue(SortedDocValues vals, String field, IntComp comp)  {
+      this.vals = vals;
+      if(vals instanceof  MultiDocValues.MultiSortedDocValues) {
+        this.segmentVals = ((MultiDocValues.MultiSortedDocValues) vals).values;
+        this.ordinalMap = ((MultiDocValues.MultiSortedDocValues) vals).mapping;
+      }
+      this.field = field;
+      this.comp = comp;
+      this.currentOrd = comp.resetValue();
+    }
+
+    public StringValue copy() {
+      return new StringValue(vals, field, comp);
+    }
+
+    public void setCurrentValue(int docId) {
+      int ord = currentVals.getOrd(docId);
+
+      if(ord < 0) {
+        currentOrd = -1;
+      } else {
+        if(globalOrds != null) {
+          currentOrd = (int)globalOrds.get(ord);
+        } else {
+          currentOrd = ord;
+        }
+      }
+    }
+
+    public void setCurrentValue(SortValue sv) {
+      StringValue v = (StringValue)sv;
+      this.currentOrd = v.currentOrd;
+    }
+
+    public void setNextReader(LeafReaderContext context) {
+      segment = context.ord;
+      if(ordinalMap != null) {
+        globalOrds = ordinalMap.getGlobalOrds(segment);
+        currentVals = segmentVals[segment];
+      } else {
+        currentVals = vals;
+      }
+    }
+
+    public void reset() {
+      this.currentOrd = comp.resetValue();
+    }
+
+    public int compareTo(SortValue o) {
+      StringValue sv = (StringValue)o;
+      return comp.compare(currentOrd, sv.currentOrd);
+    }
+
+    public String toString() {
+      return Integer.toString(this.currentOrd);
+    }
+  }
+
+  protected abstract class FieldWriter {
+    public abstract boolean write(int docId, LeafReader reader, EntryWriter out, int fieldIndex) throws IOException;
+  }
+
+  class IntFieldWriter extends FieldWriter {
+    private String field;
+
+    public IntFieldWriter(String field) {
+      this.field = field;
+    }
+
+    public boolean write(int docId, LeafReader reader, EntryWriter ew, int fieldIndex) throws IOException {
+      NumericDocValues vals = DocValues.getNumeric(reader, this.field);
+      int val = (int) vals.get(docId);
+      ew.put(this.field, val);
+      return true;
+    }
+  }
+
+  class MultiFieldWriter extends FieldWriter {
+    private String field;
+    private FieldType fieldType;
+    private SchemaField schemaField;
+    private boolean numeric;
+    private CharsRefBuilder cref = new CharsRefBuilder();
+
+    public MultiFieldWriter(String field, FieldType fieldType, SchemaField schemaField, boolean numeric) {
+      this.field = field;
+      this.fieldType = fieldType;
+      this.schemaField = schemaField;
+      this.numeric = numeric;
+    }
+
+    public boolean write(int docId, LeafReader reader, EntryWriter out, int fieldIndex) throws IOException {
+      SortedSetDocValues vals = DocValues.getSortedSet(reader, this.field);
+      vals.setDocument(docId);
+      long first = vals.nextOrd();
+      if (first == SortedSetDocValues.NO_MORE_ORDS) return false;
+      out.put(this.field,
+          (IteratorWriter) w -> {
+            boolean isFirst = true;
+            for (; ; ) {
+              long o;
+              if (isFirst) {
+                isFirst = false;
+                o = first;
+              } else o = vals.nextOrd();
+              if (o == SortedSetDocValues.NO_MORE_ORDS) break;
+              BytesRef ref = vals.lookupOrd(o);
+              fieldType.indexedToReadable(ref, cref);
+              IndexableField f = fieldType.createField(schemaField, cref.toString(), 1.0f);
+              if (f == null) w.add(cref.toString());
+              else w.add(fieldType.toObject(f));
+            }
+          });
+
+      return true;
+
+    }
+  }
+
+  class LongFieldWriter extends FieldWriter {
+    private String field;
+
+    public LongFieldWriter(String field) {
+      this.field = field;
+    }
+
+    public boolean write(int docId, LeafReader reader, EntryWriter ew, int fieldIndex) throws IOException {
+      NumericDocValues vals = DocValues.getNumeric(reader, this.field);
+      long val = vals.get(docId);
+      ew.put(field, val);
+
+      return true;
+    }
+  }
+
+  class DateFieldWriter extends FieldWriter {
+    private String field;
+
+    public DateFieldWriter(String field) {
+      this.field = field;
+    }
+
+    public boolean write(int docId, LeafReader reader, EntryWriter ew, int fieldIndex) throws IOException {
+      NumericDocValues vals = DocValues.getNumeric(reader, this.field);
+      long val = vals.get(docId);
+      ew.put(this.field, new Date(val));
+      return true;
+    }
+  }
+
+  class BoolFieldWriter extends FieldWriter {
+    private String field;
+    private FieldType fieldType;
+    private CharsRefBuilder cref = new CharsRefBuilder();
+
+    public BoolFieldWriter(String field, FieldType fieldType) {
+      this.field = field;
+      this.fieldType = fieldType;
+    }
+
+    public boolean write(int docId, LeafReader reader, EntryWriter ew, int fieldIndex) throws IOException {
+      SortedDocValues vals = DocValues.getSorted(reader, this.field);
+      int ord = vals.getOrd(docId);
+      if (ord == -1) {
+        return false;
+      }
+
+      BytesRef ref = vals.lookupOrd(ord);
+      fieldType.indexedToReadable(ref, cref);
+      ew.put(this.field, "true".equals(cref.toString()));
+      return true;
+    }
+  }
+
+  class FloatFieldWriter extends FieldWriter {
+    private String field;
+
+    public FloatFieldWriter(String field) {
+      this.field = field;
+    }
+
+    public boolean write(int docId, LeafReader reader, EntryWriter ew, int fieldIndex) throws IOException {
+      NumericDocValues vals = DocValues.getNumeric(reader, this.field);
+      int val = (int) vals.get(docId);
+      ew.put(this.field, Float.intBitsToFloat(val));
+      return true;
+    }
+  }
+
+  class DoubleFieldWriter extends FieldWriter {
+    private String field;
+
+    public DoubleFieldWriter(String field) {
+      this.field = field;
+    }
+
+    public boolean write(int docId, LeafReader reader, EntryWriter ew, int fieldIndex) throws IOException {
+      NumericDocValues vals = DocValues.getNumeric(reader, this.field);
+      long val = vals.get(docId);
+      ew.put(this.field, Double.longBitsToDouble(val));
+      return true;
+    }
+  }
+
+  class StringFieldWriter extends FieldWriter {
+    private String field;
+    private FieldType fieldType;
+    private CharsRefBuilder cref = new CharsRefBuilder();
+
+    public StringFieldWriter(String field, FieldType fieldType) {
+      this.field = field;
+      this.fieldType = fieldType;
+    }
+
+    public boolean write(int docId, LeafReader reader, EntryWriter ew, int fieldIndex) throws IOException {
+      SortedDocValues vals = DocValues.getSorted(reader, this.field);
+      int ord = vals.getOrd(docId);
+      if (ord == -1) {
+        return false;
+      }
+      BytesRef ref = vals.lookupOrd(ord);
+      fieldType.indexedToReadable(ref, cref);
+      ew.put(this.field, cref.toString());
+      return true;
+    }
+  }
+
+  public abstract class PriorityQueue<T> {
+    protected int size = 0;
+    protected final int maxSize;
+    private final T[] heap;
+
+    public PriorityQueue(int maxSize) {
+      this(maxSize, true);
+    }
+
+    public PriorityQueue(int maxSize, boolean prepopulate) {
+      final int heapSize;
+      if (0 == maxSize) {
+        // We allocate 1 extra to avoid if statement in top()
+        heapSize = 2;
+      } else {
+        if (maxSize > ArrayUtil.MAX_ARRAY_LENGTH) {
+          // Don't wrap heapSize to -1, in this case, which
+          // causes a confusing NegativeArraySizeException.
+          // Note that very likely this will simply then hit
+          // an OOME, but at least that's more indicative to
+          // caller that this values is too big.  We don't +1
+          // in this case, but it's very unlikely in practice
+          // one will actually insert this many objects into
+          // the PQ:
+          // Throw exception to prevent confusing OOME:
+          throw new IllegalArgumentException("maxSize must be <= " + ArrayUtil.MAX_ARRAY_LENGTH + "; got: " + maxSize);
+        } else {
+          // NOTE: we add +1 because all access to heap is
+          // 1-based not 0-based.  heap[0] is unused.
+          heapSize = maxSize + 1;
+        }
+      }
+      // T is unbounded type, so this unchecked cast works always:
+      @SuppressWarnings("unchecked") final T[] h = (T[]) new Object[heapSize];
+      this.heap = h;
+      this.maxSize = maxSize;
+
+      if (prepopulate) {
+        // If sentinel objects are supported, populate the queue with them
+        T sentinel = getSentinelObject();
+        if (sentinel != null) {
+          heap[1] = sentinel;
+          for (int i = 2; i < heap.length; i++) {
+            heap[i] = getSentinelObject();
+          }
+          size = maxSize;
+        }
+      }
+    }
+
+    /** Determines the ordering of objects in this priority queue.  Subclasses
+     *  must define this one method.
+     *  @return <code>true</code> iff parameter <tt>a</tt> is less than parameter <tt>b</tt>.
+     */
+    protected abstract boolean lessThan(T a, T b);
+
+
+    protected T getSentinelObject() {
+      return null;
+    }
+
+    /**
+     * Adds an Object to a PriorityQueue in log(size) time. If one tries to add
+     * more objects than maxSize from initialize an
+     *
+     * @return the new 'top' element in the queue.
+     */
+    public final T add(T element) {
+      size++;
+      heap[size] = element;
+      upHeap();
+      return heap[1];
+    }
+
+    /**
+     * Adds an Object to a PriorityQueue in log(size) time.
+     * It returns the object (if any) that was
+     * dropped off the heap because it was full. This can be
+     * the given parameter (in case it is smaller than the
+     * full heap's minimum, and couldn't be added), or another
+     * object that was previously the smallest value in the
+     * heap and now has been replaced by a larger one, or null
+     * if the queue wasn't yet full with maxSize elements.
+     */
+    public T insertWithOverflow(T element) {
+      if (size < maxSize) {
+        add(element);
+        return null;
+      } else if (size > 0 && !lessThan(element, heap[1])) {
+        T ret = heap[1];
+        heap[1] = element;
+        updateTop();
+        return ret;
+      } else {
+        return element;
+      }
+    }
+
+    /** Returns the least element of the PriorityQueue in constant time. */
+    public final T top() {
+      // We don't need to check size here: if maxSize is 0,
+      // then heap is length 2 array with both entries null.
+      // If size is 0 then heap[1] is already null.
+      return heap[1];
+    }
+
+    /** Removes and returns the least element of the PriorityQueue in log(size)
+     time. */
+    public final T pop() {
+      if (size > 0) {
+        T result = heap[1];       // save first value
+        heap[1] = heap[size];     // move last to first
+        heap[size] = null;        // permit GC of objects
+        size--;
+        downHeap();               // adjust heap
+        return result;
+      } else {
+        return null;
+      }
+    }
+
+    /**
+     * Should be called when the Object at top changes values. Still log(n) worst
+     * case, but it's at least twice as fast to
+     *
+     * <pre class="prettyprint">
+     * pq.top().change();
+     * pq.updateTop();
+     * </pre>
+     *
+     * instead of
+     *
+     * <pre class="prettyprint">
+     * o = pq.pop();
+     * o.change();
+     * pq.push(o);
+     * </pre>
+     *
+     * @return the new 'top' element.
+     */
+    public final T updateTop() {
+      downHeap();
+      return heap[1];
+    }
+
+    /** Returns the number of elements currently stored in the PriorityQueue. */
+    public final int size() {
+      return size;
+    }
+
+    /** Removes all entries from the PriorityQueue. */
+    public final void clear() {
+      for (int i = 0; i <= size; i++) {
+        heap[i] = null;
+      }
+      size = 0;
+    }
+
+    private final void upHeap() {
+      int i = size;
+      T node = heap[i];          // save bottom node
+      int j = i >>> 1;
+      while (j > 0 && lessThan(node, heap[j])) {
+        heap[i] = heap[j];       // shift parents down
+        i = j;
+        j = j >>> 1;
+      }
+      heap[i] = node;            // install saved node
+    }
+
+    private final void downHeap() {
+      int i = 1;
+      T node = heap[i];          // save top node
+      int j = i << 1;            // find smaller child
+      int k = j + 1;
+      if (k <= size && lessThan(heap[k], heap[j])) {
+        j = k;
+      }
+      while (j <= size && lessThan(heap[j], node)) {
+        heap[i] = heap[j];       // shift up child
+        i = j;
+        j = i << 1;
+        k = j + 1;
+        if (k <= size && lessThan(heap[k], heap[j])) {
+          j = k;
+        }
+      }
+      heap[i] = node;            // install saved node
+    }
+
+    /** This method returns the internal heap array as Object[].
+     * @lucene.internal
+     */
+    public final Object[] getHeapArray() {
+      return (Object[]) heap;
+    }
+  }
+
+  public class IgnoreException extends IOException {
+    public void printStackTrace(PrintWriter pw) {
+      pw.print("Early Client Disconnect");
+    }
+
+    public String getMessage() {
+      return "Early Client Disconnect";
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e6dfb76c/solr/core/src/java/org/apache/solr/response/JSONResponseWriter.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/response/JSONResponseWriter.java b/solr/core/src/java/org/apache/solr/response/JSONResponseWriter.java
index 5149560..0fcf720 100644
--- a/solr/core/src/java/org/apache/solr/response/JSONResponseWriter.java
+++ b/solr/core/src/java/org/apache/solr/response/JSONResponseWriter.java
@@ -24,7 +24,11 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
+import org.apache.solr.common.IteratorWriter;
+import org.apache.solr.common.MapWriter.EntryWriter;
+import org.apache.solr.common.PushWriter;
 import org.apache.solr.common.SolrDocument;
+import org.apache.solr.common.MapWriter;
 import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.SimpleOrderedMap;
@@ -74,6 +78,11 @@ public class JSONResponseWriter implements QueryResponseWriter {
   public String getContentType(SolrQueryRequest request, SolrQueryResponse response) {
     return contentType;
   }
+
+  public static PushWriter getPushWriter(Writer writer, SolrQueryRequest req, SolrQueryResponse rsp) {
+    return new JSONWriter(writer, req, rsp);
+  }
+
 }
 
 class JSONWriter extends TextResponseWriter {
@@ -507,6 +516,53 @@ class JSONWriter extends TextResponseWriter {
     }
   }
 
+  @Override
+  public void writeIterator(IteratorWriter val) throws IOException {
+    writeArrayOpener(-1);
+    incLevel();
+    val.writeIter(new IteratorWriter.ItemWriter() {
+      boolean first = true;
+
+      @Override
+      public IteratorWriter.ItemWriter add(Object o) throws IOException {
+        if (!first) {
+          JSONWriter.this.indent();
+          JSONWriter.this.writeArraySeparator();
+        }
+        JSONWriter.this.writeVal(null, o);
+        first = false;
+        return this;
+      }
+    });
+    decLevel();
+    writeArrayCloser();
+  }
+
+  @Override
+  public void writeMap(MapWriter val)
+      throws IOException {
+    writeMapOpener(-1);
+    incLevel();
+
+    val.writeMap(new EntryWriter() {
+      boolean isFirst = true;
+
+      @Override
+      public EntryWriter put(String k, Object v) throws IOException {
+        if (isFirst) {
+          isFirst = false;
+        } else {
+          JSONWriter.this.writeMapSeparator();
+        }
+        if (doIndent) JSONWriter.this.indent();
+        JSONWriter.this.writeKey(k, true);
+        JSONWriter.this.writeVal(k, v);
+        return this;
+      }
+    });
+    decLevel();
+    writeMapCloser();
+  }
 
   @Override
   public void writeMap(String name, Map val, boolean excludeOuter, boolean isFirstVal) throws IOException {