You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by jb...@apache.org on 2015/03/10 02:38:58 UTC

svn commit: r1665391 [1/4] - in /lucene/dev/trunk/solr: core/src/java/org/apache/solr/handler/ core/src/java/org/apache/solr/response/ core/src/java/org/apache/solr/search/ core/src/test-files/solr/collection1/conf/ core/src/test/org/apache/solr/search...

Author: jbernste
Date: Tue Mar 10 01:38:57 2015
New Revision: 1665391

URL: http://svn.apache.org/r1665391
Log:
SOLR-7082: Streaming Aggregation for SolrCloud

Added:
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/StreamHandler.java   (with props)
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/HashQParserPlugin.java   (with props)
    lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/schema-hash.xml   (with props)
    lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/solrconfig-hash.xml   (with props)
    lucene/dev/trunk/solr/core/src/test/org/apache/solr/search/TestHashQParserPlugin.java   (with props)
    lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/client/solrj/impl/InputStreamResponseParser.java   (with props)
    lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/client/solrj/io/
    lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/client/solrj/io/AscBucketComp.java   (with props)
    lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/client/solrj/io/AscFieldComp.java   (with props)
    lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/client/solrj/io/AscMetricComp.java   (with props)
    lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/client/solrj/io/Bucket.java   (with props)
    lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/client/solrj/io/BucketMetrics.java   (with props)
    lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/client/solrj/io/CloudSolrStream.java   (with props)
    lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/client/solrj/io/CountMetric.java   (with props)
    lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/client/solrj/io/DescBucketComp.java   (with props)
    lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/client/solrj/io/DescFieldComp.java   (with props)
    lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/client/solrj/io/DescMetricComp.java   (with props)
    lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/client/solrj/io/FilterStream.java   (with props)
    lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/client/solrj/io/GroupByStream.java   (with props)
    lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/client/solrj/io/HashJoinStream.java   (with props)
    lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/client/solrj/io/HashKey.java   (with props)
    lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/client/solrj/io/JSONTupleStream.java   (with props)
    lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/client/solrj/io/MaxMetric.java   (with props)
    lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/client/solrj/io/MeanMetric.java   (with props)
    lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/client/solrj/io/MergeJoinStream.java   (with props)
    lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/client/solrj/io/MergeStream.java   (with props)
    lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/client/solrj/io/Metric.java   (with props)
    lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/client/solrj/io/MetricStream.java   (with props)
    lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/client/solrj/io/MinMetric.java   (with props)
    lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/client/solrj/io/MultiComp.java   (with props)
    lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/client/solrj/io/ParallelStream.java   (with props)
    lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/client/solrj/io/PushBackStream.java   (with props)
    lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/client/solrj/io/RankStream.java   (with props)
    lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/client/solrj/io/RollupStream.java   (with props)
    lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/client/solrj/io/SolrClientCache.java   (with props)
    lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/client/solrj/io/SolrStream.java   (with props)
    lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/client/solrj/io/StreamContext.java   (with props)
    lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/client/solrj/io/SumMetric.java   (with props)
    lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/client/solrj/io/Tuple.java   (with props)
    lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/client/solrj/io/TupleStream.java   (with props)
    lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/client/solrj/io/UniqueStream.java   (with props)
    lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/client/solrj/io/package-info.java   (with props)
    lucene/dev/trunk/solr/solrj/src/test-files/solrj/solr/collection1/conf/schema-streaming.xml   (with props)
    lucene/dev/trunk/solr/solrj/src/test-files/solrj/solr/collection1/conf/solrconfig-streaming.xml   (with props)
    lucene/dev/trunk/solr/solrj/src/test/org/apache/solr/client/solrj/io/
    lucene/dev/trunk/solr/solrj/src/test/org/apache/solr/client/solrj/io/StreamingTest.java   (with props)
Modified:
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/response/TextResponseWriter.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/QParserPlugin.java
    lucene/dev/trunk/solr/core/src/test/org/apache/solr/search/QueryEqualityTest.java
    lucene/dev/trunk/solr/server/solr/configsets/data_driven_schema_configs/conf/solrconfig.xml
    lucene/dev/trunk/solr/server/solr/configsets/sample_techproducts_configs/conf/solrconfig.xml
    lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpSolrClient.java

Added: lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/StreamHandler.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/StreamHandler.java?rev=1665391&view=auto
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/StreamHandler.java (added)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/StreamHandler.java Tue Mar 10 01:38:57 2015
@@ -0,0 +1,84 @@
+/*
+ * 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.ByteArrayInputStream;
+import java.io.ObjectInputStream;
+import java.net.URLDecoder;
+import java.util.Iterator;
+import java.util.concurrent.ConcurrentHashMap;
+
+import org.apache.solr.client.solrj.io.SolrClientCache;
+import org.apache.solr.client.solrj.io.TupleStream;
+import org.apache.solr.client.solrj.io.StreamContext;
+import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.core.CloseHook;
+import org.apache.solr.core.SolrCore;
+import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.response.SolrQueryResponse;
+import org.apache.solr.common.params.SolrParams;
+import org.apache.solr.util.plugin.SolrCoreAware;
+import org.apache.solr.common.util.Base64;
+
+
+public class StreamHandler extends RequestHandlerBase implements SolrCoreAware {
+
+  private SolrClientCache clientCache = new SolrClientCache();
+
+  public void inform(SolrCore core) {
+
+    core.addCloseHook( new CloseHook() {
+      @Override
+      public void preClose(SolrCore core) {
+        //To change body of implemented methods use File | Settings | File Templates.
+      }
+
+      @Override
+      public void postClose(SolrCore core) {
+        clientCache.close();
+      }
+    });
+  }
+
+  public void handleRequestBody(SolrQueryRequest req, SolrQueryResponse rsp) throws Exception {
+    SolrParams params = req.getParams();
+    String encodedStream = params.get("stream");
+    encodedStream = URLDecoder.decode(encodedStream, "UTF-8");
+    byte[] bytes = Base64.base64ToByteArray(encodedStream);
+    ByteArrayInputStream byteStream = new ByteArrayInputStream(bytes);
+    ObjectInputStream objectInputStream = new ObjectInputStream(byteStream);
+    TupleStream tupleStream = (TupleStream)objectInputStream.readObject();
+
+    int worker = params.getInt("workerID");
+    int numWorkers = params.getInt("numWorkers");
+    StreamContext context = new StreamContext();
+    context.workerID = worker;
+    context.numWorkers = numWorkers;
+    context.clientCache = clientCache;
+    tupleStream.setStreamContext(context);
+    rsp.add("tuples", tupleStream);
+  }
+
+  public String getDescription() {
+    return "StreamHandler";
+  }
+
+  public String getSource() {
+    return null;
+  }
+}
\ No newline at end of file

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/response/TextResponseWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/response/TextResponseWriter.java?rev=1665391&r1=1665390&r2=1665391&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/response/TextResponseWriter.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/response/TextResponseWriter.java Tue Mar 10 01:38:57 2015
@@ -23,6 +23,8 @@ import java.util.*;
 
 import org.apache.lucene.document.Document;
 import org.apache.lucene.index.IndexableField;
+import org.apache.solr.client.solrj.io.TupleStream;
+import org.apache.solr.client.solrj.io.Tuple;
 import org.apache.lucene.index.StorableField;
 import org.apache.lucene.index.StoredDocument;
 import org.apache.lucene.util.BytesRef;
@@ -186,12 +188,14 @@ public abstract class TextResponseWriter
       writeMap(name, (Map)val, false, true);
     } else if (val instanceof NamedList) {
       writeNamedList(name, (NamedList)val);
+    } else if (val instanceof TupleStream) {
+      writeTupleStream((TupleStream) val);
     } else if (val instanceof Iterable) {
       writeArray(name,((Iterable)val).iterator());
     } else if (val instanceof Object[]) {
       writeArray(name,(Object[])val);
     } else if (val instanceof Iterator) {
-      writeArray(name,(Iterator)val);
+      writeArray(name, (Iterator) val);
     } else if (val instanceof byte[]) {
       byte[] arr = (byte[])val;
       writeByteArr(name, arr, 0, arr.length);
@@ -309,6 +313,26 @@ public abstract class TextResponseWriter
     }
   }
 
+  public void writeTupleStream(TupleStream tupleStream) throws IOException {
+    tupleStream.open();
+    writeStartDocumentList("response", -1, -1, -1, null);
+    boolean isFirst = true;
+    while(true) {
+      Tuple tuple = tupleStream.read();
+      if(!isFirst) {
+        writer.write(",");
+      }
+      writeMap(null, tuple.fields, false, true);
+      isFirst = false;
+      if(tuple.EOF) {
+        break;
+      }
+    }
+    writeEndDocumentList();
+    tupleStream.close();
+  }
+
+
   /** if this form of the method is called, val is the Java string form of a double */
   public abstract void writeDouble(String name, String val) throws IOException;
 

