SOLR-7082: Streaming Aggregation for SolrCloud

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1665391 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Joel Bernstein 2015-03-10 01:38:57 +00:00
parent b4ee7b6eb4
commit 9563a30751
50 changed files with 6721 additions and 2 deletions

View File

@ -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;
}
}

View File

@ -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;

View File

@ -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);
}
}
}

View File

@ -72,6 +72,7 @@ public abstract class QParserPlugin implements NamedListInitializedPlugin, SolrI
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);
}

View File

@ -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>

View File

@ -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>

View File

@ -268,6 +268,19 @@ public class QueryEqualityTest extends SolrTestCaseJ4 {
}
}
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 {

View File

@ -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());
}
}
}
}

View File

@ -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

View File

@ -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>

View File

@ -520,7 +520,7 @@ public class HttpSolrClient extends SolrClient {
null);
}
}
if (processor == null) {
if (processor == null || processor instanceof InputStreamResponseParser) {
// no processor specified, return raw stream
NamedList<Object> rsp = new NamedList<>();

View File

@ -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();
}
}

View File

@ -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;
}
}
}

View File

@ -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);
}
}

View File

@ -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));
}
}

View File

@ -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();
}
}

View File

@ -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;
}
}

View File

@ -0,0 +1,289 @@
/*
* 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.IOException;
import java.util.Map;
import java.util.Comparator;
import java.util.Random;
import java.util.TreeSet;
import java.util.List;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.concurrent.Callable;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.Future;
import java.util.concurrent.TimeoutException;
import org.apache.solr.client.solrj.impl.CloudSolrClient;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.cloud.Replica;
import org.apache.solr.common.cloud.ZkCoreNodeProps;
import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.cloud.ClusterState;
import org.apache.solr.common.cloud.ZooKeeperException;
import org.apache.solr.common.cloud.Slice;
import org.apache.solr.common.util.SolrjNamedThreadFactory;
import org.apache.zookeeper.KeeperException;
/**
* Connects to Zookeeper to pick replicas from a specific collection to send the query to.
* SolrStream instances are used to send the query to the replicas.
* SolrStreams are opened using a Thread pool, but a single thread is used to iterate through each stream's tuples.* *
**/
public class CloudSolrStream extends TupleStream {
private static final long serialVersionUID = 1;
protected String zkHost;
protected String collection;
protected Map params;
private Map<String, String> fieldMappings;
protected TreeSet<TupleWrapper> tuples;
protected Comparator<Tuple> comp;
protected List<TupleStream> solrStreams = new ArrayList();
private int zkConnectTimeout = 10000;
private int zkClientTimeout = 10000;
protected transient SolrClientCache cache;
protected transient CloudSolrClient cloudSolrClient;
private int numWorkers;
private int workerID;
protected Map<String, Tuple> eofTuples = new HashMap();
public CloudSolrStream(String zkHost, String collection, Map params) {
this.zkHost = zkHost;
this.collection = collection;
this.params = params;
this.tuples = new TreeSet();
String sort = (String)params.get("sort");
this.comp = parseComp(sort);
}
//Used by the ParallelStream
protected CloudSolrStream() {
}
public void setComp(Comparator<Tuple> comp) {
this.comp = comp;
}
public void setFieldMappings(Map<String, String> fieldMappings) {
this.fieldMappings = fieldMappings;
}
public void setStreamContext(StreamContext context) {
this.numWorkers = context.numWorkers;
this.workerID = context.workerID;
this.cache = context.clientCache;
}
public void open() throws IOException {
if(this.cache != null) {
this.cloudSolrClient = this.cache.getCloudSolrClient(zkHost);
} else {
this.cloudSolrClient = new CloudSolrClient(zkHost);
this.cloudSolrClient.connect();
}
constructStreams();
openStreams();
}
public List<TupleStream> children() {
return solrStreams;
}
private Comparator<Tuple> parseComp(String sort) {
String[] sorts = sort.split(",");
Comparator[] comps = new Comparator[sorts.length];
for(int i=0; i<sorts.length; i++) {
String s = sorts[i];
String[] spec = s.split(" ");
if(spec[1].trim().equalsIgnoreCase("asc")) {
comps[i] = new AscFieldComp(spec[0]);
} else {
comps[i] = new DescFieldComp(spec[0]);
}
}
if(comps.length > 1) {
return new MultiComp(comps);
} else {
return comps[0];
}
}
protected void constructStreams() throws IOException {
try {
ZkStateReader zkStateReader = cloudSolrClient.getZkStateReader();
ClusterState clusterState = zkStateReader.getClusterState();
//System.out.println("Connected to zk an got cluster state.");
Collection<Slice> slices = clusterState.getActiveSlices(this.collection);
long time = System.currentTimeMillis();
params.put("distrib","false"); // We are the aggregator.
for(Slice slice : slices) {
Collection<Replica> replicas = slice.getReplicas();
List<Replica> shuffler = new ArrayList();
for(Replica replica : replicas) {
shuffler.add(replica);
}
Collections.shuffle(shuffler, new Random(time));
Replica rep = shuffler.get(0);
ZkCoreNodeProps zkProps = new ZkCoreNodeProps(rep);
String url = zkProps.getCoreUrl();
SolrStream solrStream = new SolrStream(url, params);
StreamContext context = new StreamContext();
context.numWorkers = this.numWorkers;
context.workerID = this.workerID;
solrStream.setStreamContext(context);
solrStream.setFieldMappings(this.fieldMappings);
solrStreams.add(solrStream);
}
} catch (Exception e) {
throw new IOException(e);
}
}
private void openStreams() throws IOException {
ExecutorService service = Executors.newCachedThreadPool(new SolrjNamedThreadFactory("CloudSolrStream"));
List<Future<TupleWrapper>> futures = new ArrayList();
for(TupleStream solrStream : solrStreams) {
StreamOpener so = new StreamOpener((SolrStream)solrStream, comp);
Future<TupleWrapper> future = service.submit(so);
futures.add(future);
}
try {
for(Future<TupleWrapper> f : futures) {
TupleWrapper w = f.get();
if(w != null) {
tuples.add(w);
}
}
} catch (Exception e) {
throw new IOException(e);
}
service.shutdown();
}
public void close() throws IOException {
for(TupleStream solrStream : solrStreams) {
solrStream.close();
}
if(cache == null) {
cloudSolrClient.close();
}
}
public Tuple read() throws IOException {
return _read();
}
protected Tuple _read() throws IOException {
TupleWrapper tw = tuples.pollFirst();
if(tw != null) {
Tuple t = tw.getTuple();
if(tw.next()) {
tuples.add(tw);
}
return t;
} else {
Map m = new HashMap();
m.put("EOF", true);
return new Tuple(m);
}
}
protected class TupleWrapper implements Comparable<TupleWrapper> {
private Tuple tuple;
private SolrStream stream;
private Comparator comp;
public TupleWrapper(SolrStream stream, Comparator comp) {
this.stream = stream;
this.comp = comp;
}
public int compareTo(TupleWrapper w) {
if(this == w) {
return 0;
}
int i = comp.compare(tuple, w.tuple);
if(i == 0) {
return 1;
} else {
return i;
}
}
public boolean equals(Object o) {
return this == o;
}
public Tuple getTuple() {
return tuple;
}
public boolean next() throws IOException {
this.tuple = stream.read();
if(tuple.EOF) {
eofTuples.put(stream.getBaseUrl(), tuple);
}
return !tuple.EOF;
}
}
protected class StreamOpener implements Callable<TupleWrapper> {
private SolrStream stream;
private Comparator<Tuple> comp;
public StreamOpener(SolrStream stream, Comparator<Tuple> comp) {
this.stream = stream;
this.comp = comp;
}
public TupleWrapper call() throws Exception {
stream.open();
TupleWrapper wrapper = new TupleWrapper(stream, comp);
if(wrapper.next()) {
return wrapper;
} else {
return null;
}
}
}
}

