SOLR-486: Binary response format

git-svn-id: https://svn.apache.org/repos/asf/lucene/solr/trunk@648384 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Yonik Seeley 2008-04-15 19:22:38 +00:00
parent e1c82f69fb
commit edfcaa6161
25 changed files with 1607 additions and 41 deletions

View File

@ -239,8 +239,14 @@ New Features
42. SOLR-494: Added cool admin Ajaxed schema explorer.
(Greg Ludington via ehatcher)
43. SOLR-497: Added date faceting to the QueryResponse in SolrJ and QueryResponseTest (Shalin Shekhar Mangar via gsingers)
43. SOLR-497: Added date faceting to the QueryResponse in SolrJ
and QueryResponseTest (Shalin Shekhar Mangar via gsingers)
44. SOLR-486: Binary response format, faster and smaller
than XML and JSON response formats (use wt=javabin).
BinaryResponseParser for utilizing the binary format via SolrJ.
(Noble Paul, yonik)
Changes in runtime behavior
Optimizations

View File

@ -0,0 +1,45 @@
/**
* 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.impl;
import org.apache.solr.client.solrj.ResponseParser;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.common.util.NamedListCodec;
import java.io.IOException;
import java.io.InputStream;
import java.io.Reader;
public class BinaryResponseParser extends ResponseParser {
public String getWriterType() {
return "javabin";
}
public NamedList<Object> processResponse(InputStream body, String encoding) {
try {
return new NamedListCodec().unmarshal(body);
} catch (IOException e) {
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "parsing error", e);
}
}
public NamedList<Object> processResponse(Reader reader) {
throw new RuntimeException("Cannot handle character stream");
}
}

View File

@ -35,6 +35,7 @@ import org.apache.solr.common.SolrDocument;
import org.apache.solr.common.SolrDocumentList;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.common.util.SimpleOrderedMap;
/**
*
@ -123,7 +124,7 @@ public class XMLResponseParser extends ResponseParser
response = readNamedList( parser );
}
else if( name.equals( "solr" ) ) {
return new NamedList<Object>();
return new SimpleOrderedMap<Object>();
}
else {
throw new Exception( "really needs to be response or result. " +
@ -200,7 +201,7 @@ public class XMLResponseParser extends ResponseParser
}
StringBuilder builder = new StringBuilder();
NamedList<Object> nl = new NamedList<Object>();
NamedList<Object> nl = new SimpleOrderedMap<Object>();
KnownType type = null;
String name = null;

View File

@ -56,4 +56,12 @@ public class SolrDocumentList extends ArrayList<SolrDocument>
public void setStart(long start) {
this.start = start;
}
public String toString() {
return "{numFound="+numFound
+",start="+start
+ (maxScore!=null ? ""+maxScore : "")
+",docs="+super.toString()
+"}";
}
}

View File

@ -0,0 +1,211 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.solr.common.util;
import java.io.*;
/** Single threaded buffered InputStream
* Internal Solr use only, subject to change.
*/
public class FastInputStream extends InputStream implements DataInput {
private final InputStream in;
private final byte[] buf;
private int pos;
private int end;
public FastInputStream(InputStream in) {
// use default BUFSIZE of BufferedOutputStream so if we wrap that
// it won't cause double buffering.
this(in, new byte[8192], 0, 0);
}
public FastInputStream(InputStream in, byte[] tempBuffer, int start, int end) {
this.in = in;
this.buf = tempBuffer;
this.pos = start;
this.end = end;
}
public static FastInputStream wrap(InputStream in) {
return (in instanceof FastInputStream) ? (FastInputStream)in : new FastInputStream(in);
}
@Override
public int read() throws IOException {
if (pos >= end) {
refill();
if (pos >= end) return -1;
}
return buf[pos++] & 0xff;
}
public int readUnsignedByte() throws IOException {
if (pos >= end) {
refill();
if (pos >= end) throw new EOFException();
}
return buf[pos++] & 0xff;
}
public void refill() throws IOException {
// this will set end to -1 at EOF
end = in.read(buf, 0, buf.length);
pos = 0;
}
@Override
public int available() throws IOException {
return end - pos;
}
@Override
public int read(byte b[], int off, int len) throws IOException {
int r=0; // number of bytes read
// first read from our buffer;
if (end-pos > 0) {
r = Math.min(end-pos, len);
System.arraycopy(buf, pos, b, off, r);
pos += r;
}
if (r == len) return r;
// amount left to read is >= buffer size
if (len-r >= buf.length) {
int ret = in.read(b, off+r, len-r);
if (ret==-1) return r==0 ? -1 : r;
r += ret;
return r;
}
refill();
// first read from our buffer;
if (end-pos > 0) {
int toRead = Math.min(end-pos, len-r);
System.arraycopy(buf, pos, b, off+r, toRead);
pos += toRead;
r += toRead;
return r;
}
return -1;
}
@Override
public void close() throws IOException {
in.close();
}
public void readFully(byte b[]) throws IOException {
readFully(b, 0, b.length);
}
public void readFully(byte b[], int off, int len) throws IOException {
while (len>0) {
int ret = read(b, off, len);
if (ret==-1) {
throw new EOFException();
}
off += ret;
len -= ret;
}
}
public int skipBytes(int n) throws IOException {
if (end-pos >= n) {
pos += n;
return n;
}
if (end-pos<0) return -1;
int r = end-pos;
pos = end;
while (r < n) {
refill();
if (end-pos <= 0) return r;
int toRead = Math.min(end-pos, n-r);
r += toRead;
pos += toRead;
}
return r;
}
public boolean readBoolean() throws IOException {
return readByte()==1;
}
public byte readByte() throws IOException {
if (pos >= end) {
refill();
if (pos >= end) throw new EOFException();
}
return buf[pos++];
}
public short readShort() throws IOException {
return (short)((readUnsignedByte() << 8) | readUnsignedByte());
}
public int readUnsignedShort() throws IOException {
return (readUnsignedByte() << 8) | readUnsignedByte();
}
public char readChar() throws IOException {
return (char)((readUnsignedByte() << 8) | readUnsignedByte());
}
public int readInt() throws IOException {
return ((readUnsignedByte() << 24)
|(readUnsignedByte() << 16)
|(readUnsignedByte() << 8)
| readUnsignedByte());
}
public long readLong() throws IOException {
return (((long)readUnsignedByte()) << 56)
| (((long)readUnsignedByte()) << 48)
| (((long)readUnsignedByte()) << 40)
| (((long)readUnsignedByte()) << 32)
| (((long)readUnsignedByte()) << 24)
| (readUnsignedByte() << 16)
| (readUnsignedByte() << 8)
| (readUnsignedByte());
}
public float readFloat() throws IOException {
return Float.intBitsToFloat(readInt());
}
public double readDouble() throws IOException {
return Double.longBitsToDouble(readLong());
}
public String readLine() throws IOException {
return new DataInputStream(this).readLine();
}
public String readUTF() throws IOException {
return new DataInputStream(this).readUTF();
}
}