Added: lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/HashQParserPlugin.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/HashQParserPlugin.java?rev=1665391&view=auto
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/HashQParserPlugin.java (added)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/HashQParserPlugin.java Tue Mar 10 01:38:57 2015
@@ -0,0 +1,394 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.search;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.List;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.Future;
+import com.google.common.primitives.Longs;
+
+import org.apache.lucene.index.LeafReader;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.IndexReaderContext;
+import org.apache.lucene.search.LeafCollector;
+import org.apache.lucene.util.BitDocIdSet;
+import org.apache.lucene.util.CharsRef;
+import org.apache.lucene.util.CharsRefBuilder;
+import org.apache.lucene.util.FixedBitSet;
+
+import org.apache.lucene.util.Bits;
+import org.apache.lucene.search.BitsFilteredDocIdSet;
+import org.apache.solr.common.params.SolrParams;
+import org.apache.solr.common.util.SolrjNamedThreadFactory;
+import org.apache.solr.core.CloseHook;
+import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.schema.IndexSchema;
+import org.apache.solr.schema.FieldType;
+import org.apache.solr.schema.StrField;
+import org.apache.solr.schema.TrieField;
+import org.apache.solr.core.SolrCore;
+
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.ConstantScoreQuery;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Scorer;
+import org.apache.lucene.search.Weight;
+import org.apache.lucene.search.DocIdSet;
+import org.apache.lucene.search.Filter;
+import org.apache.lucene.index.SortedDocValues;
+import org.apache.lucene.index.NumericDocValues;
+import org.apache.lucene.util.BytesRef;
+
+import org.apache.solr.common.util.NamedList;
+
+/**
+* syntax fq={!hash workers=11 worker=4 keys=field1,field2}
+* */
+
+public class HashQParserPlugin extends QParserPlugin {
+
+  public static final String NAME = "hash";
+  private static Semaphore semaphore = new Semaphore(8,true);
+  private static ExecutorService threadPool = Executors.newCachedThreadPool(new SolrjNamedThreadFactory("HashQParserPlugin"));
+  private static boolean init = true;
+
+  private static synchronized void closeHook(SolrCore core) {
+    if(init) {
+      init = false;
+      core.addCloseHook(new CloseHook() {
+        @Override
+        public void preClose(SolrCore core) {
+          threadPool.shutdown();
+          //To change body of implemented methods use File | Settings | File Templates.
+        }
+
+        @Override
+        public void postClose(SolrCore core) {
+          //To change body of implemented methods use File | Settings | File Templates.
+        }
+      });
+    }
+  }
+
+  public void init(NamedList params) {
+
+  }
+
+  public QParser createParser(String query, SolrParams localParams, SolrParams params, SolrQueryRequest request) {
+    closeHook(request.getSearcher().getCore());
+    return new HashQParser(query, localParams, params, request);
+  }
+
+  private class HashQParser extends QParser {
+
+    public HashQParser(String query, SolrParams localParams, SolrParams params, SolrQueryRequest request) {
+      super(query, localParams, params, request);
+    }
+
+    public Query parse() {
+      int workers = localParams.getInt("workers");
+      int worker = localParams.getInt("worker");
+      String keys = params.get("partitionKeys");
+      return new HashQuery(keys, workers, worker);
+    }
+  }
+
+  private class HashQuery extends ExtendedQueryBase implements PostFilter {
+
+    private String keysParam;
+    private int workers;
+    private int worker;
+
+    public boolean getCache() {
+      if(getCost() > 99) {
+        return false;
+      } else {
+        return super.getCache();
+      }
+    }
+
+    public int hashCode() {
+      return keysParam.hashCode()+workers+worker+(int)getBoost();
+    }
+
+    public boolean equals(Object o) {
+      if (o instanceof HashQuery) {
+        HashQuery h = (HashQuery)o;
+        if(keysParam.equals(h.keysParam) && workers == h.workers && worker == h.worker && getBoost() == h.getBoost()) {
+          return true;
+        }
+      }
+
+      return false;
+    }
+
+    public HashQuery(String keysParam, int workers, int worker) {
+      this.keysParam = keysParam;
+      this.workers = workers;
+      this.worker = worker;
+    }
+
+    public Weight createWeight(IndexSearcher searcher, boolean needsScores) throws IOException {
+
+      String[] keys = keysParam.split(",");
+      SolrIndexSearcher solrIndexSearcher = (SolrIndexSearcher)searcher;
+      IndexReaderContext context = solrIndexSearcher.getTopReaderContext();
+
+      List<LeafReaderContext> leaves =  context.leaves();
+      ArrayBlockingQueue queue = new ArrayBlockingQueue(leaves.size());
+
+
+      for(LeafReaderContext leaf : leaves) {
+        try {
+          semaphore.acquire();
+          SegmentPartitioner segmentPartitioner = new SegmentPartitioner(leaf,worker,workers, keys, solrIndexSearcher, queue,semaphore);
+          threadPool.execute(segmentPartitioner);
+        } catch(Exception e) {
+          throw new IOException(e);
+        }
+      }
+
+      FixedBitSet[] fixedBitSets = new FixedBitSet[leaves.size()];
+      for(int i=0; i<leaves.size(); i++) {
+        try {
+          SegmentPartitioner segmentPartitioner = (SegmentPartitioner)queue.take();
+          fixedBitSets[segmentPartitioner.context.ord] = segmentPartitioner.docs;
+        }catch(Exception e) {
+          throw new IOException(e);
+        }
+      }
+
+      ConstantScoreQuery constantScoreQuery = new ConstantScoreQuery(new BitsFilter(fixedBitSets));
+      return constantScoreQuery.createWeight(searcher, false);
+    }
+
+    public class BitsFilter extends Filter {
+      private FixedBitSet[] bitSets;
+      public BitsFilter(FixedBitSet[] bitSets) {
+        this.bitSets = bitSets;
+      }
+
+      public String toString(String s) {
+        return s;
+      }
+
+      public DocIdSet getDocIdSet(LeafReaderContext context, Bits bits) {
+        return BitsFilteredDocIdSet.wrap(new BitDocIdSet(bitSets[context.ord]), bits);
+      }
+    }
+
+
+    class SegmentPartitioner implements Runnable {
+
+      public LeafReaderContext context;
+      private int worker;
+      private int workers;
+      private HashKey k;
+      private Semaphore sem;
+      private ArrayBlockingQueue queue;
+      public FixedBitSet docs;
+      public SegmentPartitioner(LeafReaderContext context,
+                                int worker,
+                                int workers,
+                                String[] keys,
+                                SolrIndexSearcher solrIndexSearcher,
+                                ArrayBlockingQueue queue, Semaphore sem) {
+        this.context = context;
+        this.worker = worker;
+        this.workers = workers;
+        this.queue = queue;
+        this.sem = sem;
+
+        HashKey[] hashKeys = new HashKey[keys.length];
+        IndexSchema schema = solrIndexSearcher.getSchema();
+        for(int i=0; i<keys.length; i++) {
+          String key = keys[i];
+          FieldType ft = schema.getField(key).getType();
+          HashKey h = null;
+          if(ft instanceof StrField) {
+            h = new BytesHash(key, ft);
+          } else {
+            h = new NumericHash(key);
+          }
+          hashKeys[i] = h;
+        }
+
+        k = (hashKeys.length > 1) ? new CompositeHash(hashKeys) : hashKeys[0];
+      }
+
+      public void run() {
+        LeafReader reader = context.reader();
+
+        try {
+          k.setNextReader(context);
+          this.docs = new FixedBitSet(reader.maxDoc());
+          int maxDoc = reader.maxDoc();
+          for(int i=0; i<maxDoc; i++) {
+            if((k.hashCode(i) & 0x7FFFFFFF) % workers == worker) {
+              docs.set(i);
+            }
+          }
+        }catch(Exception e) {
+         throw new RuntimeException(e);
+        } finally {
+          sem.release();
+          queue.add(this);
+        }
+      }
+    }
+
+    public DelegatingCollector getFilterCollector(IndexSearcher indexSearcher) {
+      String[] keys = keysParam.split(",");
+      HashKey[] hashKeys = new HashKey[keys.length];
+      SolrIndexSearcher searcher = (SolrIndexSearcher)indexSearcher;
+      IndexSchema schema = searcher.getSchema();
+      for(int i=0; i<keys.length; i++) {
+        String key = keys[i];
+        FieldType ft = schema.getField(key).getType();
+        HashKey h = null;
+        if(ft instanceof StrField) {
+          h = new BytesHash(key, ft);
+        } else {
+          h = new NumericHash(key);
+        }
+        hashKeys[i] = h;
+      }
+      HashKey k = (hashKeys.length > 1) ? new CompositeHash(hashKeys) : hashKeys[0];
+      return new HashCollector(k, workers, worker);
+    }
+  }
+
+  private class HashCollector extends DelegatingCollector {
+    private int worker;
+    private int workers;
+    private HashKey hashKey;
+    private LeafCollector leafCollector;
+
+    public HashCollector(HashKey hashKey, int workers, int worker) {
+      this.hashKey = hashKey;
+      this.workers = workers;
+      this.worker = worker;
+    }
+
+    public void setScorer(Scorer scorer) throws IOException{
+      leafCollector.setScorer(scorer);
+    }
+
+    public void doSetNextReader(LeafReaderContext context) throws IOException {
+      this.hashKey.setNextReader(context);
+      this.leafCollector = delegate.getLeafCollector(context);
+    }
+
+    public void collect(int doc) throws IOException {
+      if((hashKey.hashCode(doc) & 0x7FFFFFFF) % workers == worker) {
+        leafCollector.collect(doc);
+      }
+    }
+  }
+
+  private interface HashKey {
+    public void setNextReader(LeafReaderContext reader) throws IOException;
+    public long hashCode(int doc);
+  }
+
+  private class BytesHash implements HashKey {
+
+    private SortedDocValues values;
+    private String field;
+    private FieldType fieldType;
+    private CharsRefBuilder charsRefBuilder = new CharsRefBuilder();
+
+    public BytesHash(String field, FieldType fieldType) {
+      this.field = field;
+      this.fieldType = fieldType;
+    }
+
+    public void setNextReader(LeafReaderContext context) throws IOException {
+      values = context.reader().getSortedDocValues(field);
+    }
+
+    public long hashCode(int doc) {
+      BytesRef ref = values.get(doc);
+      this.fieldType.indexedToReadable(ref, charsRefBuilder);
+      CharsRef charsRef = charsRefBuilder.get();
+      return charsRef.hashCode();
+    }
+  }
+
+  private class NumericHash implements HashKey {
+
+    private NumericDocValues values;
+    private String field;
+
+    public NumericHash(String field) {
+      this.field = field;
+    }
+
+    public void setNextReader(LeafReaderContext context) throws IOException {
+      values = context.reader().getNumericDocValues(field);
+    }
+
+    public long hashCode(int doc) {
+      long l = values.get(doc);
+      return Longs.hashCode(l);
+    }
+  }
+
+  private class ZeroHash implements HashKey {
+
+    public long hashCode(int doc) {
+      return 0;
+    }
+
+    public void setNextReader(LeafReaderContext context) {
+
+    }
+  }
+
+  private class CompositeHash implements HashKey {
+
+    private HashKey key1;
+    private HashKey key2;
+    private HashKey key3;
+    private HashKey key4;
+
+    public CompositeHash(HashKey[] hashKeys) {
+      key1 = hashKeys[0];
+      key2 = hashKeys[1];
+      key3 = (hashKeys.length > 2) ? hashKeys[2] : new ZeroHash();
+      key4 = (hashKeys.length > 3) ? hashKeys[3] : new ZeroHash();
+    }
+
+    public void setNextReader(LeafReaderContext context) throws IOException {
+      key1.setNextReader(context);
+      key2.setNextReader(context);
+      key3.setNextReader(context);
+      key4.setNextReader(context);
+    }
+
+    public long hashCode(int doc) {
+      return key1.hashCode(doc)+key2.hashCode(doc)+key3.hashCode(doc)+key4.hashCode(doc);
+    }
+  }
+}

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/QParserPlugin.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/QParserPlugin.java?rev=1665391&r1=1665390&r2=1665391&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/QParserPlugin.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/QParserPlugin.java Tue Mar 10 01:38:57 2015
@@ -72,6 +72,7 @@ public abstract class QParserPlugin impl
     map.put(ReRankQParserPlugin.NAME, ReRankQParserPlugin.class);
     map.put(ExportQParserPlugin.NAME, ExportQParserPlugin.class);
     map.put(MLTQParserPlugin.NAME, MLTQParserPlugin.class);