View File

@ -0,0 +1,58 @@
/*
* 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.Map;
import java.util.HashMap;
public class CountMetric implements Metric, Serializable {
private static final long serialVersionUID = 1;
public static final String COUNT = "count";
private long count;
public String getName() {
return "count";
}
public void update(Tuple tuple) {
++count;
}
public double getValue() {
return count;
}
public Metric newInstance() {
return new CountMetric();
}
public Map<String, Double> metricValues() {
Map m = new HashMap();
double d = (double)count;
m.put(COUNT, d);
return m;
}
public void update(Map<String, Double> metricValues) {
double dcount = metricValues.get(COUNT);
count+=(long)dcount;
}
}

View File

@ -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 DescBucketComp implements Comparator<BucketMetrics>, Serializable {
private int ord;
public DescBucketComp(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;
}
}
}

View File

@ -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;
import java.util.Comparator;
public class DescFieldComp implements Comparator<Tuple>, Serializable {
private static final long serialVersionUID = 1;
private String field;
public DescFieldComp(String field) {
this.field = field;
}
public int compare(Tuple t1, Tuple t2) {
Comparable o1 = (Comparable)t1.get(field);
Comparable o2 = (Comparable)t2.get(field);
int i = o1.compareTo(o2);
if(i == 0) {
return i;
} else {
return -i;
}
}
}

View File

@ -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 DescMetricComp implements Comparator<Tuple>, Serializable {
private static final long serialVersionUID = 1;
private int ord;
public DescMetricComp(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))*-1;
}
}

View File

@ -0,0 +1,101 @@
/*
* 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.IOException;
import java.util.Comparator;
import java.util.List;
import java.util.ArrayList;
public class FilterStream extends TupleStream {
private static final long serialVersionUID = 1;
private TupleStream streamA;
private TupleStream streamB;
private Comparator<Tuple> comp;
private Tuple a = null;
private Tuple b = null;
/*
* Intersects streamA by streamB based on a Comparator.
* Both streams must be sorted by the fields being compared.
* StreamB must be unique for the fields being compared.
**/
public FilterStream(TupleStream streamA, TupleStream streamB, Comparator<Tuple> comp) {
this.streamA = streamA;
this.streamB = streamB;
this.comp = comp;
}
public void setStreamContext(StreamContext context) {
this.streamA.setStreamContext(context);
this.streamB.setStreamContext(context);
}
public List<TupleStream> children() {
List<TupleStream> l = new ArrayList();
l.add(streamA);
l.add(streamB);
return l;
}
public void open() throws IOException {
streamA.open();
streamB.open();
}
public void close() throws IOException {
streamA.close();
streamB.close();
}
public Tuple read() throws IOException {
a = streamA.read();
if(b == null) {
b = streamB.read();
}
while(true) {
if(a.EOF) {
return a;
}
if(b.EOF) {
return b;
}
int i = comp.compare(a, b);
if(i == 0) {
return a;
} else if(i < 0) {
// a < b so advance a
a = streamA.read();
} else {
// a > b so advance b
b = streamB.read();
}
}
}
public int getCost() {
return 0;
}
}

View File

@ -0,0 +1,142 @@
/*
* 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.IOException;
import java.io.Serializable;
import java.util.List;
import java.util.ArrayList;
import java.util.Comparator;
import java.util.PriorityQueue;
/**
* Iterates over a TupleStream Groups The TopN Tuples of a group.
**/
public class GroupByStream extends TupleStream {
private static final long serialVersionUID = 1;
private TupleStream tupleStream;
private Comparator<Tuple> interGroupComp;
private Comparator<Tuple> intraGroupComp;
private Comparator<Tuple> reverseComp;
private Tuple currentTuple;
private int size;
public GroupByStream(TupleStream tupleStream,
Comparator<Tuple> interGroupComp,
Comparator<Tuple> intraGroupComp,
int size) {
this.tupleStream = tupleStream;
this.interGroupComp = interGroupComp;
this.intraGroupComp = intraGroupComp;
this.reverseComp = new ReverseComp(intraGroupComp);
this.size = size;
}
public void setStreamContext(StreamContext context) {
this.tupleStream.setStreamContext(context);
}
public List<TupleStream> children() {
List<TupleStream> l = new ArrayList();
l.add(tupleStream);
return l;
}
public void open() throws IOException {
tupleStream.open();
currentTuple = tupleStream.read(); //Read the first Tuple so currentTuple is never null;
}
public void close() throws IOException {
tupleStream.close();
}
public Tuple read() throws IOException {
if(currentTuple.EOF) {
return currentTuple;
}
PriorityQueue<Tuple> group = new PriorityQueue<>(size, reverseComp);
group.add(currentTuple);
while(true) {
Tuple t = tupleStream.read();
if(t.EOF) {
currentTuple = t;
break;
}
if(interGroupComp.compare(currentTuple, t) == 0) {
if(group.size() >= size) {
Tuple peek = group.peek();
if(intraGroupComp.compare(t, peek) < 0) {
group.poll();
group.add(t);
}
} else {
group.add(t);
}
} else {
currentTuple = t;
break;
}
}
//We have a finished group so add the Tuples to an array.
Tuple[] members = new Tuple[group.size()];
for(int i=group.size()-1; i>=0; i--) {
Tuple t = group.poll();
members[i] = t;
}
//First Tuple is the group head.
Tuple groupHead = members[0];
if(members.length > 1) {
List groupList = new ArrayList();
for(int i=1; i<members.length; i++) {
groupList.add(members[i].fields);
}
groupHead.set("tuples", groupList);
} else {
groupHead.set("tuples", new ArrayList());
}
return groupHead;
}
public int getCost() {
return 0;
}
class ReverseComp implements Comparator<Tuple>, Serializable {
private Comparator<Tuple> comp;
public ReverseComp(Comparator<Tuple> comp) {
this.comp = comp;
}
public int compare(Tuple t1, Tuple t2) {
return comp.compare(t1, t2)*(-1);
}
}
}

View File

@ -0,0 +1,121 @@
/*
* 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.IOException;
import java.util.List;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.LinkedList;
/**
*
*
**/
public class HashJoinStream extends TupleStream {
private static final long serialVersionUID = 1;
private PushBackStream streamA;
private TupleStream streamB;
private String[] keys;
private HashMap<HashKey, List<Tuple>> hashMap = new HashMap();
public HashJoinStream(TupleStream streamA, TupleStream streamB, String[] keys) {
this.streamA = new PushBackStream(streamA);
this.streamB = streamB;
this.keys = keys;
}
public void setStreamContext(StreamContext context) {
this.streamA.setStreamContext(context);
this.streamB.setStreamContext(context);
}
public List<TupleStream> children() {
List<TupleStream> l = new ArrayList();
l.add(streamA);
l.add(streamB);
return l;
}
public void open() throws IOException {
streamB.open();
while(true) {
Tuple t = streamB.read();
if(t.EOF) {
break;
}
HashKey hashKey = new HashKey(t, keys);
if(hashMap.containsKey(hashKey)) {
List<Tuple> tuples = hashMap.get(hashKey);
tuples.add(t);
} else {
List<Tuple> tuples = new ArrayList();
tuples.add(t);
hashMap.put(hashKey, tuples);
}
}
streamB.close();
streamA.open();
}
public void close() throws IOException {
streamA.close();
}
private LinkedList<Tuple> joinTuples = new LinkedList();
public Tuple read() throws IOException {
while(true) {
Tuple tuple = streamA.read();
if(tuple.EOF) {
return tuple;
}
if(joinTuples.size() > 0) {
Tuple t = tuple.clone();
Tuple j = joinTuples.removeFirst();
t.fields.putAll(j.fields);
if(joinTuples.size() > 0) {
streamA.pushBack(tuple);
}
return t;
} else {
HashKey hashKey = new HashKey(tuple, keys);
if(hashMap.containsKey(hashKey)) {
List<Tuple> joinWith = hashMap.get(hashKey);
for(Tuple jt : joinWith) {
joinTuples.add(jt);
}
streamA.pushBack(tuple);
}
}
}
}
public int getCost() {
return 0;
}
}