View File

@ -0,0 +1,188 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.solr.common.util;
import java.io.*;
/** Single threaded buffered OutputStream
* Internal Solr use only, subject to change.
*/
public class FastOutputStream extends OutputStream implements DataOutput {
private final OutputStream out;
private final byte[] buf;
private long written; // how many bytes written
private int pos;
public FastOutputStream(OutputStream w) {
// use default BUFSIZE of BufferedOutputStream so if we wrap that
// it won't cause double buffering.
this(w, new byte[8192], 0);
}
public FastOutputStream(OutputStream sink, byte[] tempBuffer, int start) {
this.out = sink;
this.buf = tempBuffer;
this.pos = start;
}
public static FastOutputStream wrap(OutputStream sink) {
return (sink instanceof FastOutputStream) ? (FastOutputStream)sink : new FastOutputStream(sink);
}
@Override
public void write(int b) throws IOException {
write((byte)b);
}
public void write(byte b[]) throws IOException {
}
public void write(byte b) throws IOException {
if (pos >= buf.length) {
out.write(buf);
written += pos;
pos=0;
}
buf[pos++] = b;
}
@Override
public void write(byte arr[], int off, int len) throws IOException {
int space = buf.length - pos;
if (len < space) {
System.arraycopy(arr, off, buf, pos, len);
pos += len;
} else if (len<buf.length) {
// if the data to write is small enough, buffer it.
System.arraycopy(arr, off, buf, pos, space);
out.write(buf);
written += pos;
pos = len-space;
System.arraycopy(arr, off+space, buf, 0, pos);
} else {
out.write(buf,0,pos); // flush
written += pos;
pos=0;
// don't buffer, just write to sink
out.write(arr, off, len);
written += len;
}
}
/** reserve at least len bytes at the end of the buffer.
* Invalid if len > buffer.length
* @param len
*/
public void reserve(int len) throws IOException {
if (len > (buf.length - pos))
flushBuffer();
}
////////////////// DataOutput methods ///////////////////
public void writeBoolean(boolean v) throws IOException {
write(v ? 1:0);
}
public void writeByte(int v) throws IOException {
write((byte)v);
}
public void writeShort(int v) throws IOException {
write((byte)(v >>> 8));
write((byte)v);
}
public void writeChar(int v) throws IOException {
writeShort(v);
}
public void writeInt(int v) throws IOException {
reserve(4);
buf[pos] = (byte)(v>>>24);
buf[pos+1] = (byte)(v>>>16);
buf[pos+2] = (byte)(v>>>8);
buf[pos+3] = (byte)(v);
pos+=4;
}
public void writeLong(long v) throws IOException {
reserve(8);
buf[pos] = (byte)(v>>>56);
buf[pos+1] = (byte)(v>>>48);
buf[pos+2] = (byte)(v>>>40);
buf[pos+3] = (byte)(v>>>32);
buf[pos+4] = (byte)(v>>>24);
buf[pos+5] = (byte)(v>>>16);
buf[pos+6] = (byte)(v>>>8);
buf[pos+7] = (byte)(v);
pos+=8;
}
public void writeFloat(float v) throws IOException {
writeInt(Float.floatToRawIntBits(v));
}
public void writeDouble(double v) throws IOException {
writeLong(Double.doubleToRawLongBits(v));
}
public void writeBytes(String s) throws IOException {
// non-optimized version, but this shouldn't be used anyway
for (int i=0; i<s.length(); i++)
write((byte)s.charAt(i));
}
public void writeChars(String s) throws IOException {
// non-optimized version
for (int i=0; i<s.length(); i++)
writeChar(s.charAt(i));
}
public void writeUTF(String s) throws IOException {
// non-optimized version, but this shouldn't be used anyway
DataOutputStream daos = new DataOutputStream(this);
daos.writeUTF(s);
}
@Override
public void flush() throws IOException {
flushBuffer();
out.flush();
}
@Override
public void close() throws IOException {
flushBuffer();
out.close();
}
/** Only flushes the buffer of the FastOutputStream, not that of the
* underlying stream.
*/
public void flushBuffer() throws IOException {
out.write(buf, 0, pos);
written += pos;
pos=0;
}
public long size() {
return written + pos;
}
}

View File