+    map.put(HashQParserPlugin.NAME, HashQParserPlugin.class);
     standardPlugins = Collections.unmodifiableMap(map);
   }
 

Added: lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/schema-hash.xml
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/schema-hash.xml?rev=1665391&view=auto
==============================================================================
--- lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/schema-hash.xml (added)
+++ lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/schema-hash.xml Tue Mar 10 01:38:57 2015
@@ -0,0 +1,587 @@
+<?xml version="1.0" ?>
+<!--
+ Licensed to the Apache Software Foundation (ASF) under one or more
+ contributor license agreements.  See the NOTICE file distributed with
+ this work for additional information regarding copyright ownership.
+ The ASF licenses this file to You under the Apache License, Version 2.0
+ (the "License"); you may not use this file except in compliance with
+ the License.  You may obtain a copy of the License at
+
+     http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+
+<!-- The Solr schema file. This file should be named "schema.xml" and
+     should be located where the classloader for the Solr webapp can find it.
+
+     This schema is used for testing, and as such has everything and the
+     kitchen sink thrown in. See example/solr/conf/schema.xml for a
+     more concise example.
+
+  -->
+
+<schema name="test" version="1.5">
+  <types>
+
+    <!-- field type definitions... note that the "name" attribute is
+         just a label to be used by field definitions.  The "class"
+         attribute and any other attributes determine the real type and
+         behavior of the fieldtype.
+      -->
+
+    <!-- numeric field types that store and index the text
+         value verbatim (and hence don't sort correctly or support range queries.)
+         These are provided more for backward compatability, allowing one
+         to create a schema that matches an existing lucene index.
+    -->
+
+    <fieldType name="int" class="solr.TrieIntField" precisionStep="0" omitNorms="true" positionIncrementGap="0" docValues="true"/>
+    <fieldType name="float" class="solr.TrieFloatField" precisionStep="0" omitNorms="true" positionIncrementGap="0"/>
+    <fieldType name="long" class="solr.TrieLongField" precisionStep="0" omitNorms="true" positionIncrementGap="0" docValues="true"/>
+    <fieldType name="double" class="solr.TrieDoubleField" precisionStep="0" omitNorms="true" positionIncrementGap="0"/>
+
+    <fieldType name="tint" class="solr.TrieIntField" precisionStep="8" omitNorms="true" positionIncrementGap="0"/>
+    <fieldType name="tfloat" class="solr.TrieFloatField" precisionStep="8" omitNorms="true" positionIncrementGap="0"/>
+    <fieldType name="tlong" class="solr.TrieLongField" precisionStep="8" omitNorms="true" positionIncrementGap="0"/>
+    <fieldType name="tdouble" class="solr.TrieDoubleField" precisionStep="8" omitNorms="true" positionIncrementGap="0"/>
+
+    <!-- numeric field types that manipulate the value into
+       a string value that isn't human readable in it's internal form,
+       but sorts correctly and supports range queries.
+
+         If sortMissingLast="true" then a sort on this field will cause documents
+       without the field to come after documents with the field,
+       regardless of the requested sort order.
+         If sortMissingFirst="true" then a sort on this field will cause documents
+       without the field to come before documents with the field,
+       regardless of the requested sort order.
+         If sortMissingLast="false" and sortMissingFirst="false" (the default),
+       then default lucene sorting will be used which places docs without the field
+       first in an ascending sort and last in a descending sort.
+    -->
+
+
+
+    <!-- Field type demonstrating an Analyzer failure -->
+    <fieldtype name="failtype1" class="solr.TextField">
+      <analyzer type="index">
+        <tokenizer class="solr.MockTokenizerFactory"/>
+        <filter class="solr.WordDelimiterFilterFactory" generateWordParts="1" generateNumberParts="0" catenateWords="0" catenateNumbers="0" catenateAll="0"/>
+        <filter class="solr.LowerCaseFilterFactory"/>
+      </analyzer>
+    </fieldtype>
+
+    <!-- Demonstrating ignoreCaseChange -->
+    <fieldtype name="wdf_nocase" class="solr.TextField">
+      <analyzer>
+        <tokenizer class="solr.MockTokenizerFactory"/>
+        <filter class="solr.WordDelimiterFilterFactory" generateWordParts="1" generateNumberParts="0" catenateWords="0" catenateNumbers="0" catenateAll="0" splitOnCaseChange="0" preserveOriginal="0"/>
+        <filter class="solr.LowerCaseFilterFactory"/>
+      </analyzer>
+    </fieldtype>
+
+    <fieldtype name="wdf_preserve" class="solr.TextField">
+      <analyzer>
+        <tokenizer class="solr.MockTokenizerFactory"/>
+        <filter class="solr.WordDelimiterFilterFactory" generateWordParts="0" generateNumberParts="1" catenateWords="0" catenateNumbers="0" catenateAll="0" splitOnCaseChange="0" preserveOriginal="1"/>
+        <filter class="solr.LowerCaseFilterFactory"/>
+      </analyzer>
+    </fieldtype>
+
+
+    <fieldtype name="boolean" class="solr.BoolField" sortMissingLast="true"/>
+    <fieldtype name="string" class="solr.StrField" sortMissingLast="true" docValues="true"/>
+
+    <!-- format for date is 1995-12-31T23:59:59.999Z and only the fractional
+         seconds part (.999) is optional.
+      -->
+    <fieldtype name="date" class="solr.TrieDateField" precisionStep="0"/>
+    <fieldtype name="tdate" class="solr.TrieDateField" precisionStep="6"/>
+
+
+    <!-- solr.TextField allows the specification of custom
+         text analyzers specified as a tokenizer and a list
+         of token filters.
+      -->
+    <fieldtype name="text" class="solr.TextField">
+      <analyzer>
+        <tokenizer class="solr.StandardTokenizerFactory"/>
+        <filter class="solr.StandardFilterFactory"/>
+        <filter class="solr.LowerCaseFilterFactory"/>
+        <filter class="solr.StopFilterFactory"/>
+        <filter class="solr.PorterStemFilterFactory"/>
+      </analyzer>
+    </fieldtype>
+
+
+    <fieldtype name="nametext" class="solr.TextField">
+      <analyzer class="org.apache.lucene.analysis.core.WhitespaceAnalyzer"/>
+    </fieldtype>
+
+    <fieldtype name="teststop" class="solr.TextField">
+      <analyzer>
+        <tokenizer class="solr.LowerCaseTokenizerFactory"/>
+        <filter class="solr.StandardFilterFactory"/>
+      </analyzer>
+    </fieldtype>
+
+    <!-- fieldtypes in this section isolate tokenizers and tokenfilters for testing -->
+    <fieldtype name="lowertok" class="solr.TextField">
+      <analyzer><tokenizer class="solr.LowerCaseTokenizerFactory"/></analyzer>
+    </fieldtype>
+    <fieldtype name="keywordtok" class="solr.TextField">
+      <analyzer><tokenizer class="solr.MockTokenizerFactory" pattern="keyword"/></analyzer>
+    </fieldtype>
+    <fieldtype name="standardtok" class="solr.TextField">
+      <analyzer><tokenizer class="solr.StandardTokenizerFactory"/></analyzer>
+    </fieldtype>
+    <fieldtype name="lettertok" class="solr.TextField">
+      <analyzer><tokenizer class="solr.LetterTokenizerFactory"/></analyzer>
+    </fieldtype>
+    <fieldtype name="whitetok" class="solr.TextField">
+      <analyzer><tokenizer class="solr.MockTokenizerFactory"/></analyzer>
+    </fieldtype>
+    <fieldtype name="HTMLstandardtok" class="solr.TextField">
+      <analyzer>
+        <charFilter class="solr.HTMLStripCharFilterFactory"/>
+        <tokenizer class="solr.StandardTokenizerFactory"/>
+      </analyzer>
+    </fieldtype>
+    <fieldtype name="HTMLwhitetok" class="solr.TextField">
+      <analyzer>
+        <charFilter class="solr.HTMLStripCharFilterFactory"/>
+        <tokenizer class="solr.MockTokenizerFactory"/>
+      </analyzer>
+    </fieldtype>
+    <fieldtype name="standardtokfilt" class="solr.TextField">
+      <analyzer>
+        <tokenizer class="solr.StandardTokenizerFactory"/>
+        <filter class="solr.StandardFilterFactory"/>
+      </analyzer>
+    </fieldtype>
+    <fieldtype name="standardfilt" class="solr.TextField">
+      <analyzer>
+        <tokenizer class="solr.MockTokenizerFactory"/>
+        <filter class="solr.StandardFilterFactory"/>
+      </analyzer>
+    </fieldtype>
+    <fieldtype name="lowerfilt" class="solr.TextField">
+      <analyzer>
+        <tokenizer class="solr.MockTokenizerFactory"/>
+        <filter class="solr.LowerCaseFilterFactory"/>
+      </analyzer>
+    </fieldtype>
+    <fieldtype name="lowerpunctfilt" class="solr.TextField">
+      <analyzer>
+        <tokenizer class="solr.MockTokenizerFactory"/>
+        <filter class="solr.WordDelimiterFilterFactory" generateWordParts="1" generateNumberParts="1" catenateWords="1" catenateNumbers="1" catenateAll="1" splitOnCaseChange="1"/>
+        <filter class="solr.LowerCaseFilterFactory"/>
+      </analyzer>
+    </fieldtype>
+    <fieldtype name="patternreplacefilt" class="solr.TextField">
+      <analyzer type="index">
+        <tokenizer class="solr.MockTokenizerFactory" pattern="keyword"/>
+        <filter class="solr.PatternReplaceFilterFactory"
+                pattern="([^a-zA-Z])" replacement="_" replace="all"
+            />
+      </analyzer>
+      <analyzer type="query">
+        <tokenizer class="solr.MockTokenizerFactory" pattern="keyword"/>
+      </analyzer>
+    </fieldtype>
+    <fieldtype name="patterntok" class="solr.TextField">
+      <analyzer>
+        <tokenizer class="solr.PatternTokenizerFactory" pattern=","/>
+      </analyzer>
+    </fieldtype>
+    <fieldtype name="porterfilt" class="solr.TextField">
+      <analyzer>
+        <tokenizer class="solr.MockTokenizerFactory"/>
+        <filter class="solr.PorterStemFilterFactory"/>
+      </analyzer>
+    </fieldtype>
+    <!-- fieldtype name="snowballfilt" class="solr.TextField">
+      <analyzer>
+        <tokenizer class="solr.MockTokenizerFactory"/>
+        <filter class="solr.SnowballPorterFilterFactory"/>
+      </analyzer>
+    </fieldtype -->
+    <fieldtype name="engporterfilt" class="solr.TextField">
+      <analyzer>
+        <tokenizer class="solr.MockTokenizerFactory"/>
+        <filter class="solr.PorterStemFilterFactory"/>
+      </analyzer>
+    </fieldtype>
+    <fieldtype name="custengporterfilt" class="solr.TextField">
+      <analyzer>
+        <tokenizer class="solr.MockTokenizerFactory"/>
+        <filter class="solr.PorterStemFilterFactory"/>
+      </analyzer>
+    </fieldtype>
+    <fieldtype name="stopfilt" class="solr.TextField">
+      <analyzer>
+        <tokenizer class="solr.MockTokenizerFactory"/>
+        <filter class="solr.StopFilterFactory" ignoreCase="true"/>
+      </analyzer>
+    </fieldtype>
+    <fieldtype name="custstopfilt" class="solr.TextField">
+      <analyzer>
+        <tokenizer class="solr.MockTokenizerFactory"/>
+      </analyzer>
+    </fieldtype>
+    <fieldtype name="lengthfilt" class="solr.TextField">
+      <analyzer>
+        <tokenizer class="solr.MockTokenizerFactory"/>
+        <filter class="solr.LengthFilterFactory" min="2" max="5"/>
+      </analyzer>
+    </fieldtype>
+    <fieldType name="charfilthtmlmap" class="solr.TextField">
+      <analyzer>
+        <charFilter class="solr.HTMLStripCharFilterFactory"/>
+        <tokenizer class="solr.MockTokenizerFactory"/>
+      </analyzer>
+    </fieldType>
+
+    <fieldtype name="subword" class="solr.TextField" multiValued="true" positionIncrementGap="100">
+      <analyzer type="index">
+        <tokenizer class="solr.MockTokenizerFactory"/>
+        <filter class="solr.WordDelimiterFilterFactory" generateWordParts="1" generateNumberParts="1" catenateWords="1" catenateNumbers="1" catenateAll="0"/>
+        <filter class="solr.LowerCaseFilterFactory"/>
+        <filter class="solr.StopFilterFactory"/>
+        <filter class="solr.PorterStemFilterFactory"/>
+      </analyzer>
+      <analyzer type="query">
+        <tokenizer class="solr.MockTokenizerFactory"/>
+        <filter class="solr.WordDelimiterFilterFactory" generateWordParts="1" generateNumberParts="1" catenateWords="0" catenateNumbers="0" catenateAll="0"/>
+        <filter class="solr.LowerCaseFilterFactory"/>
+        <filter class="solr.StopFilterFactory"/>
+        <filter class="solr.PorterStemFilterFactory"/>
+      </analyzer>
+    </fieldtype>
+
+    <fieldtype name="numericsubword" class="solr.TextField" multiValued="true" positionIncrementGap="100">
+      <analyzer type="index">
+        <tokenizer class="solr.MockTokenizerFactory"/>
+        <filter class="solr.LowerCaseFilterFactory"/>
+        <filter class="solr.WordDelimiterFilterFactory" splitOnNumerics="0" splitOnCaseChange="0" generateWordParts="1" generateNumberParts="0" catenateWords="0" catenateNumbers="0" catenateAll="0"/>
+        <filter class="solr.StopFilterFactory"/>
+        <filter class="solr.PorterStemFilterFactory"/>
+      </analyzer>
+      <analyzer type="query">
+        <tokenizer class="solr.MockTokenizerFactory"/>
+        <filter class="solr.LowerCaseFilterFactory"/>
+        <filter class="solr.WordDelimiterFilterFactory" splitOnNumerics="0" splitOnCaseChange="0" generateWordParts="1" generateNumberParts="1" catenateWords="1" catenateNumbers="1" catenateAll="0"/>
+        <filter class="solr.StopFilterFactory"/>
+        <filter class="solr.PorterStemFilterFactory"/>
+      </analyzer>
+    </fieldtype>
+
+    <fieldtype name="protectedsubword" class="solr.TextField" multiValued="true" positionIncrementGap="100">
+      <analyzer type="index">
+        <tokenizer class="solr.MockTokenizerFactory"/>
+        <filter class="solr.LowerCaseFilterFactory"/>
+        <filter class="solr.WordDelimiterFilterFactory"  splitOnNumerics="0" splitOnCaseChange="0" generateWordParts="1" generateNumberParts="1" catenateWords="0" catenateNumbers="0" catenateAll="0"/>
+      </analyzer>
+      <analyzer type="query">
+        <tokenizer class="solr.MockTokenizerFactory"/>
+        <filter class="solr.LowerCaseFilterFactory"/>
+      </analyzer>
+    </fieldtype>
+
+
+    <!-- more flexible in matching skus, but more chance of a false match -->
+    <fieldtype name="skutype1" class="solr.TextField">
+      <analyzer type="index">
+        <tokenizer class="solr.MockTokenizerFactory"/>
+        <filter class="solr.WordDelimiterFilterFactory" generateWordParts="1" generateNumberParts="1" catenateWords="1" catenateNumbers="1" catenateAll="0"/>
+        <filter class="solr.LowerCaseFilterFactory"/>
+      </analyzer>
+      <analyzer type="query">
+        <tokenizer class="solr.MockTokenizerFactory"/>
+        <filter class="solr.WordDelimiterFilterFactory" generateWordParts="0" generateNumberParts="0" catenateWords="1" catenateNumbers="1" catenateAll="0"/>
+        <filter class="solr.LowerCaseFilterFactory"/>
+      </analyzer>
+    </fieldtype>
+
+    <!-- less flexible in matching skus, but less chance of a false match -->
+    <fieldtype name="skutype2" class="solr.TextField">
+      <analyzer type="index">
+        <tokenizer class="solr.MockTokenizerFactory"/>
+        <filter class="solr.WordDelimiterFilterFactory" generateWordParts="0" generateNumberParts="0" catenateWords="1" catenateNumbers="1" catenateAll="0"/>
+        <filter class="solr.LowerCaseFilterFactory"/>
+      </analyzer>
+      <analyzer type="query">
+        <tokenizer class="solr.MockTokenizerFactory"/>
+        <filter class="solr.WordDelimiterFilterFactory" generateWordParts="0" generateNumberParts="0" catenateWords="1" catenateNumbers="1" catenateAll="0"/>
+        <filter class="solr.LowerCaseFilterFactory"/>
+      </analyzer>
+    </fieldtype>
+
+    <!-- less flexible in matching skus, but less chance of a false match -->
+    <fieldtype name="syn" class="solr.TextField">
+      <analyzer>
+        <tokenizer class="solr.MockTokenizerFactory"/>
+      </analyzer>
+    </fieldtype>
+
+
+    <fieldtype  name="unstored" class="solr.StrField" indexed="true" stored="false"/>
+
+
+    <fieldtype name="textgap" class="solr.TextField" multiValued="true" positionIncrementGap="100">
+      <analyzer>
+        <tokenizer class="solr.MockTokenizerFactory"/>
+        <filter class="solr.LowerCaseFilterFactory"/>
+      </analyzer>
+    </fieldtype>
+
+    <fieldType name="uuid" class="solr.UUIDField" />
+
+    <!-- Try out some point types -->
+    <fieldType name="xy" class="solr.PointType" dimension="2" subFieldType="double"/>
+    <fieldType name="x" class="solr.PointType" dimension="1" subFieldType="double"/>
+    <fieldType name="tenD" class="solr.PointType" dimension="10" subFieldType="double"/>
+    <!-- Use the sub field suffix -->
+    <fieldType name="xyd" class="solr.PointType" dimension="2" subFieldSuffix="_d1"/>
+    <fieldtype name="geohash" class="solr.GeoHashField"/>
+
+
+    <fieldType name="latLon" class="solr.LatLonType" subFieldType="double"/>
+
+    <!--  some per-field similarity examples -->
+
+    <!--  specify a Similarity classname directly -->
+    <!--
+    <fieldType name="sim1" class="solr.TextField">
+      <analyzer>
+        <tokenizer class="solr.MockTokenizerFactory"/>
+      </analyzer>
+      <similarity class="org.apache.lucene.misc.SweetSpotSimilarity"/>
+    </fieldType>
+    -->
+    <!--  specify a Similarity factory -->
+    <!--
+    <fieldType name="sim2" class="solr.TextField">
+      <analyzer>
+        <tokenizer class="solr.MockTokenizerFactory"/>
+      </analyzer>
+      <similarity class="org.apache.solr.search.similarities.CustomSimilarityFactory">
+        <str name="echo">is there an echo?</str>
+      </similarity>
+    </fieldType>
+    -->
+    <!-- don't specify any sim at all: get the default  -->
+    <!--
+    <fieldType name="sim3" class="solr.TextField">
+      <analyzer>
+        <tokenizer class="solr.MockTokenizerFactory"/>
+      </analyzer>
+    </fieldType>
+    -->
+  </types>
+
+
+  <fields>
+    <field name="id" type="int" indexed="true" stored="true" multiValued="false" required="false"/>
+    <field name="signatureField" type="string" indexed="true" stored="false"/>
+    <field name="uuid" type="uuid" stored="true" />
+    <field name="name" type="nametext" indexed="true" stored="true"/>
+    <field name="text" type="text" indexed="true" stored="false"/>
+    <field name="subject" type="text" indexed="true" stored="true"/>
+    <field name="title" type="nametext" indexed="true" stored="true"/>
+    <field name="weight" type="float" indexed="true" stored="true" multiValued="false"/>
+    <field name="bday" type="date" indexed="true" stored="true" multiValued="false"/>
+
+    <field name="title_stemmed" type="text" indexed="true" stored="false"/>
+    <field name="title_lettertok" type="lettertok" indexed="true" stored="false"/>
+
+    <field name="syn" type="syn" indexed="true" stored="true"/>
+
+    <!-- to test property inheritance and overriding -->
+    <field name="shouldbeunstored" type="unstored" />
+    <field name="shouldbestored" type="unstored" stored="true"/>
+    <field name="shouldbeunindexed" type="unstored" indexed="false" stored="true"/>
+
+    <!-- Test points -->
+    <!-- Test points -->
+    <field name="home" type="xy" indexed="true" stored="true" multiValued="false"/>
+    <field name="x" type="x" indexed="true" stored="true" multiValued="false"/>
+    <field name="homed" type="xyd" indexed="true" stored="true" multiValued="false"/>
+    <field name="home_ns" type="xy" indexed="true" stored="false" multiValued="false"/>
+    <field name="work" type="xy" indexed="true" stored="true" multiValued="false"/>
+
+    <field name="home_ll" type="latLon" indexed="true" stored="true" multiValued="false"/>
+    <field name="home_gh" type="geohash" indexed="true" stored="true" multiValued="false"/>
+
+
+    <field name="point10" type="tenD" indexed="true" stored="true" multiValued="false"/>
+
+
+    <!-- test different combinations of indexed and stored -->
+    <field name="bind" type="boolean" indexed="true" stored="false"/>
+    <field name="bsto" type="boolean" indexed="false" stored="true"/>
+    <field name="bindsto" type="boolean" indexed="true" stored="true"/>
+    <field name="isto" type="int" indexed="false" stored="true"/>
+    <field name="iind" type="int" indexed="true" stored="false"/>
+    <field name="ssto" type="string" indexed="false" stored="true"/>
+    <field name="sind" type="string" indexed="true" stored="false"/>
+    <field name="sindsto" type="string" indexed="true" stored="true"/>
+
+    <!-- test combinations of term vector settings -->
+    <field name="test_basictv" type="text" termVectors="true"/>
+    <field name="test_notv" type="text" termVectors="false"/>
+    <field name="test_postv" type="text" termVectors="true" termPositions="true"/>
+    <field name="test_offtv" type="text" termVectors="true" termOffsets="true"/>
+    <field name="test_posofftv" type="text" termVectors="true"
+           termPositions="true" termOffsets="true"/>
+
+    <!-- fields to test individual tokenizers and tokenfilters -->
+    <field name="teststop" type="teststop" indexed="true" stored="true"/>
+    <field name="lowertok" type="lowertok" indexed="true" stored="true"/>
+    <field name="keywordtok" type="keywordtok" indexed="true" stored="true"/>
+    <field name="standardtok" type="standardtok" indexed="true" stored="true"/>
+    <field name="HTMLstandardtok" type="HTMLstandardtok" indexed="true" stored="true"/>
+    <field name="lettertok" type="lettertok" indexed="true" stored="true"/>
+    <field name="whitetok" type="whitetok" indexed="true" stored="true"/>
+    <field name="HTMLwhitetok" type="HTMLwhitetok" indexed="true" stored="true"/>
+    <field name="standardtokfilt" type="standardtokfilt" indexed="true" stored="true"/>
+    <field name="standardfilt" type="standardfilt" indexed="true" stored="true"/>
+    <field name="lowerfilt" type="lowerfilt" indexed="true" stored="true"/>
+    <field name="lowerfilt1" type="lowerfilt" indexed="true" stored="true"/>
+    <field name="lowerfilt1and2" type="lowerfilt" indexed="true" stored="true"/>
+    <field name="patterntok" type="patterntok" indexed="true" stored="true"/>
+    <field name="patternreplacefilt" type="patternreplacefilt" indexed="true" stored="true"/>
+    <field name="porterfilt" type="porterfilt" indexed="true" stored="true"/>
+    <field name="engporterfilt" type="engporterfilt" indexed="true" stored="true"/>
+    <field name="custengporterfilt" type="custengporterfilt" indexed="true" stored="true"/>
+    <field name="stopfilt" type="stopfilt" indexed="true" stored="true"/>
+    <field name="custstopfilt" type="custstopfilt" indexed="true" stored="true"/>
+    <field name="lengthfilt" type="lengthfilt" indexed="true" stored="true"/>
+    <field name="wdf_nocase" type="wdf_nocase" indexed="true" stored="true"/>
+    <field name="wdf_preserve" type="wdf_preserve" indexed="true" stored="true"/>
+
+    <field name="numberpartfail" type="failtype1" indexed="true" stored="true"/>
+
+    <field name="nullfirst" type="string" indexed="true" stored="true" sortMissingFirst="true" multiValued="false"/>
+
+    <field name="subword" type="subword" indexed="true" stored="true"/>
+    <field name="subword_offsets" type="subword" indexed="true" stored="true" termOffsets="true"/>
+    <field name="numericsubword" type="numericsubword" indexed="true" stored="true"/>
+    <field name="protectedsubword" type="protectedsubword" indexed="true" stored="true"/>
+
+    <field name="sku1" type="skutype1" indexed="true" stored="true"/>
+    <field name="sku2" type="skutype2" indexed="true" stored="true"/>
+
+    <field name="textgap" type="textgap" indexed="true" stored="true"/>
+
+    <field name="timestamp" type="date" indexed="true" stored="true" default="NOW" multiValued="false"/>
+    <field name="multiDefault" type="string" indexed="true" stored="true" default="muLti-Default" multiValued="true"/>
+    <field name="intDefault" type="int" indexed="true" stored="true" default="42" multiValued="false"/>
+
+    <!--
+    <field name="sim1text" type="sim1" indexed="true" stored="true"/>
+    <field name="sim2text" type="sim2" indexed="true" stored="true"/>
+    <field name="sim3text" type="sim3" indexed="true" stored="true"/>
+    -->
+
+    <field name="tlong" type="tlong" indexed="true" stored="true" />
+
+    <field name="_version_" type="long" indexed="true" stored="true"/>
+
+    <!-- Dynamic field definitions.  If a field name is not found, dynamicFields
+         will be used if the name matches any of the patterns.
+         RESTRICTION: the glob-like pattern in the name attribute must have
+         a "*" only at the start or the end.
+         EXAMPLE:  name="*_i" will match any field ending in _i (like myid_i, z_i)
+         Longer patterns will be matched first.  if equal size patterns
+         both match, the first appearing in the schema will be used.
+    -->
+    <dynamicField name="*_i"  type="int"    indexed="true"  stored="true"/>
+    <dynamicField name="*_i1"  type="int"    indexed="true" stored="true" multiValued="false"/>
+
+    <dynamicField name="*_s"  type="string"  indexed="true"  stored="true"/>
+    <dynamicField name="*_s1"  type="string"  indexed="true"  stored="true" multiValued="false"/>
+    <dynamicField name="*_l"  type="long"   indexed="true"  stored="true"/>
+    <dynamicField name="*_l1"  type="long"   indexed="true"  stored="true" multiValued="false"/>
+    <dynamicField name="*_t"  type="text"    indexed="true"  stored="true"/>
+    <dynamicField name="*_b"  type="boolean" indexed="true"  stored="true"/>
+    <dynamicField name="*_f"  type="float"  indexed="true"  stored="true"/>
+    <dynamicField name="*_f1"  type="float"  indexed="true"  stored="true" multiValued="false"/>
+    <dynamicField name="*_d"  type="double" indexed="true"  stored="true"/>
+    <dynamicField name="*_d1"  type="double" indexed="true"  stored="true" multiValued="false"/>
+    <dynamicField name="*_dt" type="date"    indexed="true"  stored="true"/>
+    <dynamicField name="*_dt1" type="date"    indexed="true"  stored="true" multiValued="false"/>
+
+    <!-- some trie-coded dynamic fields for faster range queries -->
+    <dynamicField name="*_ti" type="tint"    indexed="true"  stored="true"/>
+    <dynamicField name="*_ti1" type="tint"    indexed="true"  stored="true" multiValued="false"/>
+    <dynamicField name="*_tl" type="tlong"   indexed="true"  stored="true"/>
+    <dynamicField name="*_tl1" type="tlong"   indexed="true"  stored="true" multiValued="false"/>
+    <dynamicField name="*_tf" type="tfloat"  indexed="true"  stored="true"/>
+    <dynamicField name="*_tf1" type="tfloat"  indexed="true"  stored="true" multiValued="false"/>
+    <dynamicField name="*_td" type="tdouble" indexed="true"  stored="true"/>
+    <dynamicField name="*_td1" type="tdouble" indexed="true" stored="true" multiValued="false"/>
+    <dynamicField name="*_tds" type="tdouble" indexed="true" stored="true" multiValued="false"/>
+    <dynamicField name="*_tdt" type="tdate"  indexed="true"  stored="true"/>
+    <dynamicField name="*_tdt1" type="tdate"  indexed="true"  stored="true" multiValued="false"/>
+
+
+    <dynamicField name="*_sI" type="string"  indexed="true"  stored="false"/>
+    <dynamicField name="*_sS" type="string"  indexed="false" stored="true"/>
+    <dynamicField name="t_*"  type="text"    indexed="true"  stored="true"/>
+    <dynamicField name="tv_*"  type="text" indexed="true"  stored="true"
+                  termVectors="true" termPositions="true" termOffsets="true"/>
+    <dynamicField name="tv_mv_*"  type="text" indexed="true"  stored="true" multiValued="true"
+                  termVectors="true" termPositions="true" termOffsets="true"/>
+
+    <dynamicField name="*_p"  type="xyd" indexed="true"  stored="true" multiValued="false"/>
+
+    <!-- special fields for dynamic copyField test -->
+    <dynamicField name="dynamic_*" type="string" indexed="true" stored="true"/>
+    <dynamicField name="*_dynamic" type="string" indexed="true" stored="true"/>
+
+    <!-- for testing to ensure that longer patterns are matched first -->
+    <dynamicField name="*aa"  type="string"  indexed="true" stored="true"/>
+
+    <!-- ignored becuase not stored or indexed -->
+    <dynamicField name="*_ignored" type="text" indexed="false" stored="false"/>
+
+    <dynamicField name="*_mfacet" type="string" indexed="true" stored="false" multiValued="true" />
+
+    <!-- make sure custom sims work with dynamic fields -->
+    <!--
+    <dynamicField name="*_sim1" type="sim1" indexed="true" stored="true"/>
+    <dynamicField name="*_sim2" type="sim2" indexed="true" stored="true"/>
+    <dynamicField name="*_sim3" type="sim3" indexed="true" stored="true"/>
+    -->
+  </fields>
+
+  <defaultSearchField>text</defaultSearchField>
+  <uniqueKey>id</uniqueKey>
+
+  <!-- copyField commands copy one field to another at the time a document
+        is added to the index.  It's used either to index the same field different
+        ways, or to add multiple fields to the same field for easier/faster searching.
+   -->
+  <copyField source="title" dest="title_stemmed"/>
+  <copyField source="title" dest="title_lettertok"/>
+
+  <copyField source="title" dest="text"/>
+  <copyField source="subject" dest="text"/>
+
+  <copyField source="lowerfilt1" dest="lowerfilt1and2"/>
+  <copyField source="lowerfilt" dest="lowerfilt1and2"/>
+
+  <copyField source="*_t" dest="text"/>
+
+
+
+  <!-- dynamic destination -->
+  <copyField source="*_dynamic" dest="dynamic_*"/>
+
+</schema>

Added: lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/solrconfig-hash.xml
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/solrconfig-hash.xml?rev=1665391&view=auto
==============================================================================
--- lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/solrconfig-hash.xml (added)
+++ lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/solrconfig-hash.xml Tue Mar 10 01:38:57 2015
@@ -0,0 +1,83 @@
+<?xml version="1.0" encoding="UTF-8" ?>
+<!--
+ 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.
+-->
+
+<!--
+ This is a stripped down config file used for a simple example...
+ It is *not* a good example to work from.
+-->
+<config>
+  <luceneMatchVersion>${tests.luceneMatchVersion:LUCENE_CURRENT}</luceneMatchVersion>
+  <indexConfig>
+    <useCompoundFile>${useCompoundFile:false}</useCompoundFile>
+  </indexConfig>
+  <dataDir>${solr.data.dir:}</dataDir>
+  <directoryFactory name="DirectoryFactory" class="${solr.directoryFactory:solr.StandardDirectoryFactory}"/>
+
+  <updateHandler class="solr.DirectUpdateHandler2">
+    <updateLog>
+      <str name="dir">${solr.data.dir:}</str>
+    </updateLog>
+  </updateHandler>
+
+  <!-- realtime get handler, guaranteed to return the latest stored fields
+    of any document, without the need to commit or open a new searcher. The current
+    implementation relies on the updateLog feature being enabled. -->
+  <requestHandler name="/get" class="solr.RealTimeGetHandler">
+    <lst name="defaults">
+      <str name="omitHeader">true</str>
+    </lst>
+  </requestHandler>
+
+  <!--
+   Distributed Stream processing.
+ -->
+
+  <requestHandler name="/stream" class="solr.StreamHandler">
+    <lst name="invariants">
+      <str name="wt">json</str>
+      <str name="distrib">false</str>
+    </lst>
+  </requestHandler>
+
+  <requestDispatcher handleSelect="true" >
+    <requestParsers enableRemoteStreaming="false" multipartUploadLimitInKB="2048" />
+  </requestDispatcher>
+
+  <requestHandler name="/replication" class="solr.ReplicationHandler" startup="lazy" />
+
+  <requestHandler name="standard" class="solr.StandardRequestHandler" default="true" />
+  <requestHandler name="/update" class="solr.UpdateRequestHandler"  />
+  <requestHandler name="/admin/" class="org.apache.solr.handler.admin.AdminHandlers" />
+
+  <requestHandler name="/admin/ping" class="solr.PingRequestHandler">
+    <lst name="invariants">
+      <str name="q">*:*</str>
+    </lst>
+    <lst name="defaults">
+      <str name="echoParams">all</str>
+    </lst>
+    <str name="healthcheckFile">server-enabled.txt</str>
+  </requestHandler>
+
+  <!-- config for the admin interface -->
+  <admin>
+    <defaultQuery>solr</defaultQuery>
+  </admin>
+
+</config>
+

Modified: lucene/dev/trunk/solr/core/src/test/org/apache/solr/search/QueryEqualityTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/search/QueryEqualityTest.java?rev=1665391&r1=1665390&r2=1665391&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/search/QueryEqualityTest.java (original)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/search/QueryEqualityTest.java Tue Mar 10 01:38:57 2015
@@ -268,6 +268,19 @@ public class QueryEqualityTest extends S
     }
   }
 