View File

@ -0,0 +1,79 @@
/*
* 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 HashKey implements Serializable {
private static final long serialVersionUID = 1;
private Object[] parts;
public HashKey(String value) {
parts = (Object[])value.split("::");
}
public HashKey(Tuple t, String[] keys) {
this.parts = new Object[keys.length];
for(int i=0; i<keys.length; i++) {
parts[i] = t.get(keys[i]);
}
}
public HashKey(String[] parts) {
this.parts = parts;
}
public Object getParts() {
return parts;
}
public int hashCode() {
int h = 0;
for(Object o : parts) {
h+=o.hashCode();
}
return h;
}
public boolean equals(Object o) {
HashKey h = (HashKey)o;
for(int i=0; i<parts.length; i++) {
if(!parts[i].equals(h.parts[i])) {
return false;
}
}
return true;
}
public String toString() {
StringBuilder buf = new StringBuilder();
for(int i=0; i<parts.length; i++) {
if(i > 0) {
buf.append("::");
}
buf.append(parts[i].toString());
}
return buf.toString();
}
}

View File

@ -0,0 +1,165 @@
package org.apache.solr.client.solrj.io;
import java.io.IOException;
import java.io.InputStream;
import java.io.InputStreamReader;
import java.io.Reader;
import java.util.List;
import java.util.Map;
import org.apache.solr.client.solrj.SolrClient;
import org.apache.solr.client.solrj.SolrRequest;
import org.apache.solr.client.solrj.SolrServerException;
import org.apache.solr.client.solrj.impl.InputStreamResponseParser;
import org.apache.solr.client.solrj.request.QueryRequest;
import org.apache.solr.common.params.ModifiableSolrParams;
import org.apache.solr.common.params.SolrParams;
import org.apache.solr.common.util.NamedList;
import org.noggit.JSONParser;
import org.noggit.ObjectBuilder;
/*
* 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.
*/
/*
Queries a Solr instance, and maps SolrDocs to Tuples.
Initial version works with the json format and only SolrDocs are handled.
*/
public class JSONTupleStream {
private List<String> path; // future... for more general stream handling
private Reader reader;
private JSONParser parser;
private boolean atDocs;
public JSONTupleStream(Reader reader) {
this.reader = reader;
this.parser = new JSONParser(reader);
}
// temporary...
public static JSONTupleStream create(SolrClient server, SolrParams requestParams) throws IOException, SolrServerException {
String p = requestParams.get("qt");
if(p != null) {
ModifiableSolrParams modifiableSolrParams = (ModifiableSolrParams) requestParams;
modifiableSolrParams.remove("qt");
}
QueryRequest query = new QueryRequest( requestParams );
query.setPath(p);
query.setResponseParser(new InputStreamResponseParser("json"));
query.setMethod(SolrRequest.METHOD.POST);
NamedList<Object> genericResponse = server.request(query);
InputStream stream = (InputStream)genericResponse.get("stream");
InputStreamReader reader = new InputStreamReader(stream, "UTF-8");
return new JSONTupleStream(reader);
}
/** returns the next Tuple or null */
public Map<String,Object> next() throws IOException {
if (!atDocs) {
boolean found = advanceToDocs();
atDocs = true;
if (!found) return null;
}
// advance past ARRAY_START (in the case that we just advanced to docs, or OBJECT_END left over from the last call.
int event = parser.nextEvent();
if (event == JSONParser.ARRAY_END) return null;
Object o = ObjectBuilder.getVal(parser);
// right now, getVal will leave the last event read as OBJECT_END
return (Map<String,Object>)o;
}
public void close() throws IOException {
reader.close();
}
private void expect(int parserEventType) throws IOException {
int event = parser.nextEvent();
if (event != parserEventType) {
throw new IOException("JSONTupleStream: expected " + JSONParser.getEventString(parserEventType) + " but got " + JSONParser.getEventString(event) );
}
}
private void expect(String mapKey) {
}
private boolean advanceToMapKey(String key, boolean deepSearch) throws IOException {
for (;;) {
int event = parser.nextEvent();
switch (event) {
case JSONParser.STRING:
if (key != null) {
String val = parser.getString();
if (key.equals(val)) {
return true;
}
}
break;
case JSONParser.OBJECT_END:
return false;
case JSONParser.OBJECT_START:
if (deepSearch) {
boolean found = advanceToMapKey(key, true);
if (found) {
return true;
}
} else {
advanceToMapKey(null, false);
}
break;
case JSONParser.ARRAY_START:
skipArray(key, deepSearch);
break;
}
}
}
private void skipArray(String key, boolean deepSearch) throws IOException {
for (;;) {
int event = parser.nextEvent();
switch (event) {
case JSONParser.OBJECT_START:
advanceToMapKey(key, deepSearch);
break;
case JSONParser.ARRAY_START:
skipArray(key, deepSearch);
break;
case JSONParser.ARRAY_END:
return;
}
}
}
private boolean advanceToDocs() throws IOException {
expect(JSONParser.OBJECT_START);
boolean found = advanceToMapKey("docs", true);
expect(JSONParser.ARRAY_START);
return found;
}
}

View File