@ -0,0 +1,565 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.solr.common.util;
import org.apache.solr.common.SolrDocument;
import org.apache.solr.common.SolrDocumentList;
import java.io.*;
import java.util.*;
/**
* The class is designed to optimaly serialize/deserialize a NamedList. As we know there are only
* a limited type of items this class can do it with very minimal amount of payload and code. There are
* 15 known types and if there is an object in the object tree which does not fall into these types, It must be
* converted to one of these. Implement an ObjectResolver and pass it over
* It is expected that this class is used on both end of the pipes.
* The class has one read method and one write method for each of the datatypes
*
*/
public class NamedListCodec {
public static final byte
NULL = 0,
BOOL_TRUE = 1,
BOOL_FALSE = 2,
BYTE = 3,
SHORT = 4,
DOUBLE = 5,
INT = 6,
LONG = 7,
FLOAT = 8,
DATE = 9,
MAP = 10,
SOLRDOC = 11,
SOLRDOCLST = 12,
BYTEARR = 13,
// types that combine tag + length (or other info) in a single byte
TAG_AND_LEN=(byte)(1 << 5),
STR = (byte)(1 << 5),
SINT = (byte)(2 << 5),
SLONG = (byte)(3 << 5),
ARR = (byte)(4 << 5), //
ORDERED_MAP=(byte)(5 << 5), // SimpleOrderedMap (a NamedList subclass, and more common)
NAMED_LST = (byte)(6 << 5), // NamedList
RESERVED2 = (byte)(7 << 5);
private byte VERSION = 1;
private ObjectResolver resolver;
private FastOutputStream daos;
public NamedListCodec() { }
public NamedListCodec(ObjectResolver resolver) {
this.resolver = resolver;
}
public void marshal(NamedList nl, OutputStream os) throws IOException {
daos = FastOutputStream.wrap(os);
try {
daos.writeByte(VERSION);
writeNamedList(nl);
} finally {
daos.flushBuffer();
}
}
public NamedList unmarshal(InputStream is) throws IOException {
FastInputStream dis = FastInputStream.wrap(is);
byte version = dis.readByte();
return (NamedList)readVal(dis);
}
private SimpleOrderedMap readOrderedMap(FastInputStream dis) throws IOException {
int sz = readSize(dis);
SimpleOrderedMap nl = new SimpleOrderedMap();
for (int i = 0; i < sz; i++) {
String name = (String)readVal(dis);
Object val = readVal(dis);
nl.add(name, val);
}
return nl;
}
private NamedList readNamedList(FastInputStream dis) throws IOException {
int sz = readSize(dis);
NamedList nl = new NamedList();
for (int i = 0; i < sz; i++) {
String name = (String)readVal(dis);
Object val = readVal(dis);
nl.add(name, val);
}
return nl;
}
private void writeNamedList(NamedList nl) throws IOException {
writeTag(nl instanceof SimpleOrderedMap ? ORDERED_MAP : NAMED_LST, nl.size());
for (int i = 0; i < nl.size(); i++) {
String name = nl.getName(i);
writeStr(name);
Object val = nl.getVal(i);
writeVal(val);
}
}
public void writeVal(Object val) throws IOException {
if (writeKnownType(val)) {
return;
} else {
Object tmpVal = val;
if(resolver !=null) {
tmpVal = resolver.resolve(val, this);
if (tmpVal == null) return; // null means the resolver took care of it fully
if(writeKnownType(tmpVal)) return;
}
}
writeVal(val.getClass().getName() + ':' + val.toString());
}
byte tagByte;
private Object readVal(FastInputStream dis) throws IOException {
tagByte = dis.readByte();
// if ((tagByte & 0xe0) == 0) {
// if top 3 bits are clear, this is a normal tag
// OK, try type + size in single byte
switch(tagByte>>>5) {
case STR >>> 5 : return readStr(dis);
case SINT >>>5 : return readSmallInt(dis);
case SLONG >>>5 : return readSmallLong(dis);
case ARR >>> 5 : return readArray(dis);
case ORDERED_MAP >>> 5 : return readOrderedMap(dis);
case NAMED_LST >>> 5 : return readNamedList(dis);
}
switch(tagByte){
case NULL : return null;
case DATE : return new Date(dis.readLong());
case INT : return dis.readInt();
case BOOL_TRUE : return Boolean.TRUE;
case BOOL_FALSE : return Boolean.FALSE;
case FLOAT : return dis.readFloat();
case DOUBLE : return dis.readDouble();
case LONG : return dis.readLong();
case BYTE : return dis.readByte();
case SHORT : return dis.readShort();
case MAP : return readMap(dis);
case SOLRDOC : return readSolrDocument(dis);
case SOLRDOCLST : return readSolrDocumentList(dis);
case BYTEARR : return readByteArray(dis);
}
throw new RuntimeException("Unknown type " + tagByte);
}
private boolean writeKnownType(Object val) throws IOException {
if (writePrimitive(val)) return true;
if (val instanceof NamedList) {
writeNamedList((NamedList) val);
return true;
}
if (val instanceof SolrDocumentList) { // SolrDocumentList is a List, so must come before List check
writeSolrDocumentList((SolrDocumentList) val);
return true;
}
if (val instanceof List) {
writeArray((List) val);
return true;
}
if (val instanceof Object[]) {
writeArray((Object[]) val);
return true;
}
if (val instanceof SolrDocument) {
writeSolrDocument((SolrDocument) val);
return true;
}
if (val instanceof Iterator) {
writeIterator((Iterator) val);
return true;
}
if (val instanceof Map) {
writeMap((Map) val);
return true;
}
return false;
}
public void writeTag(byte tag) throws IOException {
daos.writeByte(tag);
}
public void writeTag(byte tag, int size) throws IOException {
if ((tag & 0xe0) != 0) {
if (size < 0x1f) {
daos.writeByte(tag | size);
} else {
daos.writeByte(tag | 0x1f);
writeVInt(size-0x1f, daos);
}
} else {
daos.writeByte(tag);
writeVInt(size, daos);
}
}
private void writeByteArray(byte[] arr, int offset, int len) throws IOException {
writeTag(BYTEARR, len);
daos.write(arr, offset, len);
}
private byte[] readByteArray(FastInputStream dis) throws IOException {
byte[] arr = new byte[readVInt(dis)];
dis.readFully(arr);
return arr;
}
public void writeSolrDocument(SolrDocument doc) throws IOException {
writeTag(SOLRDOC);
writeTag(ORDERED_MAP, doc.getFieldNames().size());
for (Map.Entry<String, Object> entry : doc) {
String name = entry.getKey();
writeStr(name);
Object val = entry.getValue();
writeVal(val);
}
}
private SolrDocument readSolrDocument(FastInputStream dis) throws IOException {
NamedList nl = (NamedList) readVal(dis);
SolrDocument doc = new SolrDocument();
for (int i = 0; i < nl.size(); i++) {
String name = nl.getName(i);
Object val = nl.getVal(i);
doc.setField(name, val);
}
return doc;
}
private SolrDocumentList readSolrDocumentList(FastInputStream dis) throws IOException {
SolrDocumentList solrDocs = new SolrDocumentList();
List list = (List) readVal(dis);
solrDocs.setNumFound((Long) list.get(0));
solrDocs.setStart((Long)list.get(1));
solrDocs.setMaxScore((Float)list.get(2));
List l = (List) readVal(dis);
solrDocs.addAll(l);
return solrDocs;
}
private void writeSolrDocumentList(SolrDocumentList docs)
throws IOException {
writeTag(SOLRDOCLST);
List l = new ArrayList(3);
l.add(docs.getNumFound());
l.add(docs.getStart());
l.add(docs.getMaxScore());
writeArray(l);
writeArray(docs);
}
private Map readMap(FastInputStream dis)
throws IOException {
int sz = readVInt(dis);
Map m = new LinkedHashMap();
for (int i = 0; i < sz; i++) {
Object key = readVal(dis);
Object val = readVal(dis);
m.put(key, val);
}
return m;
}
private void writeIterator(Iterator iter) throws IOException {
ArrayList l = new ArrayList();
while (iter.hasNext()) l.add(iter.next());
writeArray(l);
}
public void writeArray(List l) throws IOException {
writeTag(ARR, l.size());
for (int i = 0; i < l.size(); i++) {
writeVal(l.get(i));
}
}
public void writeArray(Object[] arr) throws IOException {
writeTag(ARR, arr.length);
for (int i = 0; i < arr.length; i++) {
Object o = arr[i];
writeVal(o);
}
}
private List readArray(FastInputStream dis) throws IOException {
int sz = readSize(dis);
ArrayList l = new ArrayList(sz);
for (int i = 0; i < sz; i++) {
l.add(readVal(dis));
}
return l;
}
/** write the string as tag+length, with length being the number of UTF-16 characters,
* followed by the string encoded in modified-UTF8
*/
private void writeStr(String s) throws IOException {
if (s==null) {
writeTag(NULL);
}
// Can't use string serialization or toUTF()... it's limited to 64K
// plus it's bigger than it needs to be for small strings anyway
int len = s.length();
writeTag(STR, len);
writeChars(daos, s, 0, len);
}
char[] charArr;
private String readStr(FastInputStream dis) throws IOException {
int sz = readSize(dis);
if (charArr==null || charArr.length < sz) {
charArr = new char[sz];
}
readChars(dis, charArr, 0, sz);
return new String(charArr, 0, sz);
}
private void writeInt(int val) throws IOException {
if (val>0) {
int b = SINT | (val & 0x0f);
if (val >= 0x0f) {
b |= 0x10;
daos.writeByte(b);
writeVInt(val>>>4, daos);
} else {
daos.writeByte(b);
}
} else {
daos.writeByte(INT);
daos.writeInt(val);
}
}
private int readSmallInt(FastInputStream dis) throws IOException {
int v = tagByte & 0x0F;
if ((tagByte & 0x10) != 0)
v = (readVInt(dis)<<4) | v;
return v;
}
private void writeLong(long val) throws IOException {
if ((val & 0xff00000000000000L) == 0) {
int b = SLONG | ((int)val & 0x0f);
if (val >= 0x0f) {
b |= 0x10;
daos.writeByte(b);
writeVLong(val>>>4, daos);
} else {
daos.writeByte(b);
}
} else {
daos.writeByte(LONG);
daos.writeLong(val);
}
}
private long readSmallLong(FastInputStream dis) throws IOException {
long v = tagByte & 0x0F;
if ((tagByte & 0x10) != 0)
v = (readVLong(dis)<<4) | v;
return v;
}
private boolean writePrimitive(Object val) throws IOException {
if (val == null) {
daos.writeByte(NULL);
return true;
} else if (val instanceof String) {
writeStr((String)val);
return true;
} else if (val instanceof Integer) {
writeInt(((Integer)val).intValue());
return true;
} else if (val instanceof Long) {
writeLong(((Long)val).longValue());
return true;
} else if (val instanceof Float) {
daos.writeByte(FLOAT);
daos.writeFloat(((Float) val).floatValue());
return true;
} else if (val instanceof Date) {
daos.writeByte(DATE);
daos.writeLong(((Date) val).getTime());
return true;
} else if (val instanceof Boolean) {
if ((Boolean) val) daos.writeByte(BOOL_TRUE);
else daos.writeByte(BOOL_FALSE);
return true;
} else if (val instanceof Double) {
daos.writeByte(DOUBLE);
daos.writeDouble(((Double) val).doubleValue());
return true;
} else if (val instanceof Byte) {
daos.writeByte(BYTE);
daos.writeByte(((Byte) val).intValue());
return true;
} else if (val instanceof Short) {
daos.writeByte(SHORT);
daos.writeShort(((Short) val).intValue());
return true;
} else if (val instanceof byte[]) {
writeByteArray((byte[])val, 0, ((byte[])val).length);
return true;
}
return false;
}
private void writeMap( Map val)
throws IOException {
writeTag(MAP, val.size());
for (Map.Entry entry : (Set<Map.Entry>) val.entrySet()) {
writeVal(entry.getKey());
writeVal(entry.getValue());
}
}
private int readSize(FastInputStream in) throws IOException {
int sz = tagByte & 0x1f;
if (sz == 0x1f) sz += readVInt(in);
return sz;
}
/**
* Special method for variable length int (copied from lucene). Usually used for writing the length of a collection/array/map
* In most of the cases the length can be represented in one byte (length < 127) so it saves 3 bytes/object
*
* @param i
* @param out
* @throws IOException
*/
private static void writeVInt(int i, FastOutputStream out) throws IOException {
while ((i & ~0x7F) != 0) {
out.writeByte((byte) ((i & 0x7f) | 0x80));
i >>>= 7;
}
out.writeByte((byte) i);
}
/**
* The counterpart for the above
*
* @param in
* @return the int value
* @throws IOException
*/
public static int readVInt(FastInputStream in) throws IOException {
byte b = in.readByte();
int i = b & 0x7F;
for (int shift = 7; (b & 0x80) != 0; shift += 7) {
b = in.readByte();
i |= (b & 0x7F) << shift;
}
return i;
}
private static void writeVLong(long i, FastOutputStream out) throws IOException {
while ((i & ~0x7F) != 0) {
out.writeByte((byte)((i & 0x7f) | 0x80));
i >>>= 7;
}
out.writeByte((byte) i);
}
private static long readVLong(FastInputStream in) throws IOException {
byte b = in.readByte();
long i = b & 0x7F;
for (int shift = 7; (b & 0x80) != 0; shift += 7) {
b = in.readByte();
i |= (long)(b & 0x7F) << shift;
}
return i;
}
/** Writes a sequence of UTF-8 encoded characters from a string.
* @param s the source of the characters
* @param start the first character in the sequence
* @param length the number of characters in the sequence
* @see org.apache.lucene.store.IndexInput#readChars(char[],int,int)
*/
public static void writeChars(FastOutputStream os, String s, int start, int length)
throws IOException {
final int end = start + length;
for (int i = start; i < end; i++) {
final int code = (int)s.charAt(i);
if (code >= 0x01 && code <= 0x7F)
os.write(code);
else if (((code >= 0x80) && (code <= 0x7FF)) || code == 0) {
os.write(0xC0 | (code >> 6));
os.write(0x80 | (code & 0x3F));
} else {
os.write(0xE0 | (code >>> 12));
os.write(0x80 | ((code >> 6) & 0x3F));
os.write(0x80 | (code & 0x3F));
}
}
}
/** Reads UTF-8 encoded characters into an array.
* @param buffer the array to read characters into
* @param start the offset in the array to start storing characters
* @param length the number of characters to read
* @see org.apache.lucene.store.IndexOutput#writeChars(String,int,int)
*/
public static void readChars(FastInputStream in, char[] buffer, int start, int length)
throws IOException {
final int end = start + length;
for (int i = start; i < end; i++) {
int b = in.read();
if ((b & 0x80) == 0)
buffer[i] = (char)b;
else if ((b & 0xE0) != 0xE0) {
buffer[i] = (char)(((b & 0x1F) << 6)
| (in.read() & 0x3F));
} else
buffer[i] = (char)(((b & 0x0F) << 12)
| ((in.read() & 0x3F) << 6)
| (in.read() & 0x3F));
}
}
public static interface ObjectResolver{
public Object resolve(Object o, NamedListCodec codec) throws IOException;
}
}

