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