@ -0,0 +1,92 @@
/*
* 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.Map;
import java.util.HashMap;
public class MaxMetric implements Metric, Serializable {
public static final String MAX = "max";
private long longMax = -Long.MAX_VALUE;
private double doubleMax = Double.MAX_VALUE;
private boolean isDouble;
private String column;
public MaxMetric(String column, boolean isDouble) {
this.column = column;
this.isDouble = isDouble;
}
public String getName() {
return "mix:"+column;
}
public double getValue() {
if(isDouble) {
return doubleMax;
} else {
return longMax;
}
}
public void update(Tuple tuple) {
if(isDouble) {
double d = (double)tuple.get(column);
if(d > doubleMax) {
doubleMax = d;
}
} else {
long l = (long)tuple.get(column);
if(l > longMax) {
longMax = l;
}
}
}
public Metric newInstance() {
return new MaxMetric(column, isDouble);
}
public Map<String, Double> metricValues() {
Map m = new HashMap();
if(isDouble) {
m.put(MAX,doubleMax);
} else {
doubleMax = (double)longMax;
m.put(MAX,doubleMax);
}
return m;
}
public void update(Map<String, Double> metricValues) {
if(isDouble) {
double dmax = metricValues.get(MAX);
if(dmax > doubleMax) {
doubleMax = dmax;
}
} else {
double dmax = metricValues.get(MAX);
long lmax = (long) dmax;
if(lmax > longMax) {
longMax = lmax;
}
}
}
}

View File

@ -0,0 +1,104 @@
/*
* 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.Map;
import java.util.HashMap;
public class MeanMetric implements Metric, Serializable {
private static final long serialVersionUID = 1;
public static final String SUM = "sum";
public static final String COUNT = "count";
public static final String MEAN = "mean";
private String column;
private boolean isDouble;
private double doubleSum;
private long longSum;
private long count;
public MeanMetric(String column, boolean isDouble) {
this.column = column;
this.isDouble = isDouble;
}
public String getName() {
return "mean:"+column;
}
public void update(Tuple tuple) {
++count;
if(isDouble) {
Double d = (Double)tuple.get(column);
doubleSum += d.doubleValue();
} else {
Long l = (Long)tuple.get(column);
longSum += l.doubleValue();
}
}
public Metric newInstance() {
return new MeanMetric(column, isDouble);
}
public double getValue() {
double dcount = (double)count;
if(isDouble) {
double ave = doubleSum/dcount;
return ave;
} else {
double ave = longSum/dcount;
return ave;
}
}
public Map<String, Double> metricValues() {
Map m = new HashMap();
double dcount = (double)count;
m.put(COUNT, dcount);
if(isDouble) {
double ave = doubleSum/dcount;
m.put(MEAN,ave);
m.put(SUM,doubleSum);
} else {
double ave = longSum/dcount;
doubleSum = (double)longSum;
m.put(MEAN,ave);
m.put(SUM,doubleSum);
}
return m;
}
public void update(Map<String, Double> metricValues) {
double dcount = metricValues.get(COUNT);
count += (long)dcount;
if(isDouble) {
double dsum = metricValues.get(SUM);
doubleSum+=dsum;
} else {
double dsum = metricValues.get(SUM);
longSum+=(long)dsum;
}
}
}

View File

@ -0,0 +1,162 @@
/*
* 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.IOException;
import java.util.List;
import java.util.ArrayList;
import java.util.LinkedList;
import java.util.Comparator;
/**
* Merge Joins streamA with streamB based on the Comparator.
* Supports:
* one-to-one, one-to-many, many-to-one and many-to-many joins
*
**/
public class MergeJoinStream extends TupleStream {
private static final long serialVersionUID = 1;
private PushBackStream streamA;
private PushBackStream streamB;
private Comparator<Tuple> comp;
public MergeJoinStream(TupleStream streamA, TupleStream streamB, Comparator<Tuple> comp) {
this.streamA = new PushBackStream(streamA);
this.streamB = new PushBackStream(streamB);
this.comp = comp;
}
public void setStreamContext(StreamContext context) {
this.streamA.setStreamContext(context);
this.streamB.setStreamContext(context);
}
public List<TupleStream> children() {
List<TupleStream> l = new ArrayList();
l.add(streamA);
l.add(streamB);
return l;
}
public void open() throws IOException {
streamA.open();
streamB.open();
}
public void close() throws IOException {
streamA.close();
streamB.close();
}
private LinkedList<Tuple> joinTuples = new LinkedList();
private List<Tuple> listA = new ArrayList();
private List<Tuple> listB = new ArrayList();
public Tuple read() throws IOException {
if(joinTuples.size() > 0) {
return joinTuples.removeFirst();
}
OUTER:
while(true) {
if(listA.size() == 0) {
//Stream A needs to be advanced.
Tuple a = streamA.read();
if(a.EOF) {
return a;
}
listA.add(a);
INNERA:
while(true) {
Tuple a1 = streamA.read();
if(a1.EOF) {
streamA.pushBack(a1);
break INNERA;
}
if(comp.compare(a,a1) == 0) {
listA.add(a1);
} else {
streamA.pushBack(a1);
break INNERA;
}
}
}
if(listB.size() == 0) {
//StreamB needs to be advanced.
Tuple b = streamB.read();
if(b.EOF) {
return b;
}
listB.add(b);
INNERA:
while(true) {
Tuple b1 = streamB.read();
if(b1.EOF) {
streamB.pushBack(b1);
break INNERA;
}
if(comp.compare(b,b1) == 0) {
listB.add(b1);
} else {
streamB.pushBack(b1);
break INNERA;
}
}
}
int c = comp.compare(listA.get(0),listB.get(0));
if(c == 0) {
//The Tuple lists match. So build all the Tuple combinations.
for(Tuple aa : listA) {
for(Tuple bb : listB) {
Tuple clone = aa.clone();
clone.fields.putAll(bb.fields);
joinTuples.add(clone);
}
}
//This will advance both streams.
listA.clear();
listB.clear();
return joinTuples.removeFirst();
} else if(c < 0) {
//This will advance streamA
listA.clear();
} else {
//This will advance streamB
listB.clear();
}
}
}
public int getCost() {
return 0;
}
}

View File

@ -0,0 +1,99 @@
/*
* 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.IOException;
import java.util.Comparator;
import java.util.List;
import java.util.ArrayList;
/**
* Unions streamA with streamB based on a Comparator.
* Both streams must be sorted by the fields being compared.
**/
public class MergeStream extends TupleStream {
private static final long serialVersionUID = 1;
private PushBackStream streamA;
private PushBackStream streamB;
private Comparator<Tuple> comp;
public MergeStream(TupleStream streamA, TupleStream streamB, Comparator<Tuple> comp) {
this.streamA = new PushBackStream(streamA);
this.streamB = new PushBackStream(streamB);
this.comp = comp;
}
public void setStreamContext(StreamContext context) {
this.streamA.setStreamContext(context);
this.streamB.setStreamContext(context);
}
public List<TupleStream> children() {
List<TupleStream> l = new ArrayList();
l.add(streamA);
l.add(streamB);
return l;
}
public void open() throws IOException {
streamA.open();
streamB.open();
}
public void close() throws IOException {
streamA.close();
streamB.close();
}
public Tuple read() throws IOException {
Tuple a = streamA.read();
Tuple b = streamB.read();
if(a.EOF && b.EOF) {
return a;
}
if(a.EOF) {
streamA.pushBack(a);
return b;
}
if(b.EOF) {
streamB.pushBack(b);
return a;
}
int c = comp.compare(a,b);
if(c < 0) {
streamB.pushBack(b);
return a;
} else {
streamA.pushBack(a);
return b;
}
}
public int getCost() {
return 0;
}
}

View File