View File

@ -51,15 +51,7 @@ import org.apache.solr.handler.component.QueryComponent;
import org.apache.solr.handler.component.SearchComponent;
import org.apache.solr.highlight.DefaultSolrHighlighter;
import org.apache.solr.highlight.SolrHighlighter;
import org.apache.solr.request.JSONResponseWriter;
import org.apache.solr.request.PythonResponseWriter;
import org.apache.solr.request.QueryResponseWriter;
import org.apache.solr.request.RawResponseWriter;
import org.apache.solr.request.RubyResponseWriter;
import org.apache.solr.request.SolrQueryRequest;
import org.apache.solr.request.SolrQueryResponse;
import org.apache.solr.request.SolrRequestHandler;
import org.apache.solr.request.XMLResponseWriter;
import org.apache.solr.request.*;
import org.apache.solr.schema.IndexSchema;
import org.apache.solr.search.SolrIndexSearcher;
import org.apache.solr.search.QParserPlugin;
@ -1038,6 +1030,9 @@ public final class SolrCore {
if (responseWriters.get("raw")==null) {
responseWriters.put("raw", new RawResponseWriter());
}
if (responseWriters.get("javabin") == null) {
responseWriters.put("javabin", new BinaryResponseWriter());
}
}
/** Finds a writer by name, or returns the default writer if not found. */

