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 11:08:05 UTC

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

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ef074a61/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 56c4f27..0000000
--- a/solr/core/src/java/org/apache/solr/response/SortingResponseWriter.java
+++ /dev/null
@@ -1,1805 +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.ArrayList;
-import java.util.Date;
-import java.util.List;
-
-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;
-    private int lastDocID;
-
-    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);
-      lastDocID = 0;
-    }
-
-    public void setCurrentValue(int docId) throws IOException {
-      if (docId < lastDocID) {
-        throw new AssertionError("docs were sent out-of-order: lastDocID=" + lastDocID + " vs doc=" + docId);
-      }
-      lastDocID = docId;
-      int curDocID = vals.docID();
-      if (docId > curDocID) {
-        curDocID = vals.advance(docId);
-      }
-      if (docId == curDocID) {
-        currentValue = (int) vals.longValue();
-      } else {
-        currentValue = 0;
-      }
-    }
-
-    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;
-    private int lastDocID;
-
-    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);
-      lastDocID = 0;
-    }
-
-    public void setCurrentValue(int docId) throws IOException {
-      if (docId < lastDocID) {
-        throw new AssertionError("docs were sent out-of-order: lastDocID=" + lastDocID + " vs doc=" + docId);
-      }
-      lastDocID = docId;
-      int curDocID = vals.docID();
-      if (docId > curDocID) {
-        curDocID = vals.advance(docId);
-      }
-      if (docId == curDocID) {
-        currentValue = vals.longValue();
-      } else {
-        currentValue = 0;
-      }
-    }
-
-    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;
-    private int lastDocID;
-
-    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);
-      lastDocID = 0;
-    }
-
-    public void setCurrentValue(int docId) throws IOException {
-      if (docId < lastDocID) {
-        throw new AssertionError("docs were sent out-of-order: lastDocID=" + lastDocID + " vs doc=" + docId);
-      }
-      lastDocID = docId;
-      int curDocID = vals.docID();
-      if (docId > curDocID) {
-        curDocID = vals.advance(docId);
-      }
-      if (docId == curDocID) {
-        currentValue = Float.intBitsToFloat((int)vals.longValue());
-      } else {
-        currentValue = 0f;
-      }
-    }
-
-    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;
-    private int lastDocID;
-    private LeafReader reader;
-
-    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.reader = context.reader();
-      this.vals = DocValues.getNumeric(this.reader, this.field);
-      lastDocID = 0;
-    }
-
-    public void setCurrentValue(int docId) throws IOException {
-      if (docId < lastDocID) {
-        // TODO: can we enforce caller to go in order instead?
-        this.vals = DocValues.getNumeric(this.reader, this.field);
-      }
-      lastDocID = docId;
-      int curDocID = vals.docID();
-      if (docId > curDocID) {
-        curDocID = vals.advance(docId);
-      }
-      if (docId == curDocID) {
-        currentValue = Double.longBitsToDouble(vals.longValue());
-      } else {
-        currentValue = 0f;
-      }
-    }
-
-    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) throws IOException {
-      if (docId > currentVals.docID()) {
-        currentVals.advance(docId);
-      }
-      if (docId == currentVals.docID()) {
-        int ord = currentVals.ordValue();
-        if(globalOrds != null) {
-          currentOrd = (int)globalOrds.get(ord);
-        } else {
-          currentOrd = ord;
-        }
-      } else {
-        currentOrd = -1;
-      }
-    }
-
-    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;
-      if (vals.advance(docId) == docId) {
-        val = (int) vals.longValue();
-      } else {
-        val = 0;
-      }
-      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);
-      List<Long> ords;
-      if (vals.advance(docId) == docId) {
-        ords = new ArrayList();
-        long o = -1;
-        while((o = vals.nextOrd()) != SortedSetDocValues.NO_MORE_ORDS) {
-          ords.add(o);
-        }
-        assert ords.size() > 0;
-      } else {
-        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;
-      if (vals.advance(docId) == docId) {
-        val = vals.longValue();
-      } else {
-        val = 0;
-      }
-      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;
-      if (vals.advance(docId) == docId) {
-        val = vals.longValue();
-      } else {
-        val = 0;
-      }
-
-      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);
-      if (vals.advance(docId) != docId) {
-        return false;
-      }
-      int ord = vals.ordValue();
-
-      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;
-      if (vals.advance(docId) == docId) {
-        val = (int)vals.longValue();
-      } else {
-        val = 0;
-      }
-      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);
-      long val;
-      if (vals.advance(docId) == docId) {
-        val = vals.longValue();
-      } else {
-        val = 0;
-      }
-      if(fieldIndex > 0) {
-        out.write(',');
-      }
-      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);
-      if (vals.advance(docId) != docId) {
-        return false;
-      }
-      int ord = vals.ordValue();
-
-      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/ef074a61/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/ef074a61/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/ef074a61/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/ef074a61/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/ef074a61/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/ef074a61/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/ef074a61/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/ef074a61/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;
+    }
+  }
+}