@ -0,0 +1,30 @@
/*
* 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.Map;
public interface Metric extends Serializable {
public String getName();
public double getValue();
public void update(Tuple tuple);
public Metric newInstance();
public Map<String, Double> metricValues();
public void update(Map<String, Double> metricValues);
}

View File

@ -0,0 +1,280 @@
/*
* 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.IOException;
import java.io.Serializable;
import java.util.Comparator;
import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.ArrayList;
import java.util.PriorityQueue;
/*
String[] buckets = {"a","b"};
Metric
BucketStream bucketStream = new BucketStream(stream,buckets,metrics,"my-metrics","name");
bucketStream.get(
*/
public class MetricStream extends TupleStream {
private static final long serialVersionUID = 1;
private TupleStream tupleStream;
private Bucket[] buckets;
private Metric[] metrics;
private String outKey;
private Map<HashKey, Metric[]> bucketMap;
private BucketMetrics[] bucketMetrics;
private static final HashKey metricsKey = new HashKey("metrics");
private int topN;
private Comparator<BucketMetrics> comp;
private Comparator<BucketMetrics> rcomp;
public MetricStream(TupleStream tupleStream,
Bucket[] buckets,
Metric[] metrics,
String outKey,
Comparator<BucketMetrics> comp,
int topN) {
this.tupleStream = tupleStream;
this.buckets = buckets;
this.metrics = metrics;
this.outKey = outKey;
this.topN = topN;
this.rcomp = new ReverseOrdComp(comp);
this.comp = comp;
this.bucketMap = new HashMap();
}
public MetricStream(TupleStream tupleStream,
Metric[] metrics,
String outKey) {
this.tupleStream = tupleStream;
this.metrics = metrics;
this.outKey = outKey;
this.bucketMap = new HashMap();
}
public String getOutKey() {
return this.outKey;
}
public BucketMetrics[] getBucketMetrics(){
return bucketMetrics;
}
public void setBucketMetrics(BucketMetrics[] bucketMetrics) {
this.bucketMetrics = bucketMetrics;
}
BucketMetrics[] merge(List<Map> all) {
Map<HashKey, Metric[]> bucketAccumulator = new HashMap();
for(Map top : all) {
List<String> ks = (List<String>)top.get("buckets");
List<List<Map<String,Double>>> ms = (List<List<Map<String,Double>>>)top.get("metrics");
for(int i=0; i<ks.size(); i++) {
String key = ks.get(i);
List<Map<String,Double>> bucketMs = ms.get(i);
HashKey hashKey = new HashKey(key);
if(bucketAccumulator.containsKey(hashKey)) {
Metric[] mergeMetrics = bucketAccumulator.get(hashKey);
for(int m=0; m<mergeMetrics.length; m++) {
mergeMetrics[m].update(bucketMs.get(m));
}
} else {
Metric[] mergedMetrics = new Metric[metrics.length];
for(int m=0; m<metrics.length; m++) {
mergedMetrics[m] = metrics[m].newInstance();
mergedMetrics[m].update(bucketMs.get(m));
}
bucketAccumulator.put(hashKey, mergedMetrics);
}
}
}
Iterator<Map.Entry<HashKey,Metric[]>> it = bucketAccumulator.entrySet().iterator();
PriorityQueue<BucketMetrics> priorityQueue = new PriorityQueue(topN, rcomp);
while(it.hasNext()) {
Map.Entry<HashKey, Metric[]> entry = it.next();
BucketMetrics bms = new BucketMetrics(entry.getKey(), entry.getValue());
if(priorityQueue.size() < topN) {
priorityQueue.add(bms);
} else {
BucketMetrics peek = priorityQueue.peek();
if(comp.compare(bms, peek) < 0) {
priorityQueue.poll();
priorityQueue.add(bms);
}
}
}
int s = priorityQueue.size();
BucketMetrics[] bucketMetrics = new BucketMetrics[s];
for(int i=bucketMetrics.length-1; i>=0; i--) {
BucketMetrics b = priorityQueue.poll();
bucketMetrics[i]= b;
}
return bucketMetrics;
}
private class ReverseOrdComp implements Comparator<BucketMetrics>, Serializable {
private Comparator<BucketMetrics> comp;
public ReverseOrdComp(Comparator<BucketMetrics> comp) {
this.comp = comp;
}
public int compare(BucketMetrics e1, BucketMetrics e2) {
return comp.compare(e1,e2)*-1;
}
}
public void setStreamContext(StreamContext context) {
this.tupleStream.setStreamContext(context);
}
public List<TupleStream> children() {
List<TupleStream> l = new ArrayList();
l.add(tupleStream);
return l;
}
public void open() throws IOException {
tupleStream.open();
}
public void close() throws IOException {
tupleStream.close();
}
public Tuple read() throws IOException {
Tuple tuple = tupleStream.read();
if(tuple.EOF) {
Iterator<Map.Entry<HashKey,Metric[]>> it = bucketMap.entrySet().iterator();
if(comp == null) {
//Handle No bucket constructor
Map.Entry<HashKey, Metric[]> noBucket = it.next();
BucketMetrics bms = new BucketMetrics(noBucket.getKey(), noBucket.getValue());
this.bucketMetrics = new BucketMetrics[1];
this.bucketMetrics[0] = bms;
List<Map<String, Double>> outMetrics = new ArrayList();
List<String> outKeys = new ArrayList();
for(Metric metric : bms.getMetrics()) {
Map<String, Double> outMetricValues = metric.metricValues();
String outKey = metric.getName();
outMetrics.add(outMetricValues);
outKeys.add(outKey);
}
Map outMap = new HashMap();
outMap.put("buckets",outKeys);
outMap.put("metrics",outMetrics);
tuple.set(this.outKey, outMap);
return tuple;
}
PriorityQueue<BucketMetrics> priorityQueue = new PriorityQueue(topN, rcomp);
while(it.hasNext()) {
Map.Entry<HashKey, Metric[]> entry = it.next();
BucketMetrics bms = new BucketMetrics(entry.getKey(), entry.getValue());
if(priorityQueue.size() < topN) {
priorityQueue.add(bms);
} else {
BucketMetrics peek = priorityQueue.peek();
if(comp.compare(bms, peek) < 0) {
priorityQueue.poll();
priorityQueue.add(bms);
}
}
}
int s = priorityQueue.size();
this.bucketMetrics = new BucketMetrics[s];
for(int i=bucketMetrics.length-1; i>=0; i--) {
BucketMetrics b = priorityQueue.poll();
this.bucketMetrics[i]= b;
}
List<List<Map<String, Double>>> outMetrics = new ArrayList();
List<String> outBuckets = new ArrayList();
for(BucketMetrics bms : this.bucketMetrics) {
List outBucketMetrics = new ArrayList();
for(Metric metric : bms.getMetrics()) {
Map<String, Double> outMetricValues = metric.metricValues();
outBucketMetrics.add(outMetricValues);
}
outBuckets.add(bms.getKey().toString());
outMetrics.add(outBucketMetrics);
}
Map outMap = new HashMap();
outMap.put("buckets",outBuckets);
outMap.put("metrics",outMetrics);
tuple.set(this.outKey, outMap);
return tuple;
}
HashKey hashKey = null;
if(buckets != null) {
String[] bucketValues = new String[buckets.length];
for(int i=0; i<buckets.length; i++) {
bucketValues[i] = buckets[i].getBucketValue(tuple);
}
hashKey = new HashKey(bucketValues);
} else {
hashKey = metricsKey;
}
Metric[] bucketMetrics = bucketMap.get(hashKey);
if(bucketMetrics != null) {
for(Metric bucketMetric : bucketMetrics) {
bucketMetric.update(tuple);
}
} else {
bucketMetrics = new Metric[metrics.length];
for(int i=0; i<metrics.length; i++) {
Metric bucketMetric = metrics[i].newInstance();
bucketMetric.update(tuple);
bucketMetrics[i] = bucketMetric;
}
bucketMap.put(hashKey, bucketMetrics);
}
return tuple;
}
public int getCost() {
return 0;
}
}

View File