+
+  public void testHash() throws Exception {
+    SolrQueryRequest req = req("partitionKeys","foo_s");
+
+    try {
+      assertQueryEquals("hash", req,
+          "{!hash workers=3 worker=0}");
+
+    } finally {
+      req.close();
+    }
+  }
+
   public void testQueryNested() throws Exception {
     SolrQueryRequest req = req("df", "foo_s");
     try {

Added: lucene/dev/trunk/solr/core/src/test/org/apache/solr/search/TestHashQParserPlugin.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/search/TestHashQParserPlugin.java?rev=1665391&view=auto
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/search/TestHashQParserPlugin.java (added)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/search/TestHashQParserPlugin.java Tue Mar 10 01:38:57 2015
@@ -0,0 +1,252 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.search;
+
+import org.apache.lucene.util.FixedBitSet;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.response.SolrQueryResponse;
+import org.apache.lucene.util.BytesRef;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.*;
+import java.util.Random;
+
+@LuceneTestCase.SuppressCodecs({"Lucene3x", "Lucene40","Lucene41","Lucene42","Lucene45"})
+public class TestHashQParserPlugin extends SolrTestCaseJ4 {
+
+  @BeforeClass
+  public static void beforeClass() throws Exception {
+    initCore("solrconfig-hash.xml", "schema-hash.xml");
+  }
+
+  @Override
+  @Before
+  public void setUp() throws Exception {
+    // if you override setUp or tearDown, you better call
+    // the super classes version
+    super.setUp();
+    clearIndex();
+    assertU(commit());
+  }
+
+
+  public int getCost(Random random) {
+    int i = random.nextInt(2);
+    if(i == 0) {
+      return 200;
+    } else {
+      return 1;
+    }
+  }
+
+
+  @Test
+  public void testHashPartition() throws Exception {
+
+
+    Random random = random();
+    HashSet<String> set = new HashSet();
+
+    for(int i=0; i<50; i++) {
+      int v = random.nextInt(1000000);
+      String val = Integer.toString(v);
+      if(!set.contains(val)){
+        set.add(val);
+        String[] doc = {"id", val, "a_s", val, "a_i", val, "a_l", val};
+        assertU(adoc(doc));
+        if(i % 10 == 0)
+        assertU(commit());
+
+      }
+    }
+    assertU(commit());
+
+
+    //Test with 3 worker and String hash ID.
+
+    ModifiableSolrParams params = new ModifiableSolrParams();
+    params.add("q", "*:*");
+    params.add("fq", "{!hash worker=0 workers=3 cost="+getCost(random)+"}");
+    params.add("partitionKeys", "a_s");
+    params.add("rows","50");
+    HashSet set1 = new HashSet();
+    String response = h.query(req(params));
+
+    Iterator<String> it = set.iterator();
+
+    while(it.hasNext()) {
+      String s = it.next();
+      String results = h.validateXPath(response, "*[count(//int[@name='id'][.='"+s+"'])=1]");
+      if(results == null) {
+        set1.add(s);
+      }
+    }
+
+    params = new ModifiableSolrParams();
+    params.add("q", "*:*");
+    params.add("fq", "{!hash worker=1 workers=3 cost="+getCost(random)+"}");
+    params.add("partitionKeys", "a_s");
+    params.add("rows","50");
+    HashSet set2 = new HashSet();
+    response = h.query(req(params));
+
+    it = set.iterator();
+
+    while(it.hasNext()) {
+      String s = it.next();
+      String results = h.validateXPath(response, "*[count(//int[@name='id'][.='"+s+"'])=1]");
+      if(results == null) {
+        set2.add(s);
+      }
+    }
+
+
+    params = new ModifiableSolrParams();
+    params.add("q", "*:*");
+    params.add("fq", "{!hash worker=2 workers=3 cost="+getCost(random)+"}");
+    params.add("partitionKeys", "a_s");
+    params.add("rows","50");
+    HashSet set3 = new HashSet();
+    response = h.query(req(params));
+
+    it = set.iterator();
+
+    while(it.hasNext()) {
+      String s = it.next();
+      String results = h.validateXPath(response, "*[count(//int[@name='id'][.='"+s+"'])=1]");
+      if(results == null) {
+        set3.add(s);
+      }
+    }
+
+    assert(set1.size() > 0);
+    assert(set2.size() > 0);
+    assert(set3.size() > 0);
+    assert(set1.size()+set2.size()+set3.size()==set.size());
+    assertNoOverLap(set1, set2);
+    assertNoOverLap(set1, set3);
+    assertNoOverLap(set2, set3);
+
+
+    //Test with 2 workers and int partition Key
+
+
+    params = new ModifiableSolrParams();
+    params.add("q", "*:*");
+    params.add("fq", "{!hash worker=0 workers=2 cost="+getCost(random)+"}");
+    params.add("partitionKeys", "a_i");
+    params.add("rows","50");
+    set1 = new HashSet();
+    response = h.query(req(params));
+
+    it = set.iterator();
+
+    while(it.hasNext()) {
+      String s = it.next();
+      String results = h.validateXPath(response, "*[count(//int[@name='id'][.='"+s+"'])=1]");
+      if(results == null) {
+        set1.add(s);
+      }
+    }
+
+    params = new ModifiableSolrParams();
+    params.add("q", "*:*");
+    params.add("fq", "{!hash worker=1 workers=2 cost="+getCost(random)+"}");
+    params.add("partitionKeys", "a_i");
+    params.add("rows","50");
+    set2 = new HashSet();
+    response = h.query(req(params));
+
+    it = set.iterator();
+
+    while(it.hasNext()) {
+      String s = it.next();
+      String results = h.validateXPath(response, "*[count(//int[@name='id'][.='"+s+"'])=1]");
+      if(results == null) {
+        set2.add(s);
+      }
+    }
+
+    assert(set1.size() > 0);
+    assert(set2.size() > 0);
+    assert(set1.size()+set2.size()==set.size());
+    assertNoOverLap(set1, set2);
+
+
+    //Test with 2 workers and compound partition Key
+
+
+    params = new ModifiableSolrParams();
+    params.add("q", "*:*");
+    params.add("fq", "{!hash worker=0 workers=2 cost="+getCost(random)+"}");
+    params.add("partitionKeys", "a_s,a_i,a_l");
+    params.add("rows","50");
+    set1 = new HashSet();
+    response = h.query(req(params));
+
+    it = set.iterator();
+
+    while(it.hasNext()) {
+      String s = it.next();
+      String results = h.validateXPath(response, "*[count(//int[@name='id'][.='"+s+"'])=1]");
+      if(results == null) {
+        set1.add(s);
+      }
+    }
+
+    params = new ModifiableSolrParams();
+    params.add("q", "*:*");
+    params.add("fq", "{!hash worker=1 workers=2 cost="+getCost(random)+"}");
+    params.add("partitionKeys", "a_s,a_i,a_l");
+    params.add("rows","50");
+    set2 = new HashSet();
+    response = h.query(req(params));
+
+    it = set.iterator();
+
+    while(it.hasNext()) {
+      String s = it.next();
+      String results = h.validateXPath(response, "*[count(//int[@name='id'][.='"+s+"'])=1]");
+      if(results == null) {
+        set2.add(s);
+      }
+    }
+
+    assert(set1.size() > 0);
+    assert(set2.size() > 0);
+    assert(set1.size()+set2.size()==set.size());
+    assertNoOverLap(set1, set2);
+  }
+
+
+  private void assertNoOverLap(Set setA, Set setB) throws Exception {
+    Iterator it =  setA.iterator();
+    while(it.hasNext()) {
+      Object o = it.next();
+      if(setB.contains(o)) {
+        throw new Exception("Overlapping sets for value:"+o.toString());
+      }
+    }
+  }
+}

Modified: lucene/dev/trunk/solr/server/solr/configsets/data_driven_schema_configs/conf/solrconfig.xml
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/server/solr/configsets/data_driven_schema_configs/conf/solrconfig.xml?rev=1665391&r1=1665390&r2=1665391&view=diff
==============================================================================
--- lucene/dev/trunk/solr/server/solr/configsets/data_driven_schema_configs/conf/solrconfig.xml (original)
+++ lucene/dev/trunk/solr/server/solr/configsets/data_driven_schema_configs/conf/solrconfig.xml Tue Mar 10 01:38:57 2015
@@ -893,6 +893,36 @@
     </lst>
   </requestHandler>
 
+  <!--
+    The export request handler is used to export full sorted result sets.
+    Do not change these defaults.
+  -->
+
+  <requestHandler name="/export" class="solr.SearchHandler">
+    <lst name="invariants">
+      <str name="rq">{!xport}</str>
+      <str name="wt">xsort</str>
+      <str name="distrib">false</str>
+    </lst>
+
+    <arr name="components">
+      <str>query</str>
+    </arr>
+  </requestHandler>
+
+
+  <!--
+  Distributed Stream processing.
+  -->
+
+  <requestHandler name="/stream" class="solr.StreamHandler">
+    <lst name="invariants">
+      <str name="wt">json</str>
+      <str name="distrib">false</str>
+    </lst>
+  </requestHandler>
+
+
 
   <!-- Field Analysis Request Handler
 

Modified: lucene/dev/trunk/solr/server/solr/configsets/sample_techproducts_configs/conf/solrconfig.xml
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/server/solr/configsets/sample_techproducts_configs/conf/solrconfig.xml?rev=1665391&r1=1665390&r2=1665391&view=diff
==============================================================================
--- lucene/dev/trunk/solr/server/solr/configsets/sample_techproducts_configs/conf/solrconfig.xml (original)
+++ lucene/dev/trunk/solr/server/solr/configsets/sample_techproducts_configs/conf/solrconfig.xml Tue Mar 10 01:38:57 2015
@@ -905,6 +905,18 @@
   </requestHandler>
 
 
+  <!--
+  Distributed Stream processing.
+  -->
+
+  <requestHandler name="/stream" class="solr.StreamHandler">
+    <lst name="invariants">
+      <str name="wt">json</str>
+      <str name="distrib">false</str>
+    </lst>
+  </requestHandler>
+
+
 
 
 

Modified: lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpSolrClient.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpSolrClient.java?rev=1665391&r1=1665390&r2=1665391&view=diff
==============================================================================
--- lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpSolrClient.java (original)
+++ lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpSolrClient.java Tue Mar 10 01:38:57 2015
@@ -520,7 +520,7 @@ public class HttpSolrClient extends Solr
                 null);
           }
       }
