mirror of https://github.com/apache/lucene.git
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:
parent
b4ee7b6eb4
commit
9563a30751
|
@ -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;
|
||||||
|
}
|
||||||
|
}
|
|
@ -23,6 +23,8 @@ import java.util.*;
|
||||||
|
|
||||||
import org.apache.lucene.document.Document;
|
import org.apache.lucene.document.Document;
|
||||||
import org.apache.lucene.index.IndexableField;
|
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.StorableField;
|
||||||
import org.apache.lucene.index.StoredDocument;
|
import org.apache.lucene.index.StoredDocument;
|
||||||
import org.apache.lucene.util.BytesRef;
|
import org.apache.lucene.util.BytesRef;
|
||||||
|
@ -186,12 +188,14 @@ public abstract class TextResponseWriter {
|
||||||
writeMap(name, (Map)val, false, true);
|
writeMap(name, (Map)val, false, true);
|
||||||
} else if (val instanceof NamedList) {
|
} else if (val instanceof NamedList) {
|
||||||
writeNamedList(name, (NamedList)val);
|
writeNamedList(name, (NamedList)val);
|
||||||
|
} else if (val instanceof TupleStream) {
|
||||||
|
writeTupleStream((TupleStream) val);
|
||||||
} else if (val instanceof Iterable) {
|
} else if (val instanceof Iterable) {
|
||||||
writeArray(name,((Iterable)val).iterator());
|
writeArray(name,((Iterable)val).iterator());
|
||||||
} else if (val instanceof Object[]) {
|
} else if (val instanceof Object[]) {
|
||||||
writeArray(name,(Object[])val);
|
writeArray(name,(Object[])val);
|
||||||
} else if (val instanceof Iterator) {
|
} else if (val instanceof Iterator) {
|
||||||
writeArray(name,(Iterator)val);
|
writeArray(name, (Iterator) val);
|
||||||
} else if (val instanceof byte[]) {
|
} else if (val instanceof byte[]) {
|
||||||
byte[] arr = (byte[])val;
|
byte[] arr = (byte[])val;
|
||||||
writeByteArr(name, arr, 0, arr.length);
|
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 */
|
/** 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;
|
public abstract void writeDouble(String name, String val) throws IOException;
|
||||||
|
|
||||||
|
|
|
@ -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);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -72,6 +72,7 @@ public abstract class QParserPlugin implements NamedListInitializedPlugin, SolrI
|
||||||
map.put(ReRankQParserPlugin.NAME, ReRankQParserPlugin.class);
|
map.put(ReRankQParserPlugin.NAME, ReRankQParserPlugin.class);
|
||||||
map.put(ExportQParserPlugin.NAME, ExportQParserPlugin.class);
|
map.put(ExportQParserPlugin.NAME, ExportQParserPlugin.class);
|
||||||
map.put(MLTQParserPlugin.NAME, MLTQParserPlugin.class);
|
map.put(MLTQParserPlugin.NAME, MLTQParserPlugin.class);
|
||||||
|
map.put(HashQParserPlugin.NAME, HashQParserPlugin.class);
|
||||||
standardPlugins = Collections.unmodifiableMap(map);
|
standardPlugins = Collections.unmodifiableMap(map);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -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>
|
|
@ -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>
|
||||||
|
|
|
@ -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 {
|
public void testQueryNested() throws Exception {
|
||||||
SolrQueryRequest req = req("df", "foo_s");
|
SolrQueryRequest req = req("df", "foo_s");
|
||||||
try {
|
try {
|
||||||
|
|
|
@ -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());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -893,6 +893,36 @@
|
||||||
</lst>
|
</lst>
|
||||||
</requestHandler>
|
</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
|
<!-- Field Analysis Request Handler
|
||||||
|
|
||||||
|
|
|
@ -905,6 +905,18 @@
|
||||||
</requestHandler>
|
</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>
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
|
|
@ -520,7 +520,7 @@ public class HttpSolrClient extends SolrClient {
|
||||||
null);
|
null);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
if (processor == null) {
|
if (processor == null || processor instanceof InputStreamResponseParser) {
|
||||||
|
|
||||||
// no processor specified, return raw stream
|
// no processor specified, return raw stream
|
||||||
NamedList<Object> rsp = new NamedList<>();
|
NamedList<Object> rsp = new NamedList<>();
|
||||||
|
|
|
@ -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();
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
|
@ -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;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -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);
|
||||||
|
}
|
||||||
|
}
|
|
@ -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));
|
||||||
|
}
|
||||||
|
}
|
|
@ -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();
|
||||||
|
}
|
||||||
|
}
|
|
@ -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;
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
}
|
|
@ -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;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -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;
|
||||||
|
}
|
||||||
|
}
|
|
@ -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;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -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;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -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;
|
||||||
|
}
|
||||||
|
}
|
|
@ -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;
|
||||||
|
}
|
||||||
|
}
|
|
@ -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);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -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;
|
||||||
|
}
|
||||||
|
}
|
|
@ -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();
|
||||||
|
}
|
||||||
|
}
|
|
@ -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;
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
}
|
|
@ -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;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -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;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -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;
|
||||||
|
}
|
||||||
|
}
|
|
@ -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;
|
||||||
|
}
|
||||||
|
}
|
|
@ -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);
|
||||||
|
}
|
|
@ -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;
|
||||||
|
}
|
||||||
|
}
|
|
@ -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;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -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;
|
||||||
|
}
|
||||||
|
}
|
|
@ -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);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -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;
|
||||||
|
}
|
||||||
|
}
|
|
@ -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);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -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;
|
||||||
|
}
|
||||||
|
}
|
|
@ -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);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -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;
|
||||||
|
}
|
||||||
|
}
|
|
@ -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);
|
||||||
|
}
|
||||||
|
}
|
|
@ -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;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -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;
|
||||||
|
}
|
||||||
|
}
|
|
@ -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;
|
||||||
|
}
|
||||||
|
}
|
|
@ -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;
|
||||||
|
}
|
||||||
|
}
|
|
@ -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;
|
||||||
|
|
|
@ -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>
|
|
@ -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
Loading…
Reference in New Issue