@ -0,0 +1,92 @@
/*
* 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.Map;
import java.util.HashMap;
public class MinMetric implements Metric, Serializable {
public static final String MIN = "min";
private long longMin = Long.MAX_VALUE;
private double doubleMin = Double.MAX_VALUE;
private boolean isDouble;
private String column;
public MinMetric(String column, boolean isDouble) {
this.column = column;
this.isDouble = isDouble;
}
public String getName() {
return "min:"+column;
}
public double getValue() {
if(isDouble) {
return doubleMin;
} else {
return longMin;
}
}
public void update(Tuple tuple) {
if(isDouble) {
double d = (double)tuple.get(column);
if(d < doubleMin) {
doubleMin = d;
}
} else {
long l = (long)tuple.get(column);
if(l < longMin) {
longMin = l;
}
}
}
public Metric newInstance() {
return new MinMetric(column, isDouble);
}
public Map<String, Double> metricValues() {
Map m = new HashMap();
if(isDouble) {
m.put(MIN,doubleMin);
} else {
doubleMin = (double)longMin;
m.put(MIN,doubleMin);
}
return m;
}
public void update(Map<String, Double> metricValues) {
if(isDouble) {
double dmin = metricValues.get(MIN);
if(dmin < doubleMin) {
doubleMin = dmin;
}
} else {
double dmin = metricValues.get(MIN);
long lmin = (long) dmin;
if(lmin < longMin) {
longMin = lmin;
}
}
}
}

View File

@ -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;
import java.util.Comparator;
public class MultiComp implements Comparator<Tuple>, Serializable {
private static final long serialVersionUID = 1;
private Comparator<Tuple>[] comps;
public MultiComp(Comparator<Tuple>... comps) {
this.comps = comps;
}
public int compare(Tuple t1, Tuple t2) {
for(Comparator<Tuple> comp : comps) {
int i = comp.compare(t1, t2);
if(i != 0) {
return i;
}
}
return 0;
}
}

View File

@ -0,0 +1,160 @@
/*
* 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.IOException;
import java.io.ObjectOutputStream;
import java.net.URLEncoder;
import java.util.Collection;
import java.util.Collections;
import java.util.List;
import java.util.ArrayList;
import java.util.Comparator;
import java.util.HashMap;
import java.util.Map;
import java.io.ByteArrayOutputStream;
import java.util.Random;
import java.util.TreeSet;
import java.util.Iterator;
import org.apache.solr.common.cloud.ClusterState;
import org.apache.solr.common.cloud.Replica;
import org.apache.solr.common.cloud.Slice;
import org.apache.solr.common.cloud.ZkCoreNodeProps;
import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.util.Base64;
/**
* The ParallelStream decorates a TupleStream implementation and pushes it N workers for parallel execution.
* Workers are chosen from a SolrCloud collection.
* Tuples that are streamed back from the workers are ordered by a Comparator.
**/
public class ParallelStream extends CloudSolrStream {
private TupleStream tupleStream;
private int workers;
private String encoded;
public ParallelStream(String zkHost,
String collection,
TupleStream tupleStream,
int workers,
Comparator<Tuple> comp) throws IOException {
this.zkHost = zkHost;
this.collection = collection;
this.workers = workers;
this.comp = comp;
this.tupleStream = tupleStream;
ByteArrayOutputStream bout = new ByteArrayOutputStream();
ObjectOutputStream out = new ObjectOutputStream(bout);
out.writeObject(tupleStream);
byte[] bytes = bout.toByteArray();
this.encoded = Base64.byteArrayToBase64(bytes, 0, bytes.length);
this.encoded = URLEncoder.encode(this.encoded, "UTF-8");
this.tuples = new TreeSet();
}
public List<TupleStream> children() {
List l = new ArrayList();
l.add(tupleStream);
return l;
}
public void merge(List<MetricStream> metricStreams) {
for(MetricStream metricStream : metricStreams) {
String outKey = metricStream.getOutKey();
Iterator<Tuple> it = eofTuples.values().iterator();
List values = new ArrayList();
while(it.hasNext()) {
Tuple t = it.next();
Map top = (Map)t.get(outKey);
values.add(top);
}
BucketMetrics[] bucketMetrics = metricStream.merge(values);
metricStream.setBucketMetrics(bucketMetrics);
}
}
public Tuple read() throws IOException {
Tuple tuple = _read();
if(tuple.EOF) {
List<MetricStream> metricStreams = new ArrayList();
getMetricStreams(this, metricStreams);
this.merge(metricStreams);
Map m = new HashMap();
m.put("EOF", true);
return new Tuple(m);
}
return tuple;
}
private void getMetricStreams(TupleStream tupleStream,
List<MetricStream> metricStreams) {
if(tupleStream instanceof MetricStream) {
metricStreams.add((MetricStream)tupleStream);
}
List<TupleStream> children = tupleStream.children();
for(TupleStream ts : children) {
getMetricStreams(ts, metricStreams);
}
}
protected void constructStreams() throws IOException {
try {
ZkStateReader zkStateReader = cloudSolrClient.getZkStateReader();
ClusterState clusterState = zkStateReader.getClusterState();
Collection<Slice> slices = clusterState.getActiveSlices(this.collection);
long time = System.currentTimeMillis();
int workerNum = 0;
for(Slice slice : slices) {
HashMap params = new HashMap();
params.put("distrib","false"); // We are the aggregator.
params.put("numWorkers", workers);
params.put("workerID", workerNum);
params.put("stream", this.encoded);
params.put("qt","/stream");
Collection<Replica> replicas = slice.getReplicas();
List<Replica> shuffler = new ArrayList();
for(Replica replica : replicas) {
shuffler.add(replica);
}
Collections.shuffle(shuffler, new Random(time));
Replica rep = shuffler.get(0);
ZkCoreNodeProps zkProps = new ZkCoreNodeProps(rep);
String url = zkProps.getCoreUrl();
SolrStream solrStream = new SolrStream(url, params);
solrStreams.add(solrStream);
++workerNum;
}
} catch (Exception e) {
throw new IOException(e);
}
}
}

View File

@ -0,0 +1,77 @@
/*
* 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.IOException;
import java.util.Comparator;
import java.util.List;
import java.util.ArrayList;
public class PushBackStream extends TupleStream {
private static final long serialVersionUID = 1;
private TupleStream stream;
private Tuple tuple;
/*
* A TupleStream that allows a single Tuple to be pushed back into Stream after it's been read.
*
**/
public PushBackStream(TupleStream stream) {
this.stream = stream;
}
public void setStreamContext(StreamContext context) {
this.stream.setStreamContext(context);
}
public List<TupleStream> children() {
List<TupleStream> l = new ArrayList();
l.add(stream);
return l;
}
public void open() throws IOException {
stream.open();
}
public void close() throws IOException {
stream.close();
}
public void pushBack(Tuple tuple) {
this.tuple = tuple;
}
public Tuple read() throws IOException {
if(tuple != null) {
Tuple t = tuple;
tuple = null;
return t;
} else {
return stream.read();
}
}
public int getCost() {
return 0;
}
}

View File

@ -0,0 +1,116 @@
/*
* 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.IOException;
import java.io.Serializable;
import java.util.List;
import java.util.ArrayList;
import java.util.LinkedList;
import java.util.PriorityQueue;
import java.util.Comparator;
/**
* Iterates over a TupleStream and Ranks the topN tuples based on a Comparator.
**/
public class RankStream extends TupleStream {
private static final long serialVersionUID = 1;
private TupleStream tupleStream;
private PriorityQueue<Tuple> top;
private Comparator<Tuple> comp;
private boolean finished = false;
private LinkedList<Tuple> topList;
private int size;
public RankStream(TupleStream tupleStream, int size, Comparator<Tuple> comp) {
this.tupleStream = tupleStream;
this.top = new PriorityQueue(size, new ReverseComp(comp));
this.comp = comp;
this.topList = new LinkedList();
this.size = size;
}
public void setStreamContext(StreamContext context) {
this.tupleStream.setStreamContext(context);
}
public List<TupleStream> children() {
List<TupleStream> l = new ArrayList();
l.add(tupleStream);
return l;
}
public void open() throws IOException {
tupleStream.open();
}
public void close() throws IOException {
tupleStream.close();
}
public Tuple read() throws IOException {
if(!finished) {
while(true) {
Tuple tuple = tupleStream.read();
if(tuple.EOF) {
finished = true;
int s = top.size();
for(int i=0; i<s; i++) {
Tuple t = top.poll();
topList.addFirst(t);
}
topList.addLast(tuple);
break;
} else {
Tuple peek = top.peek();
if(top.size() >= size) {
if(comp.compare(tuple, peek) < 0) {
top.poll();
top.add(tuple);
}
} else {
top.add(tuple);
}
}
}
}
return topList.pollFirst();
}
public int getCost() {
return 0;
}
class ReverseComp implements Comparator<Tuple>, Serializable {
private Comparator<Tuple> comp;
public ReverseComp(Comparator<Tuple> comp) {
this.comp = comp;
}
public int compare(Tuple t1, Tuple t2) {
return comp.compare(t1, t2)*(-1);
}
}
}