-      if (processor == null) {
+      if (processor == null || processor instanceof InputStreamResponseParser) {
         
         // no processor specified, return raw stream
         NamedList<Object> rsp = new NamedList<>();

Added: lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/client/solrj/impl/InputStreamResponseParser.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/client/solrj/impl/InputStreamResponseParser.java?rev=1665391&view=auto
==============================================================================
--- lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/client/solrj/impl/InputStreamResponseParser.java (added)
+++ lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/client/solrj/impl/InputStreamResponseParser.java Tue Mar 10 01:38:57 2015
@@ -0,0 +1,57 @@
+package org.apache.solr.client.solrj.impl;
+
+/*
+ * 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.
+ */
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.Reader;
+import java.io.StringWriter;
+
+import org.apache.commons.io.IOUtils;
+import org.apache.solr.client.solrj.ResponseParser;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.util.NamedList;
+
+/**
+ * Simply puts the InputStream into an entry in a NamedList named "stream".
+ */
+public class InputStreamResponseParser extends ResponseParser {
+
+  private final String writerType;
+
+  public InputStreamResponseParser(String writerType) {
+    this.writerType = writerType;
+  }
+
+  @Override
+  public String getWriterType() {
+    return writerType;
+  }
+
+  @Override
+  public NamedList<Object> processResponse(Reader reader) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public NamedList<Object> processResponse(InputStream body, String encoding) {
+    throw new UnsupportedOperationException();
+  }
+
+}
+

Added: lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/client/solrj/io/AscBucketComp.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/client/solrj/io/AscBucketComp.java?rev=1665391&view=auto
==============================================================================
--- lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/client/solrj/io/AscBucketComp.java (added)
+++ lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/client/solrj/io/AscBucketComp.java Tue Mar 10 01:38:57 2015
@@ -0,0 +1,42 @@
+package org.apache.solr.client.solrj.io;
+
+/*
+ * 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.
+ */
+
+import java.io.Serializable;
+import java.util.Comparator;
+
+public class AscBucketComp implements Comparator<BucketMetrics>, Serializable {
+
+  private int ord;
+
+  public AscBucketComp(int ord) {
+    this.ord = ord;
+  }
+
+  public int compare(BucketMetrics b1, BucketMetrics b2) {
+    double d1 = b1.getMetrics()[ord].getValue();
+    double d2 = b2.getMetrics()[ord].getValue();
+    if(d1 > d2) {
+      return 1;
+    } else if(d1 < d2) {
+      return -1;
+    } else {
+      return 0;
+    }
+  }
+}
\ No newline at end of file

Added: lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/client/solrj/io/AscFieldComp.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/client/solrj/io/AscFieldComp.java?rev=1665391&view=auto
==============================================================================
--- lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/client/solrj/io/AscFieldComp.java (added)
+++ lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/client/solrj/io/AscFieldComp.java Tue Mar 10 01:38:57 2015
@@ -0,0 +1,38 @@
+/*
+ * 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.client.solrj.io;
+
+import java.io.Serializable;
+import java.util.Comparator;
+
+public class AscFieldComp implements Comparator<Tuple>, Serializable {
+
+  private static final long serialVersionUID = 1;
+
+  private String field;
+
+  public AscFieldComp(String field) {
+    this.field = field;
+  }
+
+  public int compare(Tuple t1, Tuple t2) {
+    Comparable o1 = (Comparable)t1.get(field);
+    Comparable o2 = (Comparable)t2.get(field);
+    return o1.compareTo(o2);
+  }
+}
\ No newline at end of file

Added: lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/client/solrj/io/AscMetricComp.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/client/solrj/io/AscMetricComp.java?rev=1665391&view=auto
==============================================================================
--- lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/client/solrj/io/AscMetricComp.java (added)
+++ lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/client/solrj/io/AscMetricComp.java Tue Mar 10 01:38:57 2015
@@ -0,0 +1,39 @@
+package org.apache.solr.client.solrj.io;
+
+/*
+ * 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.
+ */
+
+import java.io.Serializable;
+import java.util.Comparator;
+import java.util.List;
+
+public class AscMetricComp implements Comparator<Tuple>, Serializable {
+
+  private static final long serialVersionUID = 1;
+
+  private int ord;
+
+  public AscMetricComp(int ord) {
+    this.ord = ord;
+  }
+
+  public int compare(Tuple t1, Tuple t2) {
+    List<Double> values1 = (List<Double>)t1.get("metricValues");
+    List<Double> values2 = (List<Double>)t2.get("metricValues");
+    return values1.get(ord).compareTo(values2.get(ord));
+  }
+}
\ No newline at end of file

Added: lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/client/solrj/io/Bucket.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/client/solrj/io/Bucket.java?rev=1665391&view=auto
==============================================================================
--- lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/client/solrj/io/Bucket.java (added)
+++ lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/client/solrj/io/Bucket.java Tue Mar 10 01:38:57 2015
@@ -0,0 +1,43 @@
+/*
+ * 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.client.solrj.io;
+
+/*
+* 
+*/
+
+import java.io.Serializable;
+
+public class Bucket implements Serializable {
+
+  private static final long serialVersionUID = 1;
+
+  private String bucketKey;
+
+  public Bucket() {
+
+  }
+
+  public Bucket(String bucketKey) {
+    this.bucketKey = bucketKey;
+  }
+
+  public String getBucketValue(Tuple tuple) {
+    return tuple.get(bucketKey).toString();
+  }
+}

Added: lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/client/solrj/io/BucketMetrics.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/client/solrj/io/BucketMetrics.java?rev=1665391&view=auto
==============================================================================
--- lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/client/solrj/io/BucketMetrics.java (added)
+++ lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/client/solrj/io/BucketMetrics.java Tue Mar 10 01:38:57 2015
@@ -0,0 +1,43 @@
+package org.apache.solr.client.solrj.io;
+
+/*
+ * 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.
+ */
+
+import java.io.Serializable;
+
+public class BucketMetrics implements Serializable {
+
+  private static final long serialVersionUID = 1;
+
+  private HashKey key;
+  private Metric[] metrics;
+
+  public BucketMetrics(HashKey key, Metric[] metrics) {
+    this.key = key;
+    this.metrics = metrics;
+  }
+
+  public Metric[] getMetrics() {
+    return metrics;
+  }
+
+  public HashKey getKey() {
+    return key;
+  }
+
+
+}
\ No newline at end of file