View File

@ -26,6 +26,7 @@ import org.apache.solr.common.SolrInputDocument;
import org.apache.solr.common.params.SolrParams;
import org.apache.solr.common.util.ContentStream;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.common.util.SimpleOrderedMap;
import org.apache.solr.request.SolrQueryRequest;
import org.apache.solr.request.SolrQueryResponse;
import org.apache.solr.schema.FieldType;
@ -96,7 +97,7 @@ public class AnalysisRequestHandler extends RequestHandlerBase {
NamedList<Object> processContent(XMLStreamReader parser,
IndexSchema schema) throws XMLStreamException, IOException {
NamedList<Object> result = new NamedList<Object>();
NamedList<Object> result = new SimpleOrderedMap<Object>();
while (true) {
int event = parser.next();
switch (event) {
@ -111,7 +112,7 @@ public class AnalysisRequestHandler extends RequestHandlerBase {
SolrInputDocument doc = readDoc(parser);
SchemaField uniq = schema.getUniqueKeyField();
NamedList<NamedList<NamedList<Object>>> theTokens = new NamedList<NamedList<NamedList<Object>>>();
NamedList<NamedList<NamedList<Object>>> theTokens = new SimpleOrderedMap<NamedList<NamedList<Object>>>();
result.add(doc.getFieldValue(uniq.getName()).toString(), theTokens);
for (String name : doc.getFieldNames()) {
FieldType ft = schema.getFieldType(name);
@ -132,10 +133,11 @@ public class AnalysisRequestHandler extends RequestHandlerBase {
}
static NamedList<NamedList<Object>> getTokens(TokenStream tstream) throws IOException {
// outer is namedList since order of tokens is important
NamedList<NamedList<Object>> tokens = new NamedList<NamedList<Object>>();
Token t = null;
while (((t = tstream.next()) != null)) {
NamedList<Object> token = new NamedList<Object>();
NamedList<Object> token = new SimpleOrderedMap<Object>();
tokens.add("token", token);
token.add("value", new String(t.termBuffer(), 0, t.termLength()));
token.add("start", t.startOffset());

View File

@ -129,7 +129,7 @@ public class DebugComponent extends SearchComponent
}
}
explain = HighlightComponent.removeNulls(new NamedList(Arrays.asList(arr)));
explain = HighlightComponent.removeNulls(new SimpleOrderedMap(Arrays.asList(arr)));
int idx = info.indexOf("explain",0);
if (idx>=0) {
info.setVal(idx, explain);

View File

@ -323,18 +323,18 @@ System.out.println("Got " + facet_q + " , refining count: " + sfc + " += " + cou
FacetInfo fi = rb._facetInfo;
NamedList facet_counts = new NamedList();
NamedList facet_queries = new NamedList();
NamedList facet_counts = new SimpleOrderedMap();
NamedList facet_queries = new SimpleOrderedMap();
facet_counts.add("facet_queries",facet_queries);
for (Map.Entry<String,Long> entry : fi.queryFacets.entrySet()) {
facet_queries.add(entry.getKey(), num(entry.getValue()));
}
NamedList facet_fields = new NamedList();
NamedList facet_fields = new SimpleOrderedMap();
facet_counts.add("facet_fields", facet_fields);
for (DistribFieldFacet dff : fi.topFacets.values()) {
SimpleOrderedMap fieldCounts = new SimpleOrderedMap();
NamedList fieldCounts = new NamedList(); // order is more important for facets
facet_fields.add(dff.field, fieldCounts);
ShardFacetCount[] counts = dff.countSorted;
@ -355,7 +355,7 @@ System.out.println("Got " + facet_q + " , refining count: " + sfc + " += " + cou
// TODO: list facets (sorted by natural order)
// TODO: facet dates
facet_counts.add("facet_dates", new NamedList());
facet_counts.add("facet_dates", new SimpleOrderedMap());
rb.rsp.add("facet_counts", facet_counts);

View File

@ -126,7 +126,7 @@ public class HighlightComponent extends SearchComponent
}
// remove nulls in case not all docs were able to be retrieved
rb.rsp.add("highlighting", removeNulls(new NamedList(Arrays.asList(arr))));
rb.rsp.add("highlighting", removeNulls(new SimpleOrderedMap(Arrays.asList(arr))));
}
}
@ -134,7 +134,7 @@ public class HighlightComponent extends SearchComponent
static NamedList removeNulls(NamedList nl) {
for (int i=0; i<nl.size(); i++) {
if (nl.getName(i)==null) {
NamedList newList = new NamedList();
NamedList newList = nl instanceof SimpleOrderedMap ? new SimpleOrderedMap() : new NamedList();
for (int j=0; j<nl.size(); j++) {
String n = nl.getName(j);
if (n != null) {

View File

@ -172,7 +172,7 @@ public class QueryComponent extends SearchComponent
Sort sort = rb.getSortSpec().getSort();
SortField[] sortFields = sort==null ? new SortField[]{SortField.FIELD_SCORE} : sort.getSort();
ScoreDoc sd = new ScoreDoc(0,1.0f); // won't work for comparators that look at the score
NamedList sortVals = new NamedList();
NamedList sortVals = new NamedList(); // order is important for the sort fields
StringFieldable field = new StringFieldable();
for (SortField sortField: sortFields) {

View File

@ -28,8 +28,10 @@ import org.apache.solr.request.SolrQueryRequest;
import org.apache.solr.request.SolrQueryResponse;
import org.apache.solr.client.solrj.SolrServer;
import org.apache.solr.client.solrj.SolrRequest;
import org.apache.solr.client.solrj.SolrResponse;
import org.apache.solr.client.solrj.request.QueryRequest;
import org.apache.solr.client.solrj.impl.CommonsHttpSolrServer;
import org.apache.solr.client.solrj.impl.BinaryResponseParser;
import org.apache.solr.util.plugin.SolrCoreAware;
import org.apache.solr.core.SolrCore;
import org.apache.lucene.queryParser.ParseException;
@ -316,6 +318,18 @@ class HttpCommComponent {
HttpCommComponent() {
}
private static class SimpleSolrResponse extends SolrResponse {
long elapsedTime;
NamedList<Object> nl;
public long getElapsedTime() {
return elapsedTime;
}
public NamedList<Object> getResponse() {
return nl;
}
}
void submit(final ShardRequest sreq, final String shard, final ModifiableSolrParams params) {
Callable<ShardResponse> task = new Callable<ShardResponse>() {
public ShardResponse call() throws Exception {
@ -323,6 +337,9 @@ class HttpCommComponent {
ShardResponse srsp = new ShardResponse();
srsp.req = sreq;
srsp.shard = shard;
SimpleSolrResponse ssr = new SimpleSolrResponse();
srsp.rsp = ssr;
long startTime = System.currentTimeMillis();
try {
// String url = "http://" + shard + "/select";
@ -332,11 +349,15 @@ class HttpCommComponent {
params.remove("version");
SolrServer server = new CommonsHttpSolrServer(url, client);
// SolrRequest req = new SolrRequest(SolrRequest.METHOD.GET, "/select");
// SolrRequest req = new QueryRequest(SolrRequest.METHOD.POST, "/select");
// use generic request to avoid extra processing of queries
// QueryRequest req = new QueryRequest(sreq.params);
QueryRequest req = new QueryRequest(sreq.params);
req.setMethod(SolrRequest.METHOD.POST);
req.setResponseParser(new BinaryResponseParser()); // this sets the wt param
// srsp.rsp = server.request(req);
srsp.rsp = server.query(sreq.params);
// srsp.rsp = server.query(sreq.params);
ssr.nl = server.request(req);
} catch (Throwable th) {
srsp.exception = th;
if (th instanceof SolrException) {
@ -346,6 +367,8 @@ class HttpCommComponent {
}
}
ssr.elapsedTime = System.currentTimeMillis() - startTime;
return srsp;
}
};

View File

@ -0,0 +1,146 @@
/**
* 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.request;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Fieldable;
import org.apache.solr.common.SolrDocument;
import org.apache.solr.common.SolrDocumentList;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.common.util.NamedListCodec;
import org.apache.solr.schema.IndexSchema;
import org.apache.solr.schema.SchemaField;
import org.apache.solr.schema.TextField;
import org.apache.solr.schema.FieldType;
import org.apache.solr.search.DocIterator;
import org.apache.solr.search.DocList;
import org.apache.solr.search.SolrIndexSearcher;
import java.io.IOException;
import java.io.OutputStream;
import java.io.Writer;
import java.util.*;
public class BinaryResponseWriter implements BinaryQueryResponseWriter {
public void write(OutputStream out, SolrQueryRequest req, SolrQueryResponse response) throws IOException {
Resolver resolver = new Resolver(req, response.getReturnFields());
NamedListCodec codec = new NamedListCodec(resolver);
codec.marshal(response.getValues(), out);
}
public void write(Writer writer, SolrQueryRequest request, SolrQueryResponse response) throws IOException {
throw new RuntimeException("This is a binary writer , Cannot write to a characterstream");
}
public String getContentType(SolrQueryRequest request, SolrQueryResponse response) {
return "application/octet-stream";
}
public void init(NamedList args) {
/* NOOP */
}
private static class Resolver implements NamedListCodec.ObjectResolver {
private final IndexSchema schema;
private final SolrIndexSearcher searcher;
private final Set<String> returnFields;
private final boolean includeScore;
// transmit field values using FieldType.toObject()
// rather than the String from FieldType.toExternal()
boolean useFieldObjects = true;
public Resolver(SolrQueryRequest req, Set<String> returnFields) {
this.schema = req.getSchema();
this.searcher = req.getSearcher();
this.includeScore = returnFields!=null && returnFields.contains("score");
if (returnFields != null) {
if (returnFields.size() == 0 || (returnFields.size() == 1 && includeScore) || returnFields.contains("*")) {
returnFields = null; // null means return all stored fields
}
}
this.returnFields = returnFields;
}
public Object resolve(Object o, NamedListCodec codec) throws IOException {
if (o instanceof DocList) {
writeDocList((DocList) o, codec);
return null; // null means we completely handled it
}
if (o instanceof Document) {
return getDoc((Document) o);
}
return o;
}
public void writeDocList(DocList ids, NamedListCodec codec) throws IOException {
codec.writeTag(NamedListCodec.SOLRDOCLST);
List l = new ArrayList(3);
l.add((long)ids.matches());
l.add((long)ids.offset());
Float maxScore = null;
if (includeScore && ids.hasScores()) {
maxScore = ids.maxScore();
}
l.add(maxScore);
codec.writeArray(l);
int sz = ids.size();
codec.writeTag(NamedListCodec.ARR, sz);
DocIterator iterator = ids.iterator();
for (int i = 0; i < sz; i++) {
int id = iterator.nextDoc();
Document doc = searcher.doc(id, returnFields);
SolrDocument sdoc = getDoc(doc);
if (includeScore && ids.hasScores()) {
sdoc.addField("score", iterator.score());
}
codec.writeSolrDocument(sdoc);
}
}
public SolrDocument getDoc(Document doc) {
SolrDocument solrDoc = new SolrDocument();
for (Fieldable f : (List<Fieldable>)doc.getFields()) {
String fieldName = f.name();
if (returnFields!=null && !returnFields.contains(fieldName)) continue;
FieldType ft = schema.getFieldTypeNoEx(fieldName);
Object val;
if (ft==null) { // handle fields not in the schema
if (f.isBinary()) val = f.binaryValue();
else val = f.stringValue();
} else {
val = useFieldObjects ? ft.toObject(f) : ft.toExternal(f);
}
solrDoc.addField(fieldName, val);
}
return solrDoc;
}
}
}

View File

@ -55,7 +55,7 @@ public class SortableDoubleField extends FieldType {
@Override
public Double toObject(Fieldable f) {
return Double.valueOf( toExternal(f) );
return NumberUtils.SortableStr2double(f.stringValue());
}
public String indexedToReadable(String indexedForm) {

View File

@ -55,7 +55,7 @@ public class SortableFloatField extends FieldType {
@Override
public Float toObject(Fieldable f) {
return Float.valueOf( toExternal(f) );
return NumberUtils.SortableStr2float(f.stringValue());
}
public String indexedToReadable(String indexedForm) {

View File

@ -62,7 +62,7 @@ public class SortableIntField extends FieldType {
@Override
public Integer toObject(Fieldable f) {
return Integer.valueOf( toExternal(f) );
return NumberUtils.SortableStr2int(f.stringValue(), 0, 3);
}
public void write(XMLWriter xmlWriter, String name, Fieldable f) throws IOException {

View File

@ -59,7 +59,7 @@ public class SortableLongField extends FieldType {
@Override
public Long toObject(Fieldable f) {
return Long.valueOf( toExternal(f) );
return NumberUtils.SortableStr2long(f.stringValue(),0,5);
}
public void write(XMLWriter xmlWriter, String name, Fieldable f) throws IOException {

View File

@ -25,6 +25,7 @@ import java.util.logging.Level;
import org.apache.solr.common.params.SolrParams;
import org.apache.solr.common.util.DOMUtil;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.common.util.SimpleOrderedMap;
import org.apache.solr.core.SolrCore;
import org.apache.solr.request.SolrQueryRequest;
import org.apache.solr.request.SolrQueryResponse;
@ -89,7 +90,7 @@ class LogUpdateProcessor extends UpdateRequestProcessor {
// TODO: make log level configurable as well, or is that overkill?
// (ryan) maybe? I added it mostly to show that it *can* be configurable
this.toLog = new NamedList<Object>();
this.toLog = new SimpleOrderedMap<Object>();
}
@Override

View File

@ -48,7 +48,6 @@ import junit.framework.TestCase;
* @since solr 1.3
*/
public class TestDistributedSearch extends TestCase {
Random r = new Random(0);
File testDir;
@ -388,6 +387,18 @@ public class TestDistributedSearch extends TestCase {
return compare((Object[])a, (Object[])b, flags, handle);
}
if (a instanceof byte[] && b instanceof byte[]) {
if (!Arrays.equals((byte[])a, (byte[])b)) {
return ":" + a + "!=" + b;
}
return null;
}
if (a instanceof List && b instanceof List) {
return compare(((List)a).toArray(), ((List)b).toArray(), flags, handle);
}
if (!(a.equals(b))) {
return ":" + a + "!=" + b;
}
@ -397,10 +408,7 @@ public class TestDistributedSearch extends TestCase {
void compareResponses(QueryResponse a, QueryResponse b) {
String cmp;
System.out.println(a);
System.out.println(b);
String cmp;
cmp = compare(a.getResponse(), b.getResponse(), flags, handle);
if (cmp != null) {
System.out.println(a);
@ -422,7 +430,8 @@ public class TestDistributedSearch extends TestCase {
public void doTest() throws Exception {
del("*:*");
index(id,1, i1, 100,t1,"now is the time for all good men");
index(id,1, i1, 100,t1,"now is the time for all good men"
,"foo_f", 1.414f, "foo_b", "true", "foo_d", 1.414d);
index(id,2, i1, 50 ,t1,"to come to the aid of their country.");
index(id,3, i1, 2 ,t1,"how now brown cow");
index(id,4, i1, -100 ,t1,"the quick fox jumped over the lazy dog");
@ -431,7 +440,7 @@ public class TestDistributedSearch extends TestCase {
index(id,7, i1, 123 ,t1,"humpty dumpy had a great fall");
index(id,8, i1, 876 ,t1,"all the kings horses and all the kings men");
index(id,9, i1, 7 ,t1,"couldn't put humpty together again");
index(id,10, i1, 4321 ,t1,"this too shal pass");
index(id,10, i1, 4321 ,t1,"this too shall pass");
index(id,11, i1, -987 ,t1,"An eye for eye only ends up making the whole world blind.");
index(id,12, i1, 379 ,t1,"Great works are performed, not by strength, but by perseverance.");
index(id,13, i1, 232 ,t1,"no eggs on wall, lesson learned", oddField, "odd man out");

View File

@ -0,0 +1,187 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.solr.common.util;
import org.apache.solr.common.SolrDocument;
import org.apache.solr.common.SolrDocumentList;
import org.apache.solr.TestDistributedSearch;
import java.io.ByteArrayOutputStream;
import java.io.ByteArrayInputStream;
import java.util.List;
import java.util.Random;
import java.util.ArrayList;
import junit.framework.TestCase;
public class TestNamedListCodec extends TestCase {
public void testSimple() throws Exception{
ByteArrayOutputStream baos = new ByteArrayOutputStream();
NamedList nl = new NamedList();
Float fval = new Float( 10.01f );
Boolean bval = Boolean.TRUE;
String sval = "12qwaszx";
// Set up a simple document
NamedList r = new NamedList();
nl.add("responseHeader", r);
r.add("status",0);
r.add("QTime",63);
NamedList p = new NamedList();
r.add("params",p);
p.add("rows",10);
p.add("start",0);
p.add("indent","on");
p.add("q","ipod");
SolrDocumentList list = new SolrDocumentList();
nl.add("response", list );
list.setMaxScore(1.0f);
list.setStart(10);
list.setNumFound(12);
SolrDocument doc = new SolrDocument();
doc.addField( "f", fval );
doc.addField( "b", bval );
doc.addField( "s", sval );
doc.addField( "f", 100 );
list.add(doc);
doc = new SolrDocument();
doc.addField( "f", fval );
doc.addField( "b", bval );
doc.addField( "s", sval );
doc.addField( "f", 101 );
list.add(doc);
nl.add("zzz",doc);
new NamedListCodec(null).marshal(nl,baos);
byte[] arr = baos.toByteArray();
nl = new NamedListCodec().unmarshal(new ByteArrayInputStream(arr));
assertEquals(3, nl.size());
assertEquals( "ipod",((NamedList)((NamedList)nl.getVal(0)).get("params")).get("q") );
list = (SolrDocumentList) nl.getVal(1);
assertEquals(12,list.getNumFound() );
assertEquals(10,list.getStart() );
assertEquals(101, ((List)list.get(1).getFieldValue("f")).get(1));
}
int rSz(int orderOfMagnitude) {
int sz = r.nextInt(orderOfMagnitude);
switch (sz) {
case 0: return r.nextInt(10);
case 1: return r.nextInt(100);
case 2: return r.nextInt(1000);
default: return r.nextInt(10000);
}
}
public String rStr(int sz) {
StringBuilder sb = new StringBuilder();
for (int i=0; i<sz; i++) {
sb.appendCodePoint(r.nextInt(Character.MIN_HIGH_SURROGATE));
}
return sb.toString();
}
public NamedList rNamedList(int lev) {
int sz = lev<= 0 ? 0 : r.nextInt(3);
NamedList nl = new NamedList();
for (int i=0; i<sz; i++) {
nl.add(rStr(2), makeRandom(lev-1));
}
return nl;
}
public List rList(int lev) {
int sz = lev<= 0 ? 0 : r.nextInt(3);
ArrayList lst = new ArrayList();
for (int i=0; i<sz; i++) {
lst.add(makeRandom(lev-1));
}
return lst;
}
Random r = new Random();
public Object makeRandom(int lev) {
switch (r.nextInt(10)) {
case 0:
return rList(lev);
case 1:
return rNamedList(lev);
case 2:
return rStr(rSz(4));
case 3:
return r.nextInt();
case 4:
return r.nextLong();
case 5:
return r.nextBoolean();
case 6:
byte[] arr = new byte[rSz(4)];
r.nextBytes(arr);
return arr;
case 7:
return r.nextFloat();
case 8:
return r.nextDouble();
default:
return null;
}
}
public void testRandom() throws Exception {
Random r = new Random(0);
// let's keep it deterministic since just the wrong
// random stuff could cause failure because of an OOM (too big)
NamedList nl;
NamedList res;
String cmp;
for (int i=0; i<10000; i++) { // pump up the iterations for good stress testing
nl = rNamedList(3);
ByteArrayOutputStream baos = new ByteArrayOutputStream();
new NamedListCodec(null).marshal(nl,baos);
byte[] arr = baos.toByteArray();
// System.out.println(arr.length);
res = new NamedListCodec().unmarshal(new ByteArrayInputStream(arr));
cmp = TestDistributedSearch.compare(nl,res, 0, null);
if (cmp != null) {
System.out.println(nl);
System.out.println(res);
TestCase.fail(cmp);
}
}
}
}

View File

@ -0,0 +1,178 @@
/**
* 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.request;
import org.apache.solr.util.AbstractSolrTestCase;
import org.apache.solr.client.solrj.ResponseParser;
import org.apache.solr.client.solrj.impl.BinaryResponseParser;
import org.apache.solr.client.solrj.impl.XMLResponseParser;
import java.util.ArrayList;
import java.io.*;
public class TestWriterPerf extends AbstractSolrTestCase {
public String getSchemaFile() { return "schema11.xml"; }
public String getSolrConfigFile() { return "solrconfig-functionquery.xml"; }
public String getCoreName() { return "basic"; }
public void setUp() throws Exception {
// if you override setUp or tearDown, you better call
// the super classes version
super.setUp();
}
public void tearDown() throws Exception {
// if you override setUp or tearDown, you better call
// the super classes version
super.tearDown();
}
String id = "id";
String t1 = "f_t";
String i1 = "f_i";
String tag = "f_ss";
void index(Object... olst) {
ArrayList<String> lst = new ArrayList<String>();
for (Object o : olst) lst.add(o.toString());
assertU(adoc(lst.toArray(new String[lst.size()])));
}
void makeIndex() {
index(id,1, i1, 100,t1,"now is the time for all good men", tag,"patriotic");
index(id,2, i1, 50 ,t1,"to come to the aid of their country.", tag,"patriotic",tag,"country",tag,"nation",tag,"speeches");
index(id,3, i1, 2 ,t1,"how now brown cow", tag,"cow",tag,"jersey");
index(id,4, i1, -100 ,t1,"the quick fox jumped over the lazy dog",tag,"fox",tag,"dog",tag,"quick",tag,"slow",tag,"lazy");
index(id,5, i1, 50 ,t1,"the quick fox jumped way over the lazy dog",tag,"fox",tag,"dog");
index(id,6, i1, -60 ,t1,"humpty dumpy sat on a wall",tag,"humpty",tag,"dumpty");
index(id,7, i1, 123 ,t1,"humpty dumpy had a great fall",tag,"accidents");
index(id,8, i1, 876 ,t1,"all the kings horses and all the kings men",tag,"king",tag,"horses",tag,"trouble");
index(id,9, i1, 7 ,t1,"couldn't put humpty together again",tag,"humpty",tag,"broken");
index(id,10, i1, 4321 ,t1,"this too shall pass",tag,"1",tag,"2",tag,"infinity");
index(id,11, i1, 33 ,t1,"An eye for eye only ends up making the whole world blind.",tag,"ouch",tag,"eye",tag,"peace",tag,"world");
index(id,12, i1, 379 ,t1,"Great works are performed, not by strength, but by perseverance.",tag,"herculese",tag,"strong",tag,"stubborn");
assertU(optimize());
}
/** make sure to close req after you are done using the response */
public SolrQueryResponse getResponse(SolrQueryRequest req) throws IOException, Exception {
SolrQueryResponse rsp = new SolrQueryResponse();
h.getCore().execute(h.getCore().getRequestHandler(null),req,rsp);
if (rsp.getException() != null) {
throw rsp.getException();
}
return rsp;
}
void doPerf(String writerName, SolrQueryRequest req, int encIter, int decIter) throws Exception {
SolrQueryResponse rsp = getResponse(req);
QueryResponseWriter w = h.getCore().getQueryResponseWriter(writerName);
ByteArrayOutputStream out=null;
System.gc();
long start = System.currentTimeMillis();
for (int i=0; i<encIter; i++) {
if (w instanceof BinaryQueryResponseWriter) {
BinaryQueryResponseWriter binWriter = (BinaryQueryResponseWriter) w;
out = new ByteArrayOutputStream();
binWriter.write(out, req, rsp);
out.close();
} else {
out = new ByteArrayOutputStream();
// to be fair, from my previous tests, much of the performance will be sucked up
// by java's UTF-8 encoding/decoding, not the actual writing
Writer writer = new OutputStreamWriter(out, "UTF-8");
w.write(writer, req, rsp);
writer.close();
}
}
long encodeTime = Math.max(System.currentTimeMillis() - start, 1);
byte[] arr = out.toByteArray();
start = System.currentTimeMillis();
writerName = writerName.intern();
for (int i=0; i<decIter; i++) {
ResponseParser rp = null;
if (writerName == "xml") {
rp = new XMLResponseParser();
} else if (writerName == "javabin") {
rp = new BinaryResponseParser();
} else {
break;
}
ByteArrayInputStream in = new ByteArrayInputStream(arr);
rp.processResponse(in, "UTF-8");
}
long decodeTime = Math.max(System.currentTimeMillis() - start, 1);
System.out.println("writer "+writerName+", size="+out.size()+", encodeRate="+(encodeTime==1 ? "N/A": ""+(encIter*1000L/encodeTime)) + ", decodeRate="+(decodeTime==1 ? "N/A": ""+(decIter*1000L/decodeTime)) );
req.close();
}
public void testPerf() throws Exception {
makeIndex();
SolrQueryRequest req = req("q", "id:[* TO *] all country"
,"start","0"
,"rows","100"
,"echoParams","all"
,"fl","*,score"
,"indent","false"
,"facet","true"
,"facet.field", i1
,"facet.field", tag
,"facet.field", t1
,"facet.mincount","0"
,"facet.offset","0"
,"facet.limit","100"
,"facet.sort","true"
,"hl","true"
,"hl.fl","t1"
);
// just for testing
doPerf("xml", req, 2,2);
doPerf("json", req, 2,2);
doPerf("javabin", req, 2,2);
int encIter=20000;
int decIter=50000;
// warm up hotspot
// doPerf("xml", req, 200,1000);
// doPerf("json", req, 200,1000);
// doPerf("javabin", req, 200,1000);
// doPerf("xml", req, encIter, decIter);
// doPerf("json", req, encIter, decIter);
//doPerf("javabin", req, encIter, decIter);
// doPerf("javabin", req, 1, decIter);
}
}

View File

@ -271,6 +271,7 @@
Longer patterns will be matched first. if equal size patterns
both match, the first appearing in the schema will be used. -->
<dynamicField name="*_s" type="string" indexed="true" stored="true"/>
<dynamicField name="*_ss" type="string" indexed="true" stored="true" multiValued="true"/>
<dynamicField name="*_i" type="sint" indexed="true" stored="true"/>
<dynamicField name="*_l" type="slong" indexed="true" stored="true"/>
<dynamicField name="*_f" type="sfloat" indexed="true" stored="true"/>

View File

@ -148,7 +148,7 @@ public class SolrDispatchFilter implements Filter
return mcore;
}
public void destroy() {
if (multicore != null) {
multicore.shutdown();