View File

@ -0,0 +1,138 @@
/*
* 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.IOException;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.ArrayList;
public class RollupStream extends TupleStream {
private static final long serialVersionUID = 1;
private PushBackStream tupleStream;
private Bucket[] buckets;
private Metric[] metrics;
private HashKey currentKey = new HashKey("-");
private Metric[] currentMetrics;
private boolean finished = false;
public RollupStream(TupleStream tupleStream,
Bucket[] buckets,
Metric[] metrics) {
this.tupleStream = new PushBackStream(tupleStream);
this.buckets = buckets;
this.metrics = metrics;
}
public void setStreamContext(StreamContext context) {
this.tupleStream.setStreamContext(context);
}
public List<TupleStream> children() {
List<TupleStream> l = new ArrayList();
l.add(tupleStream);
return l;
}
public void open() throws IOException {
tupleStream.open();
}
public void close() throws IOException {
tupleStream.close();
}
public Tuple read() throws IOException {
while(true) {
Tuple tuple = tupleStream.read();
if(tuple.EOF) {
if(!finished) {
Map map = new HashMap();
if(currentMetrics != null) {
List<Double> metricValues = new ArrayList();
List<String> metricNames = new ArrayList();
for (Metric metric : currentMetrics) {
metricNames.add(metric.getName());
metricValues.add(metric.getValue());
}
map.put("buckets", currentKey.toString());
map.put("metricNames", metricNames);
map.put("metricValues", metricValues);
Tuple t = new Tuple(map);
tupleStream.pushBack(tuple);
finished = true;
return t;
} else {
return tuple;
}
} else {
return tuple;
}
}
String[] bucketValues = new String[buckets.length];
for(int i=0; i<buckets.length; i++) {
bucketValues[i] = buckets[i].getBucketValue(tuple);
}
HashKey hashKey = new HashKey(bucketValues);
if(hashKey.equals(currentKey)) {
for(Metric bucketMetric : currentMetrics) {
bucketMetric.update(tuple);
}
} else {
Tuple t = null;
if(currentMetrics != null) {
Map map = new HashMap();
List<Double> metricValues = new ArrayList();
List<String> metricNames = new ArrayList();
for(Metric metric : currentMetrics) {
metricNames.add(metric.getName());
metricValues.add(metric.getValue());
}
map.put("buckets", currentKey.toString());
map.put("metricNames", metricNames);
map.put("metricValues", metricValues);
t = new Tuple(map);
}
currentMetrics = new Metric[metrics.length];
currentKey = hashKey;
for(int i=0; i<metrics.length; i++) {
Metric bucketMetric = metrics[i].newInstance();
bucketMetric.update(tuple);
currentMetrics[i] = bucketMetric;
}
if(t != null) {
return t;
}
}
}
}
public int getCost() {
return 0;
}
}

View File

@ -0,0 +1,71 @@
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.IOException;
import java.io.Serializable;
import java.util.Map;
import java.util.HashMap;
import java.util.Iterator;
import org.apache.solr.client.solrj.SolrClient;
import org.apache.solr.client.solrj.impl.CloudSolrClient;
import org.apache.solr.client.solrj.impl.HttpSolrClient;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
public class SolrClientCache implements Serializable {
protected static final Logger log = LoggerFactory.getLogger(SolrClientCache.class);
private Map<String, SolrClient> solrClients = new HashMap();
public synchronized CloudSolrClient getCloudSolrClient(String zkHost) {
CloudSolrClient client = null;
if (solrClients.containsKey(zkHost)) {
client = (CloudSolrClient) solrClients.get(zkHost);
} else {
client = new CloudSolrClient(zkHost);
client.connect();
solrClients.put(zkHost, client);
}
return client;
}
public synchronized HttpSolrClient getHttpSolrClient(String host) {
HttpSolrClient client = null;
if (solrClients.containsKey(host)) {
client = (HttpSolrClient) solrClients.get(host);
} else {
client = new HttpSolrClient(host);
solrClients.put(host, client);
}
return client;
}
public void close() {
Iterator<SolrClient> it = solrClients.values().iterator();
while(it.hasNext()) {
try {
it.next().close();
} catch (IOException e) {
log.error(e.getMessage(), e);
}
}
}
}

View File

@ -0,0 +1,159 @@
/*
* 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.IOException;
import java.util.ArrayList;
import java.util.Map;
import java.util.HashMap;
import java.util.List;
import java.util.Iterator;
import org.apache.solr.client.solrj.impl.HttpSolrClient;
import org.apache.solr.common.params.ModifiableSolrParams;
import org.apache.solr.common.params.SolrParams;
/**
* Queries a Solr instance, and maps SolrDocs to a Stream of Tuples.
**/
public class SolrStream extends TupleStream {
private static final long serialVersionUID = 1;
private String baseUrl;
private Map params;
private int numWorkers;
private int workerID;
private Map<String, String> fieldMappings;
private transient JSONTupleStream jsonTupleStream;
private transient HttpSolrClient client;
private transient SolrClientCache cache;
public SolrStream(String baseUrl, Map params) {
this.baseUrl = baseUrl;
this.params = params;
}
public void setFieldMappings(Map<String, String> fieldMappings) {
this.fieldMappings = fieldMappings;
}
public List<TupleStream> children() {
return new ArrayList();
}
public String getBaseUrl() {
return baseUrl;
}
public void setStreamContext(StreamContext context) {
this.numWorkers = context.numWorkers;
this.workerID = context.workerID;
this.cache = context.clientCache;
}
/**
* Opens the stream to a single Solr instance.
**/
public void open() throws IOException {
if(cache == null) {
client = new HttpSolrClient(baseUrl);
} else {
client = cache.getHttpSolrClient(baseUrl);
}
try {
jsonTupleStream = JSONTupleStream.create(client, loadParams(params));
} catch (Exception e) {
throw new IOException(e);
}
}
private SolrParams loadParams(Map params) {
ModifiableSolrParams solrParams = new ModifiableSolrParams();
if(this.numWorkers > 0) {
String partitionFilter = getPartitionFilter();
solrParams.add("fq", partitionFilter);
}
Iterator<Map.Entry> it = params.entrySet().iterator();
while(it.hasNext()) {
Map.Entry entry = it.next();
solrParams.add((String)entry.getKey(), entry.getValue().toString());
}
return solrParams;
}
private String getPartitionFilter() {
StringBuilder buf = new StringBuilder("{!hash workers=");
buf.append(this.numWorkers);
buf.append(" worker=");
buf.append(this.workerID);
buf.append("}");
return buf.toString();
}
/**
* Closes the Stream to a single Solr Instance
* */
public void close() throws IOException {
jsonTupleStream.close();
if(cache == null) {
client.close();
}
}
/**
* Reads a Tuple from the stream. The Stream is completed when Tuple.EOF == true.
**/
public Tuple read() throws IOException {
Map fields = jsonTupleStream.next();
if(fields == null) {
//Return the EOF tuple.
Map m = new HashMap();
m.put("EOF", true);
return new Tuple(m);
} else {
if(fieldMappings != null) {
fields = mapFields(fields, fieldMappings);
}
return new Tuple(fields);
}
}
private Map mapFields(Map fields, Map<String,String> mappings) {
Iterator<Map.Entry<String,String>> it = mappings.entrySet().iterator();
while(it.hasNext()) {
Map.Entry<String,String> entry = it.next();
String mapFrom = entry.getKey();
String mapTo = entry.getValue();
Object o = fields.get(mapFrom);
fields.remove(mapFrom);
fields.put(mapTo, o);
}
return fields;
}
}

View File

@ -0,0 +1,46 @@
/*
* 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.util.Map;
import java.util.HashMap;
import java.util.concurrent.ConcurrentHashMap;
public class StreamContext {
private Map entries = new HashMap();
public int workerID;
public int numWorkers;
public SolrClientCache clientCache;
public SolrClientCache getClientCache() {
return this.clientCache;
}
public void setSolrClientCache(SolrClientCache clientCache) {
this.clientCache = clientCache;
}
public Object get(Object key) {
return entries.get(key);
}
public void put(Object key, Object value) {
this.entries.put(key, value);
}
}

View File

@ -0,0 +1,88 @@
/*
* 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.Map;
import java.util.HashMap;
public class SumMetric implements Metric, Serializable {
private static final long serialVersionUID = 1;
public static final String SUM = "sum";
private String column;
private boolean isDouble;
private double doubleSum;
private long longSum;
public SumMetric(String column, boolean isDouble) {
this.column = column;
this.isDouble = isDouble;
}
public String getName() {
return "sum:"+column;
}
public void update(Tuple tuple) {
if(isDouble) {
Double d = (Double)tuple.get(column);
doubleSum += d.doubleValue();
} else {
Long l = (Long)tuple.get(column);
longSum += l.doubleValue();
}
}
public Metric newInstance() {
return new SumMetric(column, isDouble);
}
public Map<String, Double> metricValues() {
Map m = new HashMap();
if(isDouble) {
m.put(SUM,doubleSum);
} else {
doubleSum = (double)longSum;
m.put(SUM,doubleSum);
}
return m;
}
public double getValue() {
if(isDouble) {
return doubleSum;
} else {
return (double)longSum;
}
}
public void update(Map<String, Double> metricValues) {
if(isDouble) {
double dsum = metricValues.get(SUM);
doubleSum+=dsum;
} else {
double dsum = metricValues.get(SUM);
longSum+=(long)dsum;
}
}
}

View File

@ -0,0 +1,93 @@
/*
* 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.util.HashMap;
import java.util.Map;
import java.util.List;
import java.util.Iterator;
/**
* A simple abstraction of a record containing key/value pairs.
* Convenience methods are provided for returning single and multiValue String, Long and Double values.
* Note that ints and floats are treated as longs and doubles respectively.
*
**/
public class Tuple implements Cloneable {
public boolean EOF;
public Map fields = new HashMap();
public Tuple(Map fields) {
if(fields.containsKey("EOF")) {
EOF = true;
}
this.fields.putAll(fields);
}
public Object get(Object key) {
return this.fields.get(key);
}
public void set(Object key, Object value) {
this.fields.put(key, value);
}
public String getString(Object key) {
return (String)this.fields.get(key);
}
public Long getLong(Object key) {
return (Long)this.fields.get(key);
}
public Double getDouble(Object key) {
return (Double)this.fields.get(key);
}
public List<String> getStrings(Object key) {
return (List<String>)this.fields.get(key);
}
public List<Long> getLongs(Object key) {
return (List<Long>)this.fields.get(key);
}
public List<Double> getDoubles(Object key) {
return (List<Double>)this.fields.get(key);
}
public Iterator<Map.Entry> getFields() {
return fields.entrySet().iterator();
}
public Tuple clone() {
HashMap m = new HashMap();
m.putAll(fields);
Tuple clone = new Tuple(m);
return clone;
}
}

View File

@ -0,0 +1,51 @@
/*
* 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.IOException;
import java.io.Serializable;
import java.util.List;
import java.util.Map;
import java.util.HashMap;
import org.apache.solr.client.solrj.SolrClient;
import org.apache.solr.client.solrj.impl.CloudSolrClient;
import org.apache.solr.client.solrj.impl.HttpSolrClient;
public abstract class TupleStream implements Serializable {
private static final long serialVersionUID = 1;
public TupleStream() {
}
public abstract void setStreamContext(StreamContext context);
public abstract List<TupleStream> children();
public abstract void open() throws IOException;
public abstract void close() throws IOException;
public abstract Tuple read() throws IOException;
public int getCost() {
return 0;
}
}

View File

@ -0,0 +1,86 @@
/*
* 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.IOException;
import java.util.Comparator;
import java.util.List;
import java.util.ArrayList;
public class UniqueStream extends TupleStream {
private static final long serialVersionUID = 1;
private TupleStream tupleStream;
private Comparator<Tuple> comp;
private Tuple currentTuple = null;
public UniqueStream(TupleStream tupleStream, Comparator<Tuple> comp) {
this.tupleStream = tupleStream;
this.comp = comp;
}
public void setStreamContext(StreamContext context) {
this.tupleStream.setStreamContext(context);
}
public List<TupleStream> children() {
List<TupleStream> l = new ArrayList();
l.add(tupleStream);
return l;
}
public void open() throws IOException {
tupleStream.open();
}
public void close() throws IOException {
tupleStream.close();
}
public Tuple read() throws IOException {
Tuple tuple = tupleStream.read();
if(tuple.EOF) {
return tuple;
}
if(currentTuple == null) {
currentTuple = tuple;
return tuple;
} else {
while(true) {
int i = comp.compare(currentTuple, tuple);
if(i == 0) {
//We have duplicate tuple so read the next tuple from the stream.
tuple = tupleStream.read();
if(tuple.EOF) {
return tuple;
}
} else {
//We have a non duplicate
this.currentTuple = tuple;
return tuple;
}
}
}
}
public int getCost() {
return 0;
}
}

View File

@ -0,0 +1,24 @@
/*
* 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.
*/
/**
* Streaming Aggregation API
**/
package org.apache.solr.client.solrj.io;

View File

@ -0,0 +1,599 @@
<?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" docValues="true" class="solr.TrieIntField" precisionStep="0" omitNorms="true" positionIncrementGap="0"/>
<fieldType name="float" docValues="true" class="solr.TrieFloatField" precisionStep="0" omitNorms="true" positionIncrementGap="0"/>
<fieldType name="long" class="solr.TrieLongField" precisionStep="0" omitNorms="true" positionIncrementGap="0"/>
<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="s_multi" type="string" indexed="true" stored="true" docValues="true" multiValued="true"/>
<field name="i_multi" type="int" indexed="true" stored="true" docValues="true" multiValued="true"/>
<field name="f_multi" type="float" indexed="true" stored="true" docValues="true" multiValued="true"/>
<field name="l_multi" type="long" indexed="true" stored="true" docValues="true" multiValued="true"/>
<field name="d_multi" type="double" indexed="true" stored="true" docValues="true" multiValued="true"/>
<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>

View File

@ -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>

File diff suppressed because it is too large Load Diff