Merge remote-tracking branch 'origin/branch_6x' into branch_6x

Conflicts:
	solr/core/src/java/org/apache/solr/update/TransactionLog.java
This commit is contained in:
Noble Paul 2016-08-18 16:31:41 +05:30
commit 20a6818f1b
64 changed files with 4135 additions and 1591 deletions

View File

@ -11,6 +11,9 @@ API Changes
New Features
* LUCENE-7388: Add point based IntRangeField, FloatRangeField, LongRangeField along with
supporting queries and tests (Nick Knize)
* LUCENE-7381: Add point based DoubleRangeField and RangeFieldQuery for
indexing and querying on Ranges up to 4 dimensions (Nick Knize)
@ -68,6 +71,9 @@ Bug Fixes
norm and coordination factor using a default similarity added as ctor param.
(Uwe Schindler, Sascha Markus)
* SOLR-9413: Fix analysis/kuromoji's CSVUtil.quoteEscape logic, add TestCSVUtil test.
(AppChecker, Christine Poerschke)
Improvements
* LUCENE-7323: Compound file writing now verifies the incoming
@ -125,6 +131,13 @@ Improvements
because the ICU word-breaking algorithm has some issues. This allows for the previous
tokenization used before Lucene 5. (AM, Robert Muir)
* LUCENE-7409: Changed MMapDirectory's unmapping to work safer, but still with
no guarantees. This uses a store-store barrier and yields the current thread
before unmapping to allow in-flight requests to finish. The new code no longer
uses WeakIdentityMap as it delegates all ByteBuffer reads throgh a new
ByteBufferGuard wrapper that is shared between all ByteBufferIndexInput clones.
(Robert Muir, Uwe Schindler)
Optimizations
* LUCENE-7330, LUCENE-7339: Speed up conjunction queries. (Adrien Grand)

View File

@ -101,7 +101,7 @@ public final class CSVUtil {
String result = original;
if (result.indexOf('\"') >= 0) {
result.replace("\"", ESCAPED_QUOTE);
result = result.replace("\"", ESCAPED_QUOTE);
}
if(result.indexOf(COMMA) >= 0) {
result = "\"" + result + "\"";

View File

@ -0,0 +1,52 @@
/*
* 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.lucene.analysis.ja;
import java.io.IOException;
import org.apache.lucene.analysis.ja.util.CSVUtil;
import org.apache.lucene.util.LuceneTestCase;
/*
* Tests for the CSVUtil class.
*/
public class TestCSVUtil extends LuceneTestCase {
public void testQuoteEscapeQuotes() throws IOException {
final String input = "\"Let It Be\" is a song and album by the The Beatles.";
final String expectedOutput = input.replace("\"", "\"\"");
implTestQuoteEscape(input, expectedOutput);
}
public void testQuoteEscapeComma() throws IOException {
final String input = "To be, or not to be ...";
final String expectedOutput = '"'+input+'"';
implTestQuoteEscape(input, expectedOutput);
}
public void testQuoteEscapeQuotesAndComma() throws IOException {
final String input = "\"To be, or not to be ...\" is a well-known phrase from Shakespeare's Hamlet.";
final String expectedOutput = '"'+input.replace("\"", "\"\"")+'"';
implTestQuoteEscape(input, expectedOutput);
}
private void implTestQuoteEscape(String input, String expectedOutput) throws IOException {
final String actualOutput = CSVUtil.quoteEscape(input);
assertEquals(expectedOutput, actualOutput);
}
}

View File

@ -186,4 +186,33 @@ public class Rectangle {
return new Rectangle(minLat, maxLat, minLon, maxLon);
}
@Override
public boolean equals(Object o) {
if (this == o) return true;
if (o == null || getClass() != o.getClass()) return false;
Rectangle rectangle = (Rectangle) o;
if (Double.compare(rectangle.minLat, minLat) != 0) return false;
if (Double.compare(rectangle.minLon, minLon) != 0) return false;
if (Double.compare(rectangle.maxLat, maxLat) != 0) return false;
return Double.compare(rectangle.maxLon, maxLon) == 0;
}
@Override
public int hashCode() {
int result;
long temp;
temp = Double.doubleToLongBits(minLat);
result = (int) (temp ^ (temp >>> 32));
temp = Double.doubleToLongBits(minLon);
result = 31 * result + (int) (temp ^ (temp >>> 32));
temp = Double.doubleToLongBits(maxLat);
result = 31 * result + (int) (temp ^ (temp >>> 32));
temp = Double.doubleToLongBits(maxLon);
result = 31 * result + (int) (temp ^ (temp >>> 32));
return result;
}
}

View File

@ -0,0 +1,136 @@
/*
* 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.lucene.store;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.util.concurrent.atomic.AtomicInteger;
/**
* A guard that is created for every {@link ByteBufferIndexInput} that tries on best effort
* to reject any access to the {@link ByteBuffer} behind, once it is unmapped. A single instance
* of this is used for the original and all clones, so once the original is closed and unmapped
* all clones also throw {@link AlreadyClosedException}, triggered by a {@link NullPointerException}.
* <p>
* This code tries to hopefully flush any CPU caches using a store-store barrier. It also yields the
* current thread to give other threads a chance to finish in-flight requests...
*/
final class ByteBufferGuard {
/**
* Pass in an implementation of this interface to cleanup ByteBuffers.
* MMapDirectory implements this to allow unmapping of bytebuffers with private Java APIs.
*/
@FunctionalInterface
static interface BufferCleaner {
void freeBuffer(String resourceDescription, ByteBuffer b) throws IOException;
}
private final String resourceDescription;
private final BufferCleaner cleaner;
/** Not volatile; see comments on visibility below! */
private boolean invalidated = false;
/** Used as a store-store barrier; see comments below! */
private final AtomicInteger barrier = new AtomicInteger();
/**
* Creates an instance to be used for a single {@link ByteBufferIndexInput} which
* must be shared by all of its clones.
*/
public ByteBufferGuard(String resourceDescription, BufferCleaner cleaner) {
this.resourceDescription = resourceDescription;
this.cleaner = cleaner;
}
/**
* Invalidates this guard and unmaps (if supported).
*/
public void invalidateAndUnmap(ByteBuffer... bufs) throws IOException {
if (cleaner != null) {
invalidated = true;
// This call should hopefully flush any CPU caches and as a result make
// the "invalidated" field update visible to other threads. We specifically
// don't make "invalidated" field volatile for performance reasons, hoping the
// JVM won't optimize away reads of that field and hardware should ensure
// caches are in sync after this call. This isn't entirely "fool-proof"
// (see LUCENE-7409 discussion), but it has been shown to work in practice
// and we count on this behavior.
barrier.lazySet(0);
// we give other threads a bit of time to finish reads on their ByteBuffer...:
Thread.yield();
// finally unmap the ByteBuffers:
for (ByteBuffer b : bufs) {
cleaner.freeBuffer(resourceDescription, b);
}
}
}
private void ensureValid() {
if (invalidated) {
// this triggers an AlreadyClosedException in ByteBufferIndexInput:
throw new NullPointerException();
}
}
public void getBytes(ByteBuffer receiver, byte[] dst, int offset, int length) {
ensureValid();
receiver.get(dst, offset, length);
}
public byte getByte(ByteBuffer receiver) {
ensureValid();
return receiver.get();
}
public short getShort(ByteBuffer receiver) {
ensureValid();
return receiver.getShort();
}
public int getInt(ByteBuffer receiver) {
ensureValid();
return receiver.getInt();
}
public long getLong(ByteBuffer receiver) {
ensureValid();
return receiver.getLong();
}
public byte getByte(ByteBuffer receiver, int pos) {
ensureValid();
return receiver.get(pos);
}
public short getShort(ByteBuffer receiver, int pos) {
ensureValid();
return receiver.getShort(pos);
}
public int getInt(ByteBuffer receiver, int pos) {
ensureValid();
return receiver.getInt(pos);
}
public long getLong(ByteBuffer receiver, int pos) {
ensureValid();
return receiver.getLong(pos);
}
}

View File

@ -21,9 +21,6 @@ import java.io.EOFException;
import java.io.IOException;
import java.nio.BufferUnderflowException;
import java.nio.ByteBuffer;
import java.util.Iterator;
import org.apache.lucene.util.WeakIdentityMap;
/**
* Base IndexInput implementation that uses an array
@ -37,35 +34,32 @@ import org.apache.lucene.util.WeakIdentityMap;
* are a power-of-two (<code>chunkSizePower</code>).
*/
abstract class ByteBufferIndexInput extends IndexInput implements RandomAccessInput {
protected final BufferCleaner cleaner;
protected final long length;
protected final long chunkSizeMask;
protected final int chunkSizePower;
protected final ByteBufferGuard guard;
protected ByteBuffer[] buffers;
protected int curBufIndex = -1;
protected ByteBuffer curBuf; // redundant for speed: buffers[curBufIndex]
protected boolean isClone = false;
protected final WeakIdentityMap<ByteBufferIndexInput,Boolean> clones;
public static ByteBufferIndexInput newInstance(String resourceDescription, ByteBuffer[] buffers, long length, int chunkSizePower, BufferCleaner cleaner, boolean trackClones) {
final WeakIdentityMap<ByteBufferIndexInput,Boolean> clones = trackClones ? WeakIdentityMap.<ByteBufferIndexInput,Boolean>newConcurrentHashMap() : null;
public static ByteBufferIndexInput newInstance(String resourceDescription, ByteBuffer[] buffers, long length, int chunkSizePower, ByteBufferGuard guard) {
if (buffers.length == 1) {
return new SingleBufferImpl(resourceDescription, buffers[0], length, chunkSizePower, cleaner, clones);
return new SingleBufferImpl(resourceDescription, buffers[0], length, chunkSizePower, guard);
} else {
return new MultiBufferImpl(resourceDescription, buffers, 0, length, chunkSizePower, cleaner, clones);
return new MultiBufferImpl(resourceDescription, buffers, 0, length, chunkSizePower, guard);
}
}
ByteBufferIndexInput(String resourceDescription, ByteBuffer[] buffers, long length, int chunkSizePower, BufferCleaner cleaner, WeakIdentityMap<ByteBufferIndexInput,Boolean> clones) {
ByteBufferIndexInput(String resourceDescription, ByteBuffer[] buffers, long length, int chunkSizePower, ByteBufferGuard guard) {
super(resourceDescription);
this.buffers = buffers;
this.length = length;
this.chunkSizePower = chunkSizePower;
this.chunkSizeMask = (1L << chunkSizePower) - 1L;
this.clones = clones;
this.cleaner = cleaner;
this.guard = guard;
assert chunkSizePower >= 0 && chunkSizePower <= 30;
assert (length >>> chunkSizePower) < Integer.MAX_VALUE;
}
@ -73,7 +67,7 @@ abstract class ByteBufferIndexInput extends IndexInput implements RandomAccessIn
@Override
public final byte readByte() throws IOException {
try {
return curBuf.get();
return guard.getByte(curBuf);
} catch (BufferUnderflowException e) {
do {
curBufIndex++;
@ -83,7 +77,7 @@ abstract class ByteBufferIndexInput extends IndexInput implements RandomAccessIn
curBuf = buffers[curBufIndex];
curBuf.position(0);
} while (!curBuf.hasRemaining());
return curBuf.get();
return guard.getByte(curBuf);
} catch (NullPointerException npe) {
throw new AlreadyClosedException("Already closed: " + this);
}
@ -92,11 +86,11 @@ abstract class ByteBufferIndexInput extends IndexInput implements RandomAccessIn
@Override
public final void readBytes(byte[] b, int offset, int len) throws IOException {
try {
curBuf.get(b, offset, len);
guard.getBytes(curBuf, b, offset, len);
} catch (BufferUnderflowException e) {
int curAvail = curBuf.remaining();
while (len > curAvail) {
curBuf.get(b, offset, curAvail);
guard.getBytes(curBuf, b, offset, curAvail);
len -= curAvail;
offset += curAvail;
curBufIndex++;
@ -107,7 +101,7 @@ abstract class ByteBufferIndexInput extends IndexInput implements RandomAccessIn
curBuf.position(0);
curAvail = curBuf.remaining();
}
curBuf.get(b, offset, len);
guard.getBytes(curBuf, b, offset, len);
} catch (NullPointerException npe) {
throw new AlreadyClosedException("Already closed: " + this);
}
@ -116,7 +110,7 @@ abstract class ByteBufferIndexInput extends IndexInput implements RandomAccessIn
@Override
public final short readShort() throws IOException {
try {
return curBuf.getShort();
return guard.getShort(curBuf);
} catch (BufferUnderflowException e) {
return super.readShort();
} catch (NullPointerException npe) {
@ -127,7 +121,7 @@ abstract class ByteBufferIndexInput extends IndexInput implements RandomAccessIn
@Override
public final int readInt() throws IOException {
try {
return curBuf.getInt();
return guard.getInt(curBuf);
} catch (BufferUnderflowException e) {
return super.readInt();
} catch (NullPointerException npe) {
@ -138,7 +132,7 @@ abstract class ByteBufferIndexInput extends IndexInput implements RandomAccessIn
@Override
public final long readLong() throws IOException {
try {
return curBuf.getLong();
return guard.getLong(curBuf);
} catch (BufferUnderflowException e) {
return super.readLong();
} catch (NullPointerException npe) {
@ -181,7 +175,7 @@ abstract class ByteBufferIndexInput extends IndexInput implements RandomAccessIn
public byte readByte(long pos) throws IOException {
try {
final int bi = (int) (pos >> chunkSizePower);
return buffers[bi].get((int) (pos & chunkSizeMask));
return guard.getByte(buffers[bi], (int) (pos & chunkSizeMask));
} catch (IndexOutOfBoundsException ioobe) {
throw new EOFException("seek past EOF: " + this);
} catch (NullPointerException npe) {
@ -207,7 +201,7 @@ abstract class ByteBufferIndexInput extends IndexInput implements RandomAccessIn
public short readShort(long pos) throws IOException {
final int bi = (int) (pos >> chunkSizePower);
try {
return buffers[bi].getShort((int) (pos & chunkSizeMask));
return guard.getShort(buffers[bi], (int) (pos & chunkSizeMask));
} catch (IndexOutOfBoundsException ioobe) {
// either it's a boundary, or read past EOF, fall back:
setPos(pos, bi);
@ -221,7 +215,7 @@ abstract class ByteBufferIndexInput extends IndexInput implements RandomAccessIn
public int readInt(long pos) throws IOException {
final int bi = (int) (pos >> chunkSizePower);
try {
return buffers[bi].getInt((int) (pos & chunkSizeMask));
return guard.getInt(buffers[bi], (int) (pos & chunkSizeMask));
} catch (IndexOutOfBoundsException ioobe) {
// either it's a boundary, or read past EOF, fall back:
setPos(pos, bi);
@ -235,7 +229,7 @@ abstract class ByteBufferIndexInput extends IndexInput implements RandomAccessIn
public long readLong(long pos) throws IOException {
final int bi = (int) (pos >> chunkSizePower);
try {
return buffers[bi].getLong((int) (pos & chunkSizeMask));
return guard.getLong(buffers[bi], (int) (pos & chunkSizeMask));
} catch (IndexOutOfBoundsException ioobe) {
// either it's a boundary, or read past EOF, fall back:
setPos(pos, bi);
@ -285,11 +279,6 @@ abstract class ByteBufferIndexInput extends IndexInput implements RandomAccessIn
final ByteBufferIndexInput clone = newCloneInstance(getFullSliceDescription(sliceDescription), newBuffers, ofs, length);
clone.isClone = true;
// register the new clone in our clone list to clean it up on closing:
if (clones != null) {
this.clones.put(clone, Boolean.TRUE);
}
return clone;
}
@ -299,9 +288,9 @@ abstract class ByteBufferIndexInput extends IndexInput implements RandomAccessIn
protected ByteBufferIndexInput newCloneInstance(String newResourceDescription, ByteBuffer[] newBuffers, int offset, long length) {
if (newBuffers.length == 1) {
newBuffers[0].position(offset);
return new SingleBufferImpl(newResourceDescription, newBuffers[0].slice(), length, chunkSizePower, this.cleaner, this.clones);
return new SingleBufferImpl(newResourceDescription, newBuffers[0].slice(), length, chunkSizePower, this.guard);
} else {
return new MultiBufferImpl(newResourceDescription, newBuffers, offset, length, chunkSizePower, cleaner, clones);
return new MultiBufferImpl(newResourceDescription, newBuffers, offset, length, chunkSizePower, guard);
}
}
@ -335,25 +324,11 @@ abstract class ByteBufferIndexInput extends IndexInput implements RandomAccessIn
// make local copy, then un-set early
final ByteBuffer[] bufs = buffers;
unsetBuffers();
if (clones != null) {
clones.remove(this);
}
if (isClone) return;
// for extra safety unset also all clones' buffers:
if (clones != null) {
for (Iterator<ByteBufferIndexInput> it = this.clones.keyIterator(); it.hasNext();) {
final ByteBufferIndexInput clone = it.next();
assert clone.isClone;
clone.unsetBuffers();
}
this.clones.clear();
}
for (final ByteBuffer b : bufs) {
freeBuffer(b);
}
// tell the guard to invalidate and later unmap the bytebuffers (if supported):
guard.invalidateAndUnmap(bufs);
} finally {
unsetBuffers();
}
@ -367,31 +342,12 @@ abstract class ByteBufferIndexInput extends IndexInput implements RandomAccessIn
curBuf = null;
curBufIndex = 0;
}
/**
* Called when the contents of a buffer will be no longer needed.
*/
private void freeBuffer(ByteBuffer b) throws IOException {
if (cleaner != null) {
cleaner.freeBuffer(this, b);
}
}
/**
* Pass in an implementation of this interface to cleanup ByteBuffers.
* MMapDirectory implements this to allow unmapping of bytebuffers with private Java APIs.
*/
@FunctionalInterface
static interface BufferCleaner {
void freeBuffer(ByteBufferIndexInput parent, ByteBuffer b) throws IOException;
}
/** Optimization of ByteBufferIndexInput for when there is only one buffer */
static final class SingleBufferImpl extends ByteBufferIndexInput {
SingleBufferImpl(String resourceDescription, ByteBuffer buffer, long length, int chunkSizePower,
BufferCleaner cleaner, WeakIdentityMap<ByteBufferIndexInput,Boolean> clones) {
super(resourceDescription, new ByteBuffer[] { buffer }, length, chunkSizePower, cleaner, clones);
SingleBufferImpl(String resourceDescription, ByteBuffer buffer, long length, int chunkSizePower, ByteBufferGuard guard) {
super(resourceDescription, new ByteBuffer[] { buffer }, length, chunkSizePower, guard);
this.curBufIndex = 0;
this.curBuf = buffer;
buffer.position(0);
@ -426,7 +382,7 @@ abstract class ByteBufferIndexInput extends IndexInput implements RandomAccessIn
@Override
public byte readByte(long pos) throws IOException {
try {
return curBuf.get((int) pos);
return guard.getByte(curBuf, (int) pos);
} catch (IllegalArgumentException e) {
if (pos < 0) {
throw new IllegalArgumentException("Seeking to negative position: " + this, e);
@ -441,7 +397,7 @@ abstract class ByteBufferIndexInput extends IndexInput implements RandomAccessIn
@Override
public short readShort(long pos) throws IOException {
try {
return curBuf.getShort((int) pos);
return guard.getShort(curBuf, (int) pos);
} catch (IllegalArgumentException e) {
if (pos < 0) {
throw new IllegalArgumentException("Seeking to negative position: " + this, e);
@ -456,7 +412,7 @@ abstract class ByteBufferIndexInput extends IndexInput implements RandomAccessIn
@Override
public int readInt(long pos) throws IOException {
try {
return curBuf.getInt((int) pos);
return guard.getInt(curBuf, (int) pos);
} catch (IllegalArgumentException e) {
if (pos < 0) {
throw new IllegalArgumentException("Seeking to negative position: " + this, e);
@ -471,7 +427,7 @@ abstract class ByteBufferIndexInput extends IndexInput implements RandomAccessIn
@Override
public long readLong(long pos) throws IOException {
try {
return curBuf.getLong((int) pos);
return guard.getLong(curBuf, (int) pos);
} catch (IllegalArgumentException e) {
if (pos < 0) {
throw new IllegalArgumentException("Seeking to negative position: " + this, e);
@ -489,8 +445,8 @@ abstract class ByteBufferIndexInput extends IndexInput implements RandomAccessIn
private final int offset;
MultiBufferImpl(String resourceDescription, ByteBuffer[] buffers, int offset, long length, int chunkSizePower,
BufferCleaner cleaner, WeakIdentityMap<ByteBufferIndexInput,Boolean> clones) {
super(resourceDescription, buffers, length, chunkSizePower, cleaner, clones);
ByteBufferGuard guard) {
super(resourceDescription, buffers, length, chunkSizePower, guard);
this.offset = offset;
try {
seek(0L);

View File

@ -36,7 +36,7 @@ import java.util.concurrent.Future;
import java.lang.invoke.MethodHandle;
import java.lang.reflect.Method;
import org.apache.lucene.store.ByteBufferIndexInput.BufferCleaner;
import org.apache.lucene.store.ByteBufferGuard.BufferCleaner;
import org.apache.lucene.util.Constants;
import org.apache.lucene.util.SuppressForbidden;
@ -240,7 +240,7 @@ public class MMapDirectory extends FSDirectory {
final boolean useUnmap = getUseUnmap();
return ByteBufferIndexInput.newInstance(resourceDescription,
map(resourceDescription, c, 0, c.size()),
c.size(), chunkSizePower, useUnmap ? CLEANER : null, useUnmap);
c.size(), chunkSizePower, new ByteBufferGuard(resourceDescription, useUnmap ? CLEANER : null));
}
}
@ -370,7 +370,7 @@ public class MMapDirectory extends FSDirectory {
final MethodHandle unmapper = filterReturnValue(directBufferCleanerMethod, guardWithTest(nonNullTest, cleanMethod, noop))
.asType(methodType(void.class, ByteBuffer.class));
return (BufferCleaner) (ByteBufferIndexInput parent, ByteBuffer buffer) -> {
return (BufferCleaner) (String resourceDescription, ByteBuffer buffer) -> {
if (directBufferClass.isInstance(buffer)) {
final Throwable error = AccessController.doPrivileged((PrivilegedAction<Throwable>) () -> {
try {
@ -381,7 +381,7 @@ public class MMapDirectory extends FSDirectory {
}
});
if (error != null) {
throw new IOException("Unable to unmap the mapped buffer: " + parent.toString(), error);
throw new IOException("Unable to unmap the mapped buffer: " + resourceDescription, error);
}
}
};

View File

@ -58,7 +58,7 @@ public class TestGrowableByteArrayDataOutput extends LuceneTestCase {
public void testWriteLargeStrings() throws Exception {
int minSizeForDoublePass = GrowableByteArrayDataOutput.MIN_UTF8_SIZE_TO_ENABLE_DOUBLE_PASS_ENCODING;
int num = atLeast(1000);
int num = atLeast(100);
for (int i = 0; i < num; i++) {
String unicode = TestUtil.randomRealisticUnicodeString(random(), minSizeForDoublePass, 10 * minSizeForDoublePass);
byte[] utf8 = new byte[unicode.length() * UnicodeUtil.MAX_UTF8_BYTES_PER_CHAR];

View File

@ -30,7 +30,7 @@ public class TestGeoUtils extends LuceneTestCase {
// We rely heavily on GeoUtils.circleToBBox so we test it here:
public void testRandomCircleToBBox() throws Exception {
int iters = atLeast(1000);
int iters = atLeast(100);
for(int iter=0;iter<iters;iter++) {
double centerLat = GeoTestUtil.nextLatitude();
@ -89,7 +89,8 @@ public class TestGeoUtils extends LuceneTestCase {
// similar to testRandomCircleToBBox, but different, less evil, maybe simpler
public void testBoundingBoxOpto() {
for (int i = 0; i < 1000; i++) {
int iters = atLeast(100);
for (int i = 0; i < iters; i++) {
double lat = GeoTestUtil.nextLatitude();
double lon = GeoTestUtil.nextLongitude();
double radius = 50000000 * random().nextDouble();
@ -119,7 +120,8 @@ public class TestGeoUtils extends LuceneTestCase {
// test we can use haversinSortKey() for distance queries.
public void testHaversinOpto() {
for (int i = 0; i < 1000; i++) {
int iters = atLeast(100);
for (int i = 0; i < iters; i++) {
double lat = GeoTestUtil.nextLatitude();
double lon = GeoTestUtil.nextLongitude();
double radius = 50000000 * random().nextDouble();
@ -193,7 +195,8 @@ public class TestGeoUtils extends LuceneTestCase {
// TODO: does not really belong here, but we test it like this for now
// we can make a fake IndexReader to send boxes directly to Point visitors instead?
public void testCircleOpto() throws Exception {
for (int i = 0; i < 50; i++) {
int iters = atLeast(20);
for (int i = 0; i < iters; i++) {
// circle
final double centerLat = -90 + 180.0 * random().nextDouble();
final double centerLon = -180 + 360.0 * random().nextDouble();

View File

@ -103,7 +103,8 @@ public class TestPolygon2D extends LuceneTestCase {
/** If polygon.contains(box) returns true, then any point in that box should return true as well */
public void testContainsRandom() throws Exception {
for (int i = 0; i < 1000; i++) {
int iters = atLeast(50);
for (int i = 0; i < iters; i++) {
Polygon polygon = nextPolygon();
Polygon2D impl = Polygon2D.create(polygon);
@ -175,7 +176,8 @@ public class TestPolygon2D extends LuceneTestCase {
/** If polygon.intersects(box) returns false, then any point in that box should return false as well */
public void testIntersectRandom() {
for (int i = 0; i < 100; i++) {
int iters = atLeast(10);
for (int i = 0; i < iters; i++) {
Polygon polygon = nextPolygon();
Polygon2D impl = Polygon2D.create(polygon);
@ -268,7 +270,8 @@ public class TestPolygon2D extends LuceneTestCase {
/** Tests current impl against original algorithm */
public void testContainsAgainstOriginal() {
for (int i = 0; i < 1000; i++) {
int iters = atLeast(100);
for (int i = 0; i < iters; i++) {
Polygon polygon = nextPolygon();
// currently we don't generate these, but this test does not want holes.
while (polygon.getHoles().length > 0) {

View File

@ -1217,7 +1217,7 @@ public class TestIndexSorting extends LuceneTestCase {
if (TEST_NIGHTLY) {
numDocs = atLeast(100000);
} else {
numDocs = atLeast(10000);
numDocs = atLeast(1000);
}
List<RandomDoc> docs = new ArrayList<>();

View File

@ -72,6 +72,7 @@ public class TestIndexingSequenceNumbers extends LuceneTestCase {
dir.close();
}
@Slow
public void testStressUpdateSameID() throws Exception {
int iters = atLeast(100);
for(int iter=0;iter<iters;iter++) {
@ -144,6 +145,7 @@ public class TestIndexingSequenceNumbers extends LuceneTestCase {
long seqNo;
}
@Slow
public void testStressConcurrentCommit() throws Exception {
final int opCount = atLeast(10000);
final int idCount = TestUtil.nextInt(random(), 10, 1000);
@ -303,6 +305,7 @@ public class TestIndexingSequenceNumbers extends LuceneTestCase {
dir.close();
}
@Slow
public void testStressConcurrentDocValuesUpdatesCommit() throws Exception {
final int opCount = atLeast(10000);
final int idCount = TestUtil.nextInt(random(), 10, 1000);
@ -459,6 +462,7 @@ public class TestIndexingSequenceNumbers extends LuceneTestCase {
dir.close();
}
@Slow
public void testStressConcurrentAddAndDeleteAndCommit() throws Exception {
final int opCount = atLeast(10000);
final int idCount = TestUtil.nextInt(random(), 10, 1000);

View File

@ -25,6 +25,8 @@ import java.util.Random;
import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
import org.apache.lucene.document.FieldType;
import org.apache.lucene.document.TextField;
import org.apache.lucene.index.DirectoryReader;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexWriter;
@ -67,7 +69,7 @@ public class TestBoolean2 extends LuceneTestCase {
@BeforeClass
public static void beforeClass() throws Exception {
// in some runs, test immediate adjacency of matches - in others, force a full bucket gap betwen docs
// in some runs, test immediate adjacency of matches - in others, force a full bucket gap between docs
NUM_FILLER_DOCS = random().nextBoolean() ? 0 : BooleanScorer.SIZE;
PRE_FILLER_DOCS = TestUtil.nextInt(random(), 0, (NUM_FILLER_DOCS / 2));
@ -78,13 +80,16 @@ public class TestBoolean2 extends LuceneTestCase {
}
RandomIndexWriter writer= new RandomIndexWriter(random(), directory, newIndexWriterConfig(new MockAnalyzer(random())).setMergePolicy(newLogMergePolicy()));
// we'll make a ton of docs, disable store/norms/vectors
FieldType ft = new FieldType(TextField.TYPE_NOT_STORED);
ft.setOmitNorms(true);
Document doc = new Document();
for (int filler = 0; filler < PRE_FILLER_DOCS; filler++) {
writer.addDocument(doc);
}
for (int i = 0; i < docFields.length; i++) {
doc.add(newTextField(field, docFields[i], Field.Store.NO));
doc.add(new Field(field, docFields[i], ft));
writer.addDocument(doc);
doc = new Document();
@ -149,12 +154,12 @@ public class TestBoolean2 extends LuceneTestCase {
newIndexWriterConfig(new MockAnalyzer(random()))
.setMaxBufferedDocs(TestUtil.nextInt(random(), 50, 1000)));
doc = new Document();
doc.add(newTextField("field2", "xxx", Field.Store.NO));
doc.add(new Field("field2", "xxx", ft));
for(int i=0;i<NUM_EXTRA_DOCS/2;i++) {
w.addDocument(doc);
}
doc = new Document();
doc.add(newTextField("field2", "big bad bug", Field.Store.NO));
doc.add(new Field("field2", "big bad bug", ft));
for(int i=0;i<NUM_EXTRA_DOCS/2;i++) {
w.addDocument(doc);
}

View File

@ -23,6 +23,7 @@ import org.apache.lucene.document.Field;
import org.apache.lucene.index.Term;
import org.apache.lucene.index.RandomIndexWriter;
import org.apache.lucene.util.TestUtil;
import org.apache.lucene.util.LuceneTestCase.Slow;
import org.junit.BeforeClass;
import org.junit.Assume;
@ -34,6 +35,7 @@ import org.junit.Assume;
* all use terms from same set of source data as our regular docs (to emphasis the DocFreq factor in scoring),
* in which case the queries will be wrapped so they can be excluded.
*/
@Slow // can this be sped up to be non-slow? filler docs make it quite a bit slower and many test methods...
public class TestSimpleExplanationsWithFillerDocs extends TestSimpleExplanations {
/** num of empty docs injected between every doc in the index */

View File

@ -19,6 +19,10 @@ package org.apache.lucene.store;
import java.io.IOException;
import java.nio.file.Path;
import java.util.Random;
import java.util.concurrent.CountDownLatch;
import org.junit.Ignore;
/**
* Tests MMapDirectory
@ -39,4 +43,38 @@ public class TestMmapDirectory extends BaseDirectoryTestCase {
MMapDirectory.UNMAP_SUPPORTED);
}
@Ignore("This test is for JVM testing purposes. There are no guarantees that it may not fail with SIGSEGV!")
public void testAceWithThreads() throws Exception {
for (int iter = 0; iter < 10; iter++) {
Directory dir = getDirectory(createTempDir("testAceWithThreads"));
IndexOutput out = dir.createOutput("test", IOContext.DEFAULT);
Random random = random();
for (int i = 0; i < 8 * 1024 * 1024; i++) {
out.writeInt(random.nextInt());
}
out.close();
IndexInput in = dir.openInput("test", IOContext.DEFAULT);
IndexInput clone = in.clone();
final byte accum[] = new byte[32 * 1024 * 1024];
final CountDownLatch shotgun = new CountDownLatch(1);
Thread t1 = new Thread(() -> {
try {
shotgun.await();
for (int i = 0; i < 10; i++) {
clone.seek(0);
clone.readBytes(accum, 0, accum.length);
}
} catch (IOException | AlreadyClosedException ok) {
// OK
} catch (InterruptedException e) {
throw new RuntimeException(e);
}
});
t1.start();
shotgun.countDown();
in.close();
t1.join();
dir.close();
}
}
}

View File

@ -265,6 +265,7 @@ public class TestMoreLikeThis extends LuceneTestCase {
return writer.numDocs() - 1;
}
@AwaitsFix(bugUrl = "https://issues.apache.org/jira/browse/LUCENE-7161")
public void testMultiFieldShouldReturnPerFieldBooleanQuery() throws Exception {
IndexReader reader = null;
Directory dir = newDirectory();

View File

@ -0,0 +1,262 @@
/*
* 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.lucene.document;
import org.apache.lucene.document.RangeFieldQuery.QueryType;
import org.apache.lucene.search.Query;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.NumericUtils;
/**
* An indexed Float Range field.
* <p>
* This field indexes dimensional ranges defined as min/max pairs. It supports
* up to a maximum of 4 dimensions (indexed as 8 numeric values). With 1 dimension representing a single float range,
* 2 dimensions representing a bounding box, 3 dimensions a bounding cube, and 4 dimensions a tesseract.
* <p>
* Multiple values for the same field in one document is supported, and open ended ranges can be defined using
* {@code Float.NEGATIVE_INFINITY} and {@code Float.POSITIVE_INFINITY}.
*
* <p>
* This field defines the following static factory methods for common search operations over float ranges:
* <ul>
* <li>{@link #newIntersectsQuery newIntersectsQuery()} matches ranges that intersect the defined search range.
* <li>{@link #newWithinQuery newWithinQuery()} matches ranges that are within the defined search range.
* <li>{@link #newContainsQuery newContainsQuery()} matches ranges that contain the defined search range.
* </ul>
*/
public class FloatRangeField extends Field {
/** stores float values so number of bytes is 4 */
public static final int BYTES = Float.BYTES;
/**
* Create a new FloatRangeField type, from min/max parallel arrays
*
* @param name field name. must not be null.
* @param min range min values; each entry is the min value for the dimension
* @param max range max values; each entry is the max value for the dimension
*/
public FloatRangeField(String name, final float[] min, final float[] max) {
super(name, getType(min.length));
setRangeValues(min, max);
}
/** set the field type */
private static FieldType getType(int dimensions) {
if (dimensions > 4) {
throw new IllegalArgumentException("FloatRangeField does not support greater than 4 dimensions");
}
FieldType ft = new FieldType();
// dimensions is set as 2*dimension size (min/max per dimension)
ft.setDimensions(dimensions*2, BYTES);
ft.freeze();
return ft;
}
/**
* Changes the values of the field.
* @param min array of min values. (accepts {@code Float.NEGATIVE_INFINITY})
* @param max array of max values. (accepts {@code Float.POSITIVE_INFINITY})
* @throws IllegalArgumentException if {@code min} or {@code max} is invalid
*/
public void setRangeValues(float[] min, float[] max) {
checkArgs(min, max);
if (min.length*2 != type.pointDimensionCount() || max.length*2 != type.pointDimensionCount()) {
throw new IllegalArgumentException("field (name=" + name + ") uses " + type.pointDimensionCount()/2
+ " dimensions; cannot change to (incoming) " + min.length + " dimensions");
}
final byte[] bytes;
if (fieldsData == null) {
bytes = new byte[BYTES*2*min.length];
fieldsData = new BytesRef(bytes);
} else {
bytes = ((BytesRef)fieldsData).bytes;
}
verifyAndEncode(min, max, bytes);
}
/** validate the arguments */
private static void checkArgs(final float[] min, final float[] max) {
if (min == null || max == null || min.length == 0 || max.length == 0) {
throw new IllegalArgumentException("min/max range values cannot be null or empty");
}
if (min.length != max.length) {
throw new IllegalArgumentException("min/max ranges must agree");
}
if (min.length > 4) {
throw new IllegalArgumentException("FloatRangeField does not support greater than 4 dimensions");
}
}
/**
* Encodes the min, max ranges into a byte array
*/
private static byte[] encode(float[] min, float[] max) {
checkArgs(min, max);
byte[] b = new byte[BYTES*2*min.length];
verifyAndEncode(min, max, b);
return b;
}
/**
* encode the ranges into a sortable byte array ({@code Float.NaN} not allowed)
* <p>
* example for 4 dimensions (8 bytes per dimension value):
* minD1 ... minD4 | maxD1 ... maxD4
*/
static void verifyAndEncode(float[] min, float[] max, byte[] bytes) {
for (int d=0,i=0,j=min.length*BYTES; d<min.length; ++d, i+=BYTES, j+=BYTES) {
if (Double.isNaN(min[d])) {
throw new IllegalArgumentException("invalid min value (" + Float.NaN + ")" + " in FloatRangeField");
}
if (Double.isNaN(max[d])) {
throw new IllegalArgumentException("invalid max value (" + Float.NaN + ")" + " in FloatRangeField");
}
if (min[d] > max[d]) {
throw new IllegalArgumentException("min value (" + min[d] + ") is greater than max value (" + max[d] + ")");
}
encode(min[d], bytes, i);
encode(max[d], bytes, j);
}
}
/** encode the given value into the byte array at the defined offset */
private static void encode(float val, byte[] bytes, int offset) {
NumericUtils.intToSortableBytes(NumericUtils.floatToSortableInt(val), bytes, offset);
}
/**
* Get the min value for the given dimension
* @param dimension the dimension, always positive
* @return the decoded min value
*/
public float getMin(int dimension) {
if (dimension < 0 || dimension >= type.pointDimensionCount()/2) {
throw new IllegalArgumentException("dimension request (" + dimension +
") out of bounds for field (name=" + name + " dimensions=" + type.pointDimensionCount()/2 + "). ");
}
return decodeMin(((BytesRef)fieldsData).bytes, dimension);
}
/**
* Get the max value for the given dimension
* @param dimension the dimension, always positive
* @return the decoded max value
*/
public float getMax(int dimension) {
if (dimension < 0 || dimension >= type.pointDimensionCount()/2) {
throw new IllegalArgumentException("dimension request (" + dimension +
") out of bounds for field (name=" + name + " dimensions=" + type.pointDimensionCount()/2 + "). ");
}
return decodeMax(((BytesRef)fieldsData).bytes, dimension);
}
/** decodes the min value (for the defined dimension) from the encoded input byte array */
static float decodeMin(byte[] b, int dimension) {
int offset = dimension*BYTES;
return NumericUtils.sortableIntToFloat(NumericUtils.sortableBytesToInt(b, offset));
}
/** decodes the max value (for the defined dimension) from the encoded input byte array */
static float decodeMax(byte[] b, int dimension) {
int offset = b.length/2 + dimension*BYTES;
return NumericUtils.sortableIntToFloat(NumericUtils.sortableBytesToInt(b, offset));
}
/**
* Create a query for matching indexed ranges that intersect the defined range.
* @param field field name. must not be null.
* @param min array of min values. (accepts {@code Float.NEGATIVE_INFINITY})
* @param max array of max values. (accepts {@code Float.MAX_VALUE})
* @return query for matching intersecting ranges (overlap, within, or contains)
* @throws IllegalArgumentException if {@code field} is null, {@code min} or {@code max} is invalid
*/
public static Query newIntersectsQuery(String field, final float[] min, final float[] max) {
return new RangeFieldQuery(field, encode(min, max), min.length, QueryType.INTERSECTS) {
@Override
protected String toString(byte[] ranges, int dimension) {
return FloatRangeField.toString(ranges, dimension);
}
};
}
/**
* Create a query for matching indexed float ranges that contain the defined range.
* @param field field name. must not be null.
* @param min array of min values. (accepts {@code Float.NEGATIVE_INFINITY})
* @param max array of max values. (accepts {@code Float.POSITIVE_INFINITY})
* @return query for matching ranges that contain the defined range
* @throws IllegalArgumentException if {@code field} is null, {@code min} or {@code max} is invalid
*/
public static Query newContainsQuery(String field, final float[] min, final float[] max) {
return new RangeFieldQuery(field, encode(min, max), min.length, QueryType.CONTAINS) {
@Override
protected String toString(byte[] ranges, int dimension) {
return FloatRangeField.toString(ranges, dimension);
}
};
}
/**
* Create a query for matching indexed ranges that are within the defined range.
* @param field field name. must not be null.
* @param min array of min values. (accepts {@code Float.NEGATIVE_INFINITY})
* @param max array of max values. (accepts {@code Float.POSITIVE_INFINITY})
* @return query for matching ranges within the defined range
* @throws IllegalArgumentException if {@code field} is null, {@code min} or {@code max} is invalid
*/
public static Query newWithinQuery(String field, final float[] min, final float[] max) {
checkArgs(min, max);
return new RangeFieldQuery(field, encode(min, max), min.length, QueryType.WITHIN) {
@Override
protected String toString(byte[] ranges, int dimension) {
return FloatRangeField.toString(ranges, dimension);
}
};
}
@Override
public String toString() {
StringBuilder sb = new StringBuilder();
sb.append(getClass().getSimpleName());
sb.append(" <");
sb.append(name);
sb.append(':');
byte[] b = ((BytesRef)fieldsData).bytes;
toString(b, 0);
for (int d=1; d<type.pointDimensionCount(); ++d) {
sb.append(' ');
toString(b, d);
}
sb.append('>');
return sb.toString();
}
/**
* Returns the String representation for the range at the given dimension
* @param ranges the encoded ranges, never null
* @param dimension the dimension of interest
* @return The string representation for the range at the provided dimension
*/
private static String toString(byte[] ranges, int dimension) {
return "[" + Float.toString(decodeMin(ranges, dimension)) + " : "
+ Float.toString(decodeMax(ranges, dimension)) + "]";
}
}

View File

@ -0,0 +1,262 @@
/*
* 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.lucene.document;
import org.apache.lucene.document.RangeFieldQuery.QueryType;
import org.apache.lucene.search.Query;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.NumericUtils;
/**
* An indexed Integer Range field.
* <p>
* This field indexes dimensional ranges defined as min/max pairs. It supports
* up to a maximum of 4 dimensions (indexed as 8 numeric values). With 1 dimension representing a single integer range,
* 2 dimensions representing a bounding box, 3 dimensions a bounding cube, and 4 dimensions a tesseract.
* <p>
* Multiple values for the same field in one document is supported, and open ended ranges can be defined using
* {@code Integer.MIN_VALUE} and {@code Integer.MAX_VALUE}.
*
* <p>
* This field defines the following static factory methods for common search operations over integer ranges:
* <ul>
* <li>{@link #newIntersectsQuery newIntersectsQuery()} matches ranges that intersect the defined search range.
* <li>{@link #newWithinQuery newWithinQuery()} matches ranges that are within the defined search range.
* <li>{@link #newContainsQuery newContainsQuery()} matches ranges that contain the defined search range.
* </ul>
*/
public class IntRangeField extends Field {
/** stores integer values so number of bytes is 4 */
public static final int BYTES = Integer.BYTES;
/**
* Create a new IntRangeField type, from min/max parallel arrays
*
* @param name field name. must not be null.
* @param min range min values; each entry is the min value for the dimension
* @param max range max values; each entry is the max value for the dimension
*/
public IntRangeField(String name, final int[] min, final int[] max) {
super(name, getType(min.length));
setRangeValues(min, max);
}
/** set the field type */
private static FieldType getType(int dimensions) {
if (dimensions > 4) {
throw new IllegalArgumentException("IntRangeField does not support greater than 4 dimensions");
}
FieldType ft = new FieldType();
// dimensions is set as 2*dimension size (min/max per dimension)
ft.setDimensions(dimensions*2, BYTES);
ft.freeze();
return ft;
}
/**
* Changes the values of the field.
* @param min array of min values. (accepts {@code Integer.NEGATIVE_INFINITY})
* @param max array of max values. (accepts {@code Integer.POSITIVE_INFINITY})
* @throws IllegalArgumentException if {@code min} or {@code max} is invalid
*/
public void setRangeValues(int[] min, int[] max) {
checkArgs(min, max);
if (min.length*2 != type.pointDimensionCount() || max.length*2 != type.pointDimensionCount()) {
throw new IllegalArgumentException("field (name=" + name + ") uses " + type.pointDimensionCount()/2
+ " dimensions; cannot change to (incoming) " + min.length + " dimensions");
}
final byte[] bytes;
if (fieldsData == null) {
bytes = new byte[BYTES*2*min.length];
fieldsData = new BytesRef(bytes);
} else {
bytes = ((BytesRef)fieldsData).bytes;
}
verifyAndEncode(min, max, bytes);
}
/** validate the arguments */
private static void checkArgs(final int[] min, final int[] max) {
if (min == null || max == null || min.length == 0 || max.length == 0) {
throw new IllegalArgumentException("min/max range values cannot be null or empty");
}
if (min.length != max.length) {
throw new IllegalArgumentException("min/max ranges must agree");
}
if (min.length > 4) {
throw new IllegalArgumentException("IntRangeField does not support greater than 4 dimensions");
}
}
/**
* Encodes the min, max ranges into a byte array
*/
private static byte[] encode(int[] min, int[] max) {
checkArgs(min, max);
byte[] b = new byte[BYTES*2*min.length];
verifyAndEncode(min, max, b);
return b;
}
/**
* encode the ranges into a sortable byte array ({@code Double.NaN} not allowed)
* <p>
* example for 4 dimensions (8 bytes per dimension value):
* minD1 ... minD4 | maxD1 ... maxD4
*/
static void verifyAndEncode(int[] min, int[] max, byte[] bytes) {
for (int d=0,i=0,j=min.length*BYTES; d<min.length; ++d, i+=BYTES, j+=BYTES) {
if (Double.isNaN(min[d])) {
throw new IllegalArgumentException("invalid min value (" + Double.NaN + ")" + " in IntRangeField");
}
if (Double.isNaN(max[d])) {
throw new IllegalArgumentException("invalid max value (" + Double.NaN + ")" + " in IntRangeField");
}
if (min[d] > max[d]) {
throw new IllegalArgumentException("min value (" + min[d] + ") is greater than max value (" + max[d] + ")");
}
encode(min[d], bytes, i);
encode(max[d], bytes, j);
}
}
/** encode the given value into the byte array at the defined offset */
private static void encode(int val, byte[] bytes, int offset) {
NumericUtils.intToSortableBytes(val, bytes, offset);
}
/**
* Get the min value for the given dimension
* @param dimension the dimension, always positive
* @return the decoded min value
*/
public int getMin(int dimension) {
if (dimension < 0 || dimension >= type.pointDimensionCount()/2) {
throw new IllegalArgumentException("dimension request (" + dimension +
") out of bounds for field (name=" + name + " dimensions=" + type.pointDimensionCount()/2 + "). ");
}
return decodeMin(((BytesRef)fieldsData).bytes, dimension);
}
/**
* Get the max value for the given dimension
* @param dimension the dimension, always positive
* @return the decoded max value
*/
public int getMax(int dimension) {
if (dimension < 0 || dimension >= type.pointDimensionCount()/2) {
throw new IllegalArgumentException("dimension request (" + dimension +
") out of bounds for field (name=" + name + " dimensions=" + type.pointDimensionCount()/2 + "). ");
}
return decodeMax(((BytesRef)fieldsData).bytes, dimension);
}
/** decodes the min value (for the defined dimension) from the encoded input byte array */
static int decodeMin(byte[] b, int dimension) {
int offset = dimension*BYTES;
return NumericUtils.sortableBytesToInt(b, offset);
}
/** decodes the max value (for the defined dimension) from the encoded input byte array */
static int decodeMax(byte[] b, int dimension) {
int offset = b.length/2 + dimension*BYTES;
return NumericUtils.sortableBytesToInt(b, offset);
}
/**
* Create a query for matching indexed ranges that intersect the defined range.
* @param field field name. must not be null.
* @param min array of min values. (accepts {@code Integer.MIN_VALUE})
* @param max array of max values. (accepts {@code Integer.MAX_VALUE})
* @return query for matching intersecting ranges (overlap, within, or contains)
* @throws IllegalArgumentException if {@code field} is null, {@code min} or {@code max} is invalid
*/
public static Query newIntersectsQuery(String field, final int[] min, final int[] max) {
return new RangeFieldQuery(field, encode(min, max), min.length, QueryType.INTERSECTS) {
@Override
protected String toString(byte[] ranges, int dimension) {
return IntRangeField.toString(ranges, dimension);
}
};
}
/**
* Create a query for matching indexed ranges that contain the defined range.
* @param field field name. must not be null.
* @param min array of min values. (accepts {@code Integer.MIN_VALUE})
* @param max array of max values. (accepts {@code Integer.MAX_VALUE})
* @return query for matching ranges that contain the defined range
* @throws IllegalArgumentException if {@code field} is null, {@code min} or {@code max} is invalid
*/
public static Query newContainsQuery(String field, final int[] min, final int[] max) {
return new RangeFieldQuery(field, encode(min, max), min.length, QueryType.CONTAINS) {
@Override
protected String toString(byte[] ranges, int dimension) {
return IntRangeField.toString(ranges, dimension);
}
};
}
/**
* Create a query for matching indexed ranges that are within the defined range.
* @param field field name. must not be null.
* @param min array of min values. (accepts {@code Integer.MIN_VALUE})
* @param max array of max values. (accepts {@code Integer.MAX_VALUE})
* @return query for matching ranges within the defined range
* @throws IllegalArgumentException if {@code field} is null, {@code min} or {@code max} is invalid
*/
public static Query newWithinQuery(String field, final int[] min, final int[] max) {
checkArgs(min, max);
return new RangeFieldQuery(field, encode(min, max), min.length, QueryType.WITHIN) {
@Override
protected String toString(byte[] ranges, int dimension) {
return IntRangeField.toString(ranges, dimension);
}
};
}
@Override
public String toString() {
StringBuilder sb = new StringBuilder();
sb.append(getClass().getSimpleName());
sb.append(" <");
sb.append(name);
sb.append(':');
byte[] b = ((BytesRef)fieldsData).bytes;
toString(b, 0);
for (int d=1; d<type.pointDimensionCount(); ++d) {
sb.append(' ');
toString(b, d);
}
sb.append('>');
return sb.toString();
}
/**
* Returns the String representation for the range at the given dimension
* @param ranges the encoded ranges, never null
* @param dimension the dimension of interest
* @return The string representation for the range at the provided dimension
*/
private static String toString(byte[] ranges, int dimension) {
return "[" + Integer.toString(decodeMin(ranges, dimension)) + " : "
+ Integer.toString(decodeMax(ranges, dimension)) + "]";
}
}

View File

@ -0,0 +1,260 @@
/*
* 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.lucene.document;
import org.apache.lucene.document.RangeFieldQuery.QueryType;
import org.apache.lucene.search.Query;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.NumericUtils;
/**
* An indexed Long Range field.
* <p>
* This field indexes dimensional ranges defined as min/max pairs. It supports
* up to a maximum of 4 dimensions (indexed as 8 numeric values). With 1 dimension representing a single long range,
* 2 dimensions representing a bounding box, 3 dimensions a bounding cube, and 4 dimensions a tesseract.
* <p>
* Multiple values for the same field in one document is supported, and open ended ranges can be defined using
* {@code Long.MIN_VALUE} and {@code Long.MAX_VALUE}.
*
* <p>
* This field defines the following static factory methods for common search operations over long ranges:
* <ul>
* <li>{@link #newIntersectsQuery newIntersectsQuery()} matches ranges that intersect the defined search range.
* <li>{@link #newWithinQuery newWithinQuery()} matches ranges that are within the defined search range.
* <li>{@link #newContainsQuery newContainsQuery()} matches ranges that contain the defined search range.
* </ul>
*/
public class LongRangeField extends Field {
/** stores long values so number of bytes is 8 */
public static final int BYTES = Long.BYTES;
/**
* Create a new LongRangeField type, from min/max parallel arrays
*
* @param name field name. must not be null.
* @param min range min values; each entry is the min value for the dimension
* @param max range max values; each entry is the max value for the dimension
*/
public LongRangeField(String name, final long[] min, final long[] max) {
super(name, getType(min.length));
setRangeValues(min, max);
}
/** set the field type */
private static FieldType getType(int dimensions) {
if (dimensions > 4) {
throw new IllegalArgumentException("LongRangeField does not support greater than 4 dimensions");
}
FieldType ft = new FieldType();
// dimensions is set as 2*dimension size (min/max per dimension)
ft.setDimensions(dimensions*2, BYTES);
ft.freeze();
return ft;
}
/**
* Changes the values of the field.
* @param min array of min values. (accepts {@code Long.MIN_VALUE})
* @param max array of max values. (accepts {@code Long.MAX_VALUE})
* @throws IllegalArgumentException if {@code min} or {@code max} is invalid
*/
public void setRangeValues(long[] min, long[] max) {
checkArgs(min, max);
if (min.length*2 != type.pointDimensionCount() || max.length*2 != type.pointDimensionCount()) {
throw new IllegalArgumentException("field (name=" + name + ") uses " + type.pointDimensionCount()/2
+ " dimensions; cannot change to (incoming) " + min.length + " dimensions");
}
final byte[] bytes;
if (fieldsData == null) {
bytes = new byte[BYTES*2*min.length];
fieldsData = new BytesRef(bytes);
} else {
bytes = ((BytesRef)fieldsData).bytes;
}
verifyAndEncode(min, max, bytes);
}
/** validate the arguments */
private static void checkArgs(final long[] min, final long[] max) {
if (min == null || max == null || min.length == 0 || max.length == 0) {
throw new IllegalArgumentException("min/max range values cannot be null or empty");
}
if (min.length != max.length) {
throw new IllegalArgumentException("min/max ranges must agree");
}
if (min.length > 4) {
throw new IllegalArgumentException("LongRangeField does not support greater than 4 dimensions");
}
}
/** Encodes the min, max ranges into a byte array */
private static byte[] encode(long[] min, long[] max) {
checkArgs(min, max);
byte[] b = new byte[BYTES*2*min.length];
verifyAndEncode(min, max, b);
return b;
}
/**
* encode the ranges into a sortable byte array ({@code Double.NaN} not allowed)
* <p>
* example for 4 dimensions (8 bytes per dimension value):
* minD1 ... minD4 | maxD1 ... maxD4
*/
static void verifyAndEncode(long[] min, long[] max, byte[] bytes) {
for (int d=0,i=0,j=min.length*BYTES; d<min.length; ++d, i+=BYTES, j+=BYTES) {
if (Double.isNaN(min[d])) {
throw new IllegalArgumentException("invalid min value (" + Double.NaN + ")" + " in IntRangeField");
}
if (Double.isNaN(max[d])) {
throw new IllegalArgumentException("invalid max value (" + Double.NaN + ")" + " in IntRangeField");
}
if (min[d] > max[d]) {
throw new IllegalArgumentException("min value (" + min[d] + ") is greater than max value (" + max[d] + ")");
}
encode(min[d], bytes, i);
encode(max[d], bytes, j);
}
}
/** encode the given value into the byte array at the defined offset */
private static void encode(long val, byte[] bytes, int offset) {
NumericUtils.longToSortableBytes(val, bytes, offset);
}
/**
* Get the min value for the given dimension
* @param dimension the dimension, always positive
* @return the decoded min value
*/
public long getMin(int dimension) {
if (dimension < 0 || dimension >= type.pointDimensionCount()/2) {
throw new IllegalArgumentException("dimension request (" + dimension +
") out of bounds for field (name=" + name + " dimensions=" + type.pointDimensionCount()/2 + "). ");
}
return decodeMin(((BytesRef)fieldsData).bytes, dimension);
}
/**
* Get the max value for the given dimension
* @param dimension the dimension, always positive
* @return the decoded max value
*/
public long getMax(int dimension) {
if (dimension < 0 || dimension >= type.pointDimensionCount()/2) {
throw new IllegalArgumentException("dimension request (" + dimension +
") out of bounds for field (name=" + name + " dimensions=" + type.pointDimensionCount()/2 + "). ");
}
return decodeMax(((BytesRef)fieldsData).bytes, dimension);
}
/** decodes the min value (for the defined dimension) from the encoded input byte array */
static long decodeMin(byte[] b, int dimension) {
int offset = dimension*BYTES;
return NumericUtils.sortableBytesToLong(b, offset);
}
/** decodes the max value (for the defined dimension) from the encoded input byte array */
static long decodeMax(byte[] b, int dimension) {
int offset = b.length/2 + dimension*BYTES;
return NumericUtils.sortableBytesToLong(b, offset);
}
/**
* Create a query for matching indexed ranges that intersect the defined range.
* @param field field name. must not be null.
* @param min array of min values. (accepts {@code Long.MIN_VALUE})
* @param max array of max values. (accepts {@code Long.MAX_VALUE})
* @return query for matching intersecting ranges (overlap, within, or contains)
* @throws IllegalArgumentException if {@code field} is null, {@code min} or {@code max} is invalid
*/
public static Query newIntersectsQuery(String field, final long[] min, final long[] max) {
return new RangeFieldQuery(field, encode(min, max), min.length, QueryType.INTERSECTS) {
@Override
protected String toString(byte[] ranges, int dimension) {
return LongRangeField.toString(ranges, dimension);
}
};
}
/**
* Create a query for matching indexed ranges that contain the defined range.
* @param field field name. must not be null.
* @param min array of min values. (accepts {@code Long.MIN_VALUE})
* @param max array of max values. (accepts {@code Long.MAX_VALUE})
* @return query for matching ranges that contain the defined range
* @throws IllegalArgumentException if {@code field} is null, {@code min} or {@code max} is invalid
*/
public static Query newContainsQuery(String field, final long[] min, final long[] max) {
return new RangeFieldQuery(field, encode(min, max), min.length, QueryType.CONTAINS) {
@Override
protected String toString(byte[] ranges, int dimension) {
return LongRangeField.toString(ranges, dimension);
}
};
}
/**
* Create a query for matching indexed ranges that are within the defined range.
* @param field field name. must not be null.
* @param min array of min values. (accepts {@code Long.MIN_VALUE})
* @param max array of max values. (accepts {@code Long.MAX_VALUE})
* @return query for matching ranges within the defined range
* @throws IllegalArgumentException if {@code field} is null, {@code min} or {@code max} is invalid
*/
public static Query newWithinQuery(String field, final long[] min, final long[] max) {
checkArgs(min, max);
return new RangeFieldQuery(field, encode(min, max), min.length, QueryType.WITHIN) {
@Override
protected String toString(byte[] ranges, int dimension) {
return LongRangeField.toString(ranges, dimension);
}
};
}
@Override
public String toString() {
StringBuilder sb = new StringBuilder();
sb.append(getClass().getSimpleName());
sb.append(" <");
sb.append(name);
sb.append(':');
byte[] b = ((BytesRef)fieldsData).bytes;
toString(b, 0);
for (int d=1; d<type.pointDimensionCount(); ++d) {
sb.append(' ');
toString(b, d);
}
sb.append('>');
return sb.toString();
}
/**
* Returns the String representation for the range at the given dimension
* @param ranges the encoded ranges, never null
* @param dimension the dimension of interest
* @return The string representation for the range at the provided dimension
*/
private static String toString(byte[] ranges, int dimension) {
return "[" + Long.toString(decodeMin(ranges, dimension)) + " : "
+ Long.toString(decodeMax(ranges, dimension)) + "]";
}
}

View File

@ -17,7 +17,6 @@
package org.apache.lucene.search;
import java.io.IOException;
import java.util.Arrays;
import java.util.HashSet;
import java.util.Set;
@ -41,16 +40,18 @@ import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.LuceneTestCase;
/**
* Abstract class to do basic tests for a RangeField query.
* Abstract class to do basic tests for a RangeField query. Testing rigor inspired by {@code BaseGeoPointTestCase}
*/
public abstract class BaseRangeFieldQueryTestCase extends LuceneTestCase {
protected abstract Field newRangeField(double[] min, double[] max);
protected abstract Field newRangeField(Range box);
protected abstract Query newIntersectsQuery(double[] min, double[] max);
protected abstract Query newIntersectsQuery(Range box);
protected abstract Query newContainsQuery(double[] min, double[] max);
protected abstract Query newContainsQuery(Range box);
protected abstract Query newWithinQuery(double[] min, double[] max);
protected abstract Query newWithinQuery(Range box);
protected abstract Range nextRange(int dimensions);
protected int dimension() {
return random().nextInt(4) + 1;
@ -82,18 +83,18 @@ public abstract class BaseRangeFieldQueryTestCase extends LuceneTestCase {
System.out.println("TEST: numDocs=" + numDocs);
}
Box[][] boxes = new Box[numDocs][];
Range[][] ranges = new Range[numDocs][];
boolean haveRealDoc = true;
nextdoc: for (int id=0; id<numDocs; ++id) {
int x = random().nextInt(20);
if (boxes[id] == null) {
boxes[id] = new Box[] {nextBox(dimensions)};
if (ranges[id] == null) {
ranges[id] = new Range[] {nextRange(dimensions)};
}
if (x == 17) {
// dome docs don't have a box:
boxes[id][0].min[0] = Double.NaN;
ranges[id][0].isMissing = true;
if (VERBOSE) {
System.out.println(" id=" + id + " is missing");
}
@ -103,19 +104,19 @@ public abstract class BaseRangeFieldQueryTestCase extends LuceneTestCase {
if (multiValued == true && random().nextBoolean()) {
// randomly add multi valued documents (up to 2 fields)
int n = random().nextInt(2) + 1;
boxes[id] = new Box[n];
ranges[id] = new Range[n];
for (int i=0; i<n; ++i) {
boxes[id][i] = nextBox(dimensions);
ranges[id][i] = nextRange(dimensions);
}
}
if (id > 0 && x < 9 && haveRealDoc) {
int oldID;
int i=0;
// don't step on missing boxes:
// don't step on missing ranges:
while (true) {
oldID = random().nextInt(id);
if (Double.isNaN(boxes[oldID][0].min[0]) == false) {
if (ranges[oldID][0].isMissing == false) {
break;
} else if (++i > id) {
continue nextdoc;
@ -125,11 +126,11 @@ public abstract class BaseRangeFieldQueryTestCase extends LuceneTestCase {
if (x == dimensions*2) {
// Fully identical box (use first box in case current is multivalued but old is not)
for (int d=0; d<dimensions; ++d) {
boxes[id][0].min[d] = boxes[oldID][0].min[d];
boxes[id][0].max[d] = boxes[oldID][0].max[d];
ranges[id][0].setMin(d, ranges[oldID][0].getMin(d));
ranges[id][0].setMax(d, ranges[oldID][0].getMax(d));
}
if (VERBOSE) {
System.out.println(" id=" + id + " box=" + boxes[id] + " (same box as doc=" + oldID + ")");
System.out.println(" id=" + id + " box=" + ranges[id] + " (same box as doc=" + oldID + ")");
}
} else {
for (int m = 0, even = dimensions % 2; m < dimensions * 2; ++m) {
@ -137,14 +138,14 @@ public abstract class BaseRangeFieldQueryTestCase extends LuceneTestCase {
int d = (int)Math.floor(m/2);
// current could be multivalue but old may not be, so use first box
if (even == 0) {
boxes[id][0].setVal(d, boxes[oldID][0].min[d]);
ranges[id][0].setMin(d, ranges[oldID][0].getMin(d));
if (VERBOSE) {
System.out.println(" id=" + id + " box=" + boxes[id] + " (same min[" + d + "] as doc=" + oldID + ")");
System.out.println(" id=" + id + " box=" + ranges[id] + " (same min[" + d + "] as doc=" + oldID + ")");
}
} else {
boxes[id][0].setVal(d, boxes[oldID][0].max[d]);
ranges[id][0].setMax(d, ranges[oldID][0].getMax(d));
if (VERBOSE) {
System.out.println(" id=" + id + " box=" + boxes[id] + " (same max[" + d + "] as doc=" + oldID + ")");
System.out.println(" id=" + id + " box=" + ranges[id] + " (same max[" + d + "] as doc=" + oldID + ")");
}
}
}
@ -152,20 +153,20 @@ public abstract class BaseRangeFieldQueryTestCase extends LuceneTestCase {
}
}
}
verify(boxes);
verify(ranges);
}
private void verify(Box[][] boxes) throws Exception {
private void verify(Range[][] ranges) throws Exception {
IndexWriterConfig iwc = newIndexWriterConfig();
// Else seeds may not reproduce:
iwc.setMergeScheduler(new SerialMergeScheduler());
// Else we can get O(N^2) merging
int mbd = iwc.getMaxBufferedDocs();
if (mbd != -1 && mbd < boxes.length/100) {
iwc.setMaxBufferedDocs(boxes.length/100);
if (mbd != -1 && mbd < ranges.length/100) {
iwc.setMaxBufferedDocs(ranges.length/100);
}
Directory dir;
if (boxes.length > 50000) {
if (ranges.length > 50000) {
dir = newFSDirectory(createTempDir(getClass().getSimpleName()));
} else {
dir = newDirectory();
@ -173,13 +174,13 @@ public abstract class BaseRangeFieldQueryTestCase extends LuceneTestCase {
Set<Integer> deleted = new HashSet<>();
IndexWriter w = new IndexWriter(dir, iwc);
for (int id=0; id < boxes.length; ++id) {
for (int id=0; id < ranges.length; ++id) {
Document doc = new Document();
doc.add(newStringField("id", ""+id, Field.Store.NO));
doc.add(new NumericDocValuesField("id", id));
if (Double.isNaN(boxes[id][0].min[0]) == false) {
for (int n=0; n<boxes[id].length; ++n) {
doc.add(newRangeField(boxes[id][n].min, boxes[id][n].max));
if (ranges[id][0].isMissing == false) {
for (int n=0; n<ranges[id].length; ++n) {
doc.add(newRangeField(ranges[id][n]));
}
}
w.addDocument(doc);
@ -200,7 +201,7 @@ public abstract class BaseRangeFieldQueryTestCase extends LuceneTestCase {
w.close();
IndexSearcher s = newSearcher(r);
int dimensions = boxes[0][0].min.length;
int dimensions = ranges[0][0].numDimensions();
int iters = atLeast(25);
NumericDocValues docIDToID = MultiDocValues.getNumericValues(r, "id");
Bits liveDocs = MultiFields.getLiveDocs(s.getIndexReader());
@ -211,20 +212,20 @@ public abstract class BaseRangeFieldQueryTestCase extends LuceneTestCase {
System.out.println("\nTEST: iter=" + iter + " s=" + s);
}
// occasionally test open ended bounding boxes
Box queryBox = nextBox(dimensions);
// occasionally test open ended bounding ranges
Range queryRange = nextRange(dimensions);
int rv = random().nextInt(3);
Query query;
Box.QueryType queryType;
Range.QueryType queryType;
if (rv == 0) {
queryType = Box.QueryType.INTERSECTS;
query = newIntersectsQuery(queryBox.min, queryBox.max);
queryType = Range.QueryType.INTERSECTS;
query = newIntersectsQuery(queryRange);
} else if (rv == 1) {
queryType = Box.QueryType.CONTAINS;
query = newContainsQuery(queryBox.min, queryBox.max);
queryType = Range.QueryType.CONTAINS;
query = newContainsQuery(queryRange);
} else {
queryType = Box.QueryType.WITHIN;
query = newWithinQuery(queryBox.min, queryBox.max);
queryType = Range.QueryType.WITHIN;
query = newWithinQuery(queryRange);
}
if (VERBOSE) {
@ -255,25 +256,25 @@ public abstract class BaseRangeFieldQueryTestCase extends LuceneTestCase {
if (liveDocs != null && liveDocs.get(docID) == false) {
// document is deleted
expected = false;
} else if (Double.isNaN(boxes[id][0].min[0])) {
} else if (ranges[id][0].isMissing) {
expected = false;
} else {
expected = expectedResult(queryBox, boxes[id], queryType);
expected = expectedResult(queryRange, ranges[id], queryType);
}
if (hits.get(docID) != expected) {
StringBuilder b = new StringBuilder();
b.append("FAIL (iter " + iter + "): ");
if (expected == true) {
b.append("id=" + id + (boxes[id].length > 1 ? " (MultiValue) " : " ") + "should match but did not\n");
b.append("id=" + id + (ranges[id].length > 1 ? " (MultiValue) " : " ") + "should match but did not\n");
} else {
b.append("id=" + id + " should not match but did\n");
}
b.append(" queryBox=" + queryBox + "\n");
b.append(" box" + ((boxes[id].length > 1) ? "es=" : "=" ) + boxes[id][0]);
for (int n=1; n<boxes[id].length; ++n) {
b.append(" queryRange=" + queryRange + "\n");
b.append(" box" + ((ranges[id].length > 1) ? "es=" : "=" ) + ranges[id][0]);
for (int n=1; n<ranges[id].length; ++n) {
b.append(", ");
b.append(boxes[id][n]);
b.append(ranges[id][n]);
}
b.append("\n queryType=" + queryType + "\n");
b.append(" deleted?=" + (liveDocs != null && liveDocs.get(docID) == false));
@ -284,144 +285,51 @@ public abstract class BaseRangeFieldQueryTestCase extends LuceneTestCase {
IOUtils.close(r, dir);
}
protected boolean expectedResult(Box queryBox, Box[] box, Box.QueryType queryType) {
for (int i=0; i<box.length; ++i) {
if (expectedBBoxQueryResult(queryBox, box[i], queryType) == true) {
protected boolean expectedResult(Range queryRange, Range[] range, Range.QueryType queryType) {
for (int i=0; i<range.length; ++i) {
if (expectedBBoxQueryResult(queryRange, range[i], queryType) == true) {
return true;
}
}
return false;
}
protected boolean expectedBBoxQueryResult(Box queryBox, Box box, Box.QueryType queryType) {
if (box.equals(queryBox)) {
protected boolean expectedBBoxQueryResult(Range queryRange, Range range, Range.QueryType queryType) {
if (queryRange.isEqual(range)) {
return true;
}
Box.QueryType relation = box.relate(queryBox);
if (queryType == Box.QueryType.INTERSECTS) {
Range.QueryType relation = range.relate(queryRange);
if (queryType == Range.QueryType.INTERSECTS) {
return relation != null;
}
return relation == queryType;
}
protected double nextDoubleInternal() {
if (rarely()) {
return random().nextBoolean() ? Double.POSITIVE_INFINITY : Double.NEGATIVE_INFINITY;
}
double max = 100 / 2;
return (max + max) * random().nextDouble() - max;
}
protected Box nextBox(int dimensions) {
double[] min = new double[dimensions];
double[] max = new double[dimensions];
for (int d=0; d<dimensions; ++d) {
min[d] = nextDoubleInternal();
max[d] = nextDoubleInternal();
}
return new Box(min, max);
}
protected static class Box {
double[] min;
double[] max;
abstract static class Range {
protected boolean isMissing = false;
enum QueryType { INTERSECTS, WITHIN, CONTAINS }
Box(double[] min, double[] max) {
assert min != null && max != null && min.length > 0 && max.length > 0
: "test box: min/max cannot be null or empty";
assert min.length == max.length : "test box: min/max length do not agree";
this.min = new double[min.length];
this.max = new double[max.length];
for (int d=0; d<min.length; ++d) {
this.min[d] = Math.min(min[d], max[d]);
this.max[d] = Math.max(min[d], max[d]);
}
}
protected abstract int numDimensions();
protected abstract Object getMin(int dim);
protected abstract void setMin(int dim, Object val);
protected abstract Object getMax(int dim);
protected abstract void setMax(int dim, Object val);
protected abstract boolean isEqual(Range other);
protected abstract boolean isDisjoint(Range other);
protected abstract boolean isWithin(Range other);
protected abstract boolean contains(Range other);
protected void setVal(int dimension, double val) {
if (val <= min[dimension]) {
min[dimension] = val;
} else {
max[dimension] = val;
}
}
@Override
public boolean equals(Object o) {
return o != null
&& getClass() == o.getClass()
&& equalTo(getClass().cast(o));
}
private boolean equalTo(Box o) {
return Arrays.equals(min, o.min)
&& Arrays.equals(max, o.max);
}
@Override
public int hashCode() {
int result = Arrays.hashCode(min);
result = 31 * result + Arrays.hashCode(max);
return result;
}
QueryType relate(Box other) {
// check disjoint
for (int d=0; d<this.min.length; ++d) {
if (this.min[d] > other.max[d] || this.max[d] < other.min[d]) {
// disjoint:
return null;
}
}
// check within
boolean within = true;
for (int d=0; d<this.min.length; ++d) {
if ((this.min[d] >= other.min[d] && this.max[d] <= other.max[d]) == false) {
// not within:
within = false;
break;
}
}
if (within == true) {
protected QueryType relate(Range other) {
if (isDisjoint(other)) {
// if disjoint; return null:
return null;
} else if (isWithin(other)) {
return QueryType.WITHIN;
}
// check contains
boolean contains = true;
for (int d=0; d<this.min.length; ++d) {
if ((this.min[d] <= other.min[d] && this.max[d] >= other.max[d]) == false) {
// not contains:
contains = false;
break;
}
}
if (contains == true) {
} else if (contains(other)) {
return QueryType.CONTAINS;
}
return QueryType.INTERSECTS;
}
@Override
public String toString() {
StringBuilder b = new StringBuilder();
b.append("Box(");
b.append(min[0]);
b.append(" TO ");
b.append(max[0]);
for (int d=1; d<min.length; ++d) {
b.append(", ");
b.append(min[d]);
b.append(" TO ");
b.append(max[d]);
}
b.append(")");
return b.toString();
}
}
}

View File

@ -16,6 +16,8 @@
*/
package org.apache.lucene.search;
import java.util.Arrays;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.DoubleRangeField;
import org.apache.lucene.index.IndexReader;
@ -23,25 +25,50 @@ import org.apache.lucene.index.RandomIndexWriter;
import org.apache.lucene.store.Directory;
/**
* Random testing for RangeFieldQueries. Testing rigor inspired by {@code BaseGeoPointTestCase}
* Random testing for RangeFieldQueries.
*/
public class TestDoubleRangeFieldQueries extends BaseRangeFieldQueryTestCase {
private static final String FIELD_NAME = "rangeField";
private static final String FIELD_NAME = "doubleRangeField";
protected DoubleRangeField newRangeField(double[] min, double[] max) {
return new DoubleRangeField(FIELD_NAME, min, max);
private double nextDoubleInternal() {
if (rarely()) {
return random().nextBoolean() ? Double.POSITIVE_INFINITY : Double.NEGATIVE_INFINITY;
}
double max = Double.MAX_VALUE / 2;
return (max + max) * random().nextDouble() - max;
}
protected Query newIntersectsQuery(double[] min, double[] max) {
return DoubleRangeField.newIntersectsQuery(FIELD_NAME, min, max);
@Override
protected Range nextRange(int dimensions) {
double[] min = new double[dimensions];
double[] max = new double[dimensions];
for (int d=0; d<dimensions; ++d) {
min[d] = nextDoubleInternal();
max[d] = nextDoubleInternal();
}
return new DoubleRange(min, max);
}
protected Query newContainsQuery(double[] min, double[] max) {
return DoubleRangeField.newContainsQuery(FIELD_NAME, min, max);
@Override
protected DoubleRangeField newRangeField(Range r) {
return new DoubleRangeField(FIELD_NAME, ((DoubleRange)r).min, ((DoubleRange)r).max);
}
protected Query newWithinQuery(double[] min, double[] max) {
return DoubleRangeField.newWithinQuery(FIELD_NAME, min, max);
@Override
protected Query newIntersectsQuery(Range r) {
return DoubleRangeField.newIntersectsQuery(FIELD_NAME, ((DoubleRange)r).min, ((DoubleRange)r).max);
}
@Override
protected Query newContainsQuery(Range r) {
return DoubleRangeField.newContainsQuery(FIELD_NAME, ((DoubleRange)r).min, ((DoubleRange)r).max);
}
@Override
protected Query newWithinQuery(Range r) {
return DoubleRangeField.newWithinQuery(FIELD_NAME, ((DoubleRange)r).min, ((DoubleRange)r).max);
}
/** Basic test */
@ -103,4 +130,111 @@ public class TestDoubleRangeFieldQueries extends BaseRangeFieldQueryTestCase {
writer.close();
dir.close();
}
/** DoubleRange test class implementation - use to validate DoubleRangeField */
private class DoubleRange extends Range {
double[] min;
double[] max;
DoubleRange(double[] min, double[] max) {
assert min != null && max != null && min.length > 0 && max.length > 0
: "test box: min/max cannot be null or empty";
assert min.length == max.length : "test box: min/max length do not agree";
this.min = new double[min.length];
this.max = new double[max.length];
for (int d=0; d<min.length; ++d) {
if (min[d] > max[d]) {
// swap if max < min:
double temp = min[d];
min[d] = max[d];
max[d] = temp;
}
}
}
@Override
protected int numDimensions() {
return min.length;
}
@Override
protected Double getMin(int dim) {
return min[dim];
}
@Override
protected void setMin(int dim, Object val) {
min[dim] = (Double)val;
}
@Override
protected Double getMax(int dim) {
return max[dim];
}
@Override
protected void setMax(int dim, Object val) {
max[dim] = (Double)val;
}
@Override
protected boolean isEqual(Range other) {
DoubleRange o = (DoubleRange)other;
return Arrays.equals(min, o.min) && Arrays.equals(max, o.max);
}
@Override
protected boolean isDisjoint(Range o) {
DoubleRange other = (DoubleRange)o;
for (int d=0; d<this.min.length; ++d) {
if (this.min[d] > other.max[d] || this.max[d] < other.min[d]) {
// disjoint:
return true;
}
}
return false;
}
@Override
protected boolean isWithin(Range o) {
DoubleRange other = (DoubleRange)o;
for (int d=0; d<this.min.length; ++d) {
if ((this.min[d] >= other.min[d] && this.max[d] <= other.max[d]) == false) {
// not within:
return false;
}
}
return true;
}
@Override
protected boolean contains(Range o) {
DoubleRange other = (DoubleRange) o;
for (int d=0; d<this.min.length; ++d) {
if ((this.min[d] <= other.min[d] && this.max[d] >= other.max[d]) == false) {
// not contains:
return false;
}
}
return true;
}
@Override
public String toString() {
StringBuilder b = new StringBuilder();
b.append("Box(");
b.append(min[0]);
b.append(" TO ");
b.append(max[0]);
for (int d=1; d<min.length; ++d) {
b.append(", ");
b.append(min[d]);
b.append(" TO ");
b.append(max[d]);
}
b.append(")");
return b.toString();
}
}
}

View File

@ -0,0 +1,240 @@
/*
* 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.lucene.search;
import java.util.Arrays;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.FloatRangeField;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.RandomIndexWriter;
import org.apache.lucene.store.Directory;
/**
* Random testing for FloatRangeField Queries.
*/
public class TestFloatRangeFieldQueries extends BaseRangeFieldQueryTestCase {
private static final String FIELD_NAME = "floatRangeField";
private float nextFloatInternal() {
if (rarely()) {
return random().nextBoolean() ? Float.NEGATIVE_INFINITY : Float.POSITIVE_INFINITY;
}
float max = Float.MAX_VALUE / 2;
return (max + max) * random().nextFloat() - max;
}
@Override
protected Range nextRange(int dimensions) {
float[] min = new float[dimensions];
float[] max = new float[dimensions];
for (int d=0; d<dimensions; ++d) {
min[d] = nextFloatInternal();
max[d] = nextFloatInternal();
}
return new FloatRange(min, max);
}
@Override
protected FloatRangeField newRangeField(Range r) {
return new FloatRangeField(FIELD_NAME, ((FloatRange)r).min, ((FloatRange)r).max);
}
@Override
protected Query newIntersectsQuery(Range r) {
return FloatRangeField.newIntersectsQuery(FIELD_NAME, ((FloatRange)r).min, ((FloatRange)r).max);
}
@Override
protected Query newContainsQuery(Range r) {
return FloatRangeField.newContainsQuery(FIELD_NAME, ((FloatRange)r).min, ((FloatRange)r).max);
}
@Override
protected Query newWithinQuery(Range r) {
return FloatRangeField.newWithinQuery(FIELD_NAME, ((FloatRange)r).min, ((FloatRange)r).max);
}
/** Basic test */
public void testBasics() throws Exception {
Directory dir = newDirectory();
RandomIndexWriter writer = new RandomIndexWriter(random(), dir);
// intersects (within)
Document document = new Document();
document.add(new FloatRangeField(FIELD_NAME, new float[] {-10.0f, -10.0f}, new float[] {9.1f, 10.1f}));
writer.addDocument(document);
// intersects (crosses)
document = new Document();
document.add(new FloatRangeField(FIELD_NAME, new float[] {10.0f, -10.0f}, new float[] {20.0f, 10.0f}));
writer.addDocument(document);
// intersects (contains)
document = new Document();
document.add(new FloatRangeField(FIELD_NAME, new float[] {-20.0f, -20.0f}, new float[] {30.0f, 30.1f}));
writer.addDocument(document);
// intersects (crosses)
document = new Document();
document.add(new FloatRangeField(FIELD_NAME, new float[] {-11.1f, -11.2f}, new float[] {1.23f, 11.5f}));
writer.addDocument(document);
// intersects (crosses)
document = new Document();
document.add(new FloatRangeField(FIELD_NAME, new float[] {12.33f, 1.2f}, new float[] {15.1f, 29.9f}));
writer.addDocument(document);
// disjoint
document = new Document();
document.add(new FloatRangeField(FIELD_NAME, new float[] {-122.33f, 1.2f}, new float[] {-115.1f, 29.9f}));
writer.addDocument(document);
// intersects (crosses)
document = new Document();
document.add(new FloatRangeField(FIELD_NAME, new float[] {Float.NEGATIVE_INFINITY, 1.2f}, new float[] {-11.0f, 29.9f}));
writer.addDocument(document);
// equal (within, contains, intersects)
document = new Document();
document.add(new FloatRangeField(FIELD_NAME, new float[] {-11f, -15f}, new float[] {15f, 20f}));
writer.addDocument(document);
// search
IndexReader reader = writer.getReader();
IndexSearcher searcher = newSearcher(reader);
assertEquals(7, searcher.count(FloatRangeField.newIntersectsQuery(FIELD_NAME,
new float[] {-11.0f, -15.0f}, new float[] {15.0f, 20.0f})));
assertEquals(2, searcher.count(FloatRangeField.newWithinQuery(FIELD_NAME,
new float[] {-11.0f, -15.0f}, new float[] {15.0f, 20.0f})));
assertEquals(2, searcher.count(FloatRangeField.newContainsQuery(FIELD_NAME,
new float[] {-11.0f, -15.0f}, new float[] {15.0f, 20.0f})));
reader.close();
writer.close();
dir.close();
}
/** FloatRange test class implementation - use to validate FloatRangeField */
private class FloatRange extends Range {
float[] min;
float[] max;
FloatRange(float[] min, float[] max) {
assert min != null && max != null && min.length > 0 && max.length > 0
: "test box: min/max cannot be null or empty";
assert min.length == max.length : "test box: min/max length do not agree";
this.min = new float[min.length];
this.max = new float[max.length];
for (int d=0; d<min.length; ++d) {
if (min[d] > max[d]) {
// swap if max < min:
float temp = min[d];
min[d] = max[d];
max[d] = temp;
}
}
}
@Override
protected int numDimensions() {
return min.length;
}
@Override
protected Float getMin(int dim) {
return min[dim];
}
@Override
protected void setMin(int dim, Object val) {
min[dim] = (Float)val;
}
@Override
protected Float getMax(int dim) {
return max[dim];
}
@Override
protected void setMax(int dim, Object val) {
max[dim] = (Float)val;
}
@Override
protected boolean isEqual(Range other) {
FloatRange o = (FloatRange)other;
return Arrays.equals(min, o.min) && Arrays.equals(max, o.max);
}
@Override
protected boolean isDisjoint(Range o) {
FloatRange other = (FloatRange)o;
for (int d=0; d<this.min.length; ++d) {
if (this.min[d] > other.max[d] || this.max[d] < other.min[d]) {
// disjoint:
return true;
}
}
return false;
}
@Override
protected boolean isWithin(Range o) {
FloatRange other = (FloatRange)o;
for (int d=0; d<this.min.length; ++d) {
if ((this.min[d] >= other.min[d] && this.max[d] <= other.max[d]) == false) {
// not within:
return false;
}
}
return true;
}
@Override
protected boolean contains(Range o) {
FloatRange other = (FloatRange) o;
for (int d=0; d<this.min.length; ++d) {
if ((this.min[d] <= other.min[d] && this.max[d] >= other.max[d]) == false) {
// not contains:
return false;
}
}
return true;
}
@Override
public String toString() {
StringBuilder b = new StringBuilder();
b.append("Box(");
b.append(min[0]);
b.append(" TO ");
b.append(max[0]);
for (int d=1; d<min.length; ++d) {
b.append(", ");
b.append(min[d]);
b.append(" TO ");
b.append(max[d]);
}
b.append(")");
return b.toString();
}
}
}

View File

@ -0,0 +1,240 @@
/*
* 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.lucene.search;
import java.util.Arrays;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.IntRangeField;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.RandomIndexWriter;
import org.apache.lucene.store.Directory;
/**
* Random testing for IntRangeField Queries.
*/
public class TestIntRangeFieldQueries extends BaseRangeFieldQueryTestCase {
private static final String FIELD_NAME = "intRangeField";
private int nextIntInternal() {
if (rarely()) {
return random().nextBoolean() ? Integer.MAX_VALUE : Integer.MIN_VALUE;
}
int max = Integer.MAX_VALUE / 2;
return (max + max) * random().nextInt() - max;
}
@Override
protected Range nextRange(int dimensions) {
int[] min = new int[dimensions];
int[] max = new int[dimensions];
for (int d=0; d<dimensions; ++d) {
min[d] = nextIntInternal();
max[d] = nextIntInternal();
}
return new IntRange(min, max);
}
@Override
protected IntRangeField newRangeField(Range r) {
return new IntRangeField(FIELD_NAME, ((IntRange)r).min, ((IntRange)r).max);
}
@Override
protected Query newIntersectsQuery(Range r) {
return IntRangeField.newIntersectsQuery(FIELD_NAME, ((IntRange)r).min, ((IntRange)r).max);
}
@Override
protected Query newContainsQuery(Range r) {
return IntRangeField.newContainsQuery(FIELD_NAME, ((IntRange)r).min, ((IntRange)r).max);
}
@Override
protected Query newWithinQuery(Range r) {
return IntRangeField.newWithinQuery(FIELD_NAME, ((IntRange)r).min, ((IntRange)r).max);
}
/** Basic test */
public void testBasics() throws Exception {
Directory dir = newDirectory();
RandomIndexWriter writer = new RandomIndexWriter(random(), dir);
// intersects (within)
Document document = new Document();
document.add(new IntRangeField(FIELD_NAME, new int[] {-10, -10}, new int[] {9, 10}));
writer.addDocument(document);
// intersects (crosses)
document = new Document();
document.add(new IntRangeField(FIELD_NAME, new int[] {10, -10}, new int[] {20, 10}));
writer.addDocument(document);
// intersects (contains)
document = new Document();
document.add(new IntRangeField(FIELD_NAME, new int[] {-20, -20}, new int[] {30, 30}));
writer.addDocument(document);
// intersects (within)
document = new Document();
document.add(new IntRangeField(FIELD_NAME, new int[] {-11, -11}, new int[] {1, 11}));
writer.addDocument(document);
// intersects (crosses)
document = new Document();
document.add(new IntRangeField(FIELD_NAME, new int[] {12, 1}, new int[] {15, 29}));
writer.addDocument(document);
// disjoint
document = new Document();
document.add(new IntRangeField(FIELD_NAME, new int[] {-122, 1}, new int[] {-115, 29}));
writer.addDocument(document);
// intersects (crosses)
document = new Document();
document.add(new IntRangeField(FIELD_NAME, new int[] {Integer.MIN_VALUE, 1}, new int[] {-11, 29}));
writer.addDocument(document);
// equal (within, contains, intersects)
document = new Document();
document.add(new IntRangeField(FIELD_NAME, new int[] {-11, -15}, new int[] {15, 20}));
writer.addDocument(document);
// search
IndexReader reader = writer.getReader();
IndexSearcher searcher = newSearcher(reader);
assertEquals(7, searcher.count(IntRangeField.newIntersectsQuery(FIELD_NAME,
new int[] {-11, -15}, new int[] {15, 20})));
assertEquals(3, searcher.count(IntRangeField.newWithinQuery(FIELD_NAME,
new int[] {-11, -15}, new int[] {15, 20})));
assertEquals(2, searcher.count(IntRangeField.newContainsQuery(FIELD_NAME,
new int[] {-11, -15}, new int[] {15, 20})));
reader.close();
writer.close();
dir.close();
}
/** IntRange test class implementation - use to validate IntRangeField */
private class IntRange extends Range {
int[] min;
int[] max;
IntRange(int[] min, int[] max) {
assert min != null && max != null && min.length > 0 && max.length > 0
: "test box: min/max cannot be null or empty";
assert min.length == max.length : "test box: min/max length do not agree";
this.min = new int[min.length];
this.max = new int[max.length];
for (int d=0; d<min.length; ++d) {
if (min[d] > max[d]) {
// swap if max < min:
int temp = min[d];
min[d] = max[d];
max[d] = temp;
}
}
}
@Override
protected int numDimensions() {
return min.length;
}
@Override
protected Integer getMin(int dim) {
return min[dim];
}
@Override
protected void setMin(int dim, Object val) {
min[dim] = (Integer)val;
}
@Override
protected Integer getMax(int dim) {
return max[dim];
}
@Override
protected void setMax(int dim, Object val) {
max[dim] = (Integer)val;
}
@Override
protected boolean isEqual(Range other) {
IntRange o = (IntRange)other;
return Arrays.equals(min, o.min) && Arrays.equals(max, o.max);
}
@Override
protected boolean isDisjoint(Range o) {
IntRange other = (IntRange)o;
for (int d=0; d<this.min.length; ++d) {
if (this.min[d] > other.max[d] || this.max[d] < other.min[d]) {
// disjoint:
return true;
}
}
return false;
}
@Override
protected boolean isWithin(Range o) {
IntRange other = (IntRange)o;
for (int d=0; d<this.min.length; ++d) {
if ((this.min[d] >= other.min[d] && this.max[d] <= other.max[d]) == false) {
// not within:
return false;
}
}
return true;
}
@Override
protected boolean contains(Range o) {
IntRange other = (IntRange) o;
for (int d=0; d<this.min.length; ++d) {
if ((this.min[d] <= other.min[d] && this.max[d] >= other.max[d]) == false) {
// not contains:
return false;
}
}
return true;
}
@Override
public String toString() {
StringBuilder b = new StringBuilder();
b.append("Box(");
b.append(min[0]);
b.append(" TO ");
b.append(max[0]);
for (int d=1; d<min.length; ++d) {
b.append(", ");
b.append(min[d]);
b.append(" TO ");
b.append(max[d]);
}
b.append(")");
return b.toString();
}
}
}

View File

@ -0,0 +1,240 @@
/*
* 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.lucene.search;
import java.util.Arrays;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.LongRangeField;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.RandomIndexWriter;
import org.apache.lucene.store.Directory;
/**
* Random testing for LongRangeField Queries.
*/
public class TestLongRangeFieldQueries extends BaseRangeFieldQueryTestCase {
private static final String FIELD_NAME = "longRangeField";
private long nextLongInternal() {
if (rarely()) {
return random().nextBoolean() ? Long.MAX_VALUE : Long.MIN_VALUE;
}
long max = Long.MAX_VALUE / 2;
return (max + max) * random().nextLong() - max;
}
@Override
protected Range nextRange(int dimensions) {
long[] min = new long[dimensions];
long[] max = new long[dimensions];
for (int d=0; d<dimensions; ++d) {
min[d] = nextLongInternal();
max[d] = nextLongInternal();
}
return new LongRange(min, max);
}
@Override
protected LongRangeField newRangeField(Range r) {
return new LongRangeField(FIELD_NAME, ((LongRange)r).min, ((LongRange)r).max);
}
@Override
protected Query newIntersectsQuery(Range r) {
return LongRangeField.newIntersectsQuery(FIELD_NAME, ((LongRange)r).min, ((LongRange)r).max);
}
@Override
protected Query newContainsQuery(Range r) {
return LongRangeField.newContainsQuery(FIELD_NAME, ((LongRange)r).min, ((LongRange)r).max);
}
@Override
protected Query newWithinQuery(Range r) {
return LongRangeField.newWithinQuery(FIELD_NAME, ((LongRange)r).min, ((LongRange)r).max);
}
/** Basic test */
public void testBasics() throws Exception {
Directory dir = newDirectory();
RandomIndexWriter writer = new RandomIndexWriter(random(), dir);
// intersects (within)
Document document = new Document();
document.add(new LongRangeField(FIELD_NAME, new long[] {-10, -10}, new long[] {9, 10}));
writer.addDocument(document);
// intersects (crosses)
document = new Document();
document.add(new LongRangeField(FIELD_NAME, new long[] {10, -10}, new long[] {20, 10}));
writer.addDocument(document);
// intersects (contains)
document = new Document();
document.add(new LongRangeField(FIELD_NAME, new long[] {-20, -20}, new long[] {30, 30}));
writer.addDocument(document);
// intersects (within)
document = new Document();
document.add(new LongRangeField(FIELD_NAME, new long[] {-11, -11}, new long[] {1, 11}));
writer.addDocument(document);
// intersects (crosses)
document = new Document();
document.add(new LongRangeField(FIELD_NAME, new long[] {12, 1}, new long[] {15, 29}));
writer.addDocument(document);
// disjoint
document = new Document();
document.add(new LongRangeField(FIELD_NAME, new long[] {-122, 1}, new long[] {-115, 29}));
writer.addDocument(document);
// intersects (crosses)
document = new Document();
document.add(new LongRangeField(FIELD_NAME, new long[] {Long.MIN_VALUE, 1}, new long[] {-11, 29}));
writer.addDocument(document);
// equal (within, contains, intersects)
document = new Document();
document.add(new LongRangeField(FIELD_NAME, new long[] {-11, -15}, new long[] {15, 20}));
writer.addDocument(document);
// search
IndexReader reader = writer.getReader();
IndexSearcher searcher = newSearcher(reader);
assertEquals(7, searcher.count(LongRangeField.newIntersectsQuery(FIELD_NAME,
new long[] {-11, -15}, new long[] {15, 20})));
assertEquals(3, searcher.count(LongRangeField.newWithinQuery(FIELD_NAME,
new long[] {-11, -15}, new long[] {15, 20})));
assertEquals(2, searcher.count(LongRangeField.newContainsQuery(FIELD_NAME,
new long[] {-11, -15}, new long[] {15, 20})));
reader.close();
writer.close();
dir.close();
}
/** LongRange test class implementation - use to validate LongRangeField */
private class LongRange extends Range {
long[] min;
long[] max;
LongRange(long[] min, long[] max) {
assert min != null && max != null && min.length > 0 && max.length > 0
: "test box: min/max cannot be null or empty";
assert min.length == max.length : "test box: min/max length do not agree";
this.min = new long[min.length];
this.max = new long[max.length];
for (int d=0; d<min.length; ++d) {
if (min[d] > max[d]) {
// swap if max < min:
long temp = min[d];
min[d] = max[d];
max[d] = temp;
}
}
}
@Override
protected int numDimensions() {
return min.length;
}
@Override
protected Long getMin(int dim) {
return min[dim];
}
@Override
protected void setMin(int dim, Object val) {
min[dim] = (Long)val;
}
@Override
protected Long getMax(int dim) {
return max[dim];
}
@Override
protected void setMax(int dim, Object val) {
max[dim] = (Long)val;
}
@Override
protected boolean isEqual(Range other) {
LongRange o = (LongRange)other;
return Arrays.equals(min, o.min) && Arrays.equals(max, o.max);
}
@Override
protected boolean isDisjoint(Range o) {
LongRange other = (LongRange)o;
for (int d=0; d<this.min.length; ++d) {
if (this.min[d] > other.max[d] || this.max[d] < other.min[d]) {
// disjoint:
return true;
}
}
return false;
}
@Override
protected boolean isWithin(Range o) {
LongRange other = (LongRange)o;
for (int d=0; d<this.min.length; ++d) {
if ((this.min[d] >= other.min[d] && this.max[d] <= other.max[d]) == false) {
// not within:
return false;
}
}
return true;
}
@Override
protected boolean contains(Range o) {
LongRange other = (LongRange) o;
for (int d=0; d<this.min.length; ++d) {
if ((this.min[d] <= other.min[d] && this.max[d] >= other.max[d]) == false) {
// not contains:
return false;
}
}
return true;
}
@Override
public String toString() {
StringBuilder b = new StringBuilder();
b.append("Box(");
b.append(min[0]);
b.append(" TO ");
b.append(max[0]);
for (int d=1; d<min.length; ++d) {
b.append(", ");
b.append(min[d]);
b.append(" TO ");
b.append(max[d]);
}
b.append(")");
return b.toString();
}
}
}

View File

@ -177,11 +177,14 @@ public final class TestUtil {
assert hasNext;
T v = iterator.next();
assert allowNull || v != null;
try {
iterator.remove();
throw new AssertionError("broken iterator (supports remove): " + iterator);
} catch (UnsupportedOperationException expected) {
// ok
// for the first element, check that remove is not supported
if (i == 0) {
try {
iterator.remove();
throw new AssertionError("broken iterator (supports remove): " + iterator);
} catch (UnsupportedOperationException expected) {
// ok
}
}
}
assert !iterator.hasNext();

View File

@ -28,10 +28,6 @@ grant {
// should be enclosed within common.dir, but just in case:
permission java.io.FilePermission "${junit4.childvm.cwd}", "read";
// jenkins wants to read outside its sandbox, to use a special linedocs file.
// this is best effort and not really supported.
permission java.io.FilePermission "/home/jenkins/lucene-data/enwiki.random.lines.txt", "read";
// write only to sandbox
permission java.io.FilePermission "${junit4.childvm.cwd}${/}temp", "read,write,delete";
permission java.io.FilePermission "${junit4.childvm.cwd}${/}temp${/}-", "read,write,delete";

View File

@ -81,6 +81,10 @@ New Features
* SOLR-6465: CDCR: fall back to whole-index replication when tlogs are insufficient.
(Noble Paul, Renaud Delbru, shalin)
* SOLR-9320: A REPLACENODE command to decommission an existing node with another new node
(noble, Nitin Sharma, Varun Thacker)
* SOLR-9318: A DELETENODE command to delete all replicas in that node (noble, Nitin Sharma, Varun Thacker)
Bug Fixes
----------------------
@ -164,6 +168,21 @@ Bug Fixes
* SOLR-9397: Config API does not support adding caches (noble)
* SOLR-9405: ConcurrentModificationException in ZkStateReader.getStateWatchers.
(Alan Woodward, Edward Ribeiro, shalin)
* SOLR-9232: Admin UI now fully implements Swap Cores interface (Alexandre Rafalovitch)
* SOLR-8715: Admin UI's Schema screen now works for fields with stored=false and some content indexed (Alexandre Rafalovitch)
* SOLR-8911: In Admin UI, enable scrolling for overflowing Versions and JVM property values (Alexandre Rafalovitch)
* SOLR-9002: Admin UI now correctly displays json and text files in the collection/Files screen (Upayavira, Alexandre Rafalovitch)
* SOLR-8993: Admin UI now correctly supports multiple DIH handler end-points (Upayavira, Alexandre Rafalovitch)
* SOLR-9032: Admin UI now correctly implements Create Alias command (Upayavira, Alexandre Rafalovitch)
Optimizations
----------------------
@ -219,6 +238,15 @@ Other Changes
* SOLR-9367: Improved TestInjection's randomization logic to use LuceneTestCase.random() (hossman)
* SOLR-9331: Remove ReRankQuery's length constructor argument and member. (Christine Poerschke)
* SOLR-9092: For the delete replica command we attempt to send the core admin delete request only
if that node is actually up. (Jessica Cheng Mallet, Varun Thacker)
* SOLR-9410: Make ReRankQParserPlugin's private ReRankWeight a public class of its own. (Christine Poerschke)
* SOLR-9404: Refactor move/renames in JSON FacetProcessor and FacetFieldProcessor. (David Smiley)
================== 6.1.0 ==================
Consult the LUCENE_CHANGES.txt file for additional, low level, changes in this release.

View File

@ -0,0 +1,90 @@
/*
* 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.cloud;
import java.lang.invoke.MethodHandles;
import java.util.List;
import java.util.Locale;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.cloud.ClusterState;
import org.apache.solr.common.cloud.ZkNodeProps;
import org.apache.solr.common.util.NamedList;
import org.apache.zookeeper.KeeperException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import static org.apache.solr.common.cloud.ZkStateReader.COLLECTION_PROP;
import static org.apache.solr.common.cloud.ZkStateReader.SHARD_ID_PROP;
public class DeleteNodeCmd implements OverseerCollectionMessageHandler.Cmd {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
private final OverseerCollectionMessageHandler ocmh;
public DeleteNodeCmd(OverseerCollectionMessageHandler ocmh) {
this.ocmh = ocmh;
}
@Override
public void call(ClusterState state, ZkNodeProps message, NamedList results) throws Exception {
ocmh.checkRequired(message, "node");
String node = message.getStr("node");
if (!state.liveNodesContain(node)) {
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Source Node: " + node + " is not live");
}
List<ZkNodeProps> sourceReplicas = ReplaceNodeCmd.getReplicasOfNode(node, state);
cleanupReplicas(results, state, sourceReplicas, ocmh, node);
}
static void cleanupReplicas(NamedList results,
ClusterState clusterState,
List<ZkNodeProps> sourceReplicas,
OverseerCollectionMessageHandler ocmh, String node) throws InterruptedException {
CountDownLatch cleanupLatch = new CountDownLatch(sourceReplicas.size());
for (ZkNodeProps sourceReplica : sourceReplicas) {
log.info("Deleting replica for collection={} shard={} on node={}", sourceReplica.getStr(COLLECTION_PROP), sourceReplica.getStr(SHARD_ID_PROP), node);
NamedList deleteResult = new NamedList();
try {
ocmh.deleteReplica(clusterState, sourceReplica.plus("parallel", "true"), deleteResult, () -> {
cleanupLatch.countDown();
if (deleteResult.get("failure") != null) {
synchronized (results) {
results.add("failure", String.format(Locale.ROOT, "Failed to delete replica for collection=%s shard=%s" +
" on node=%s", sourceReplica.getStr(COLLECTION_PROP), sourceReplica.getStr(SHARD_ID_PROP), node));
}
}
});
} catch (KeeperException e) {
log.warn("Error deleting ", e);
cleanupLatch.countDown();
} catch (Exception e) {
log.warn("Error deleting ", e);
cleanupLatch.countDown();
throw e;
}
}
log.debug("Waiting for delete node action to complete");
cleanupLatch.await(5, TimeUnit.MINUTES);
}
}

View File

@ -16,6 +16,10 @@
*/
package org.apache.solr.cloud;
import java.io.IOException;
import org.apache.commons.io.IOUtils;
import org.apache.solr.common.cloud.ZkNodeProps;
import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.handler.component.ShardHandler;
import org.apache.solr.handler.component.ShardHandlerFactory;
@ -83,12 +87,20 @@ public class OverseerCollectionConfigSetProcessor extends OverseerTaskProcessor
zkStateReader, myId, shardHandlerFactory, adminPath, stats, overseer, overseerNodePrioritizer);
final OverseerConfigSetMessageHandler configMessageHandler = new OverseerConfigSetMessageHandler(
zkStateReader);
return message -> {
String operation = message.getStr(Overseer.QUEUE_OPERATION);
if (operation != null && operation.startsWith(CONFIGSETS_ACTION_PREFIX)) {
return configMessageHandler;
return new OverseerMessageHandlerSelector() {
@Override
public void close() throws IOException {
IOUtils.closeQuietly(collMessageHandler);
}
@Override
public OverseerMessageHandler selectOverseerMessageHandler(ZkNodeProps message) {
String operation = message.getStr(Overseer.QUEUE_OPERATION);
if (operation != null && operation.startsWith(CONFIGSETS_ACTION_PREFIX)) {
return configMessageHandler;
}
return collMessageHandler;
}
return collMessageHandler;
};
}
}

View File

@ -16,6 +16,7 @@
*/
package org.apache.solr.cloud;
import java.io.Closeable;
import java.io.IOException;
import java.lang.invoke.MethodHandles;
import java.net.URI;
@ -35,8 +36,13 @@ import java.util.Optional;
import java.util.Properties;
import java.util.Random;
import java.util.Set;
import java.util.concurrent.Callable;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.SynchronousQueue;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicReference;
import com.google.common.collect.ImmutableMap;
import org.apache.commons.lang.StringUtils;
import org.apache.solr.client.solrj.SolrResponse;
import org.apache.solr.client.solrj.SolrServerException;
@ -75,6 +81,7 @@ import org.apache.solr.common.params.CoreAdminParams;
import org.apache.solr.common.params.CoreAdminParams.CoreAdminAction;
import org.apache.solr.common.params.ModifiableSolrParams;
import org.apache.solr.common.params.ShardParams;
import org.apache.solr.common.util.ExecutorUtil;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.common.util.SimpleOrderedMap;
import org.apache.solr.common.util.StrUtils;
@ -88,6 +95,7 @@ import org.apache.solr.handler.component.ShardHandlerFactory;
import org.apache.solr.handler.component.ShardRequest;
import org.apache.solr.handler.component.ShardResponse;
import org.apache.solr.update.SolrIndexSplitter;
import org.apache.solr.util.DefaultSolrThreadFactory;
import org.apache.solr.util.RTimer;
import org.apache.solr.util.TimeOut;
import org.apache.solr.util.stats.Snapshot;
@ -119,10 +127,12 @@ import static org.apache.solr.common.params.CollectionParams.CollectionAction.BA
import static org.apache.solr.common.params.CollectionParams.CollectionAction.CREATE;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.CREATESHARD;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.DELETE;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.DELETENODE;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.DELETEREPLICAPROP;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.DELETESHARD;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.MIGRATESTATEFORMAT;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.REMOVEROLE;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.REPLACENODE;
import static org.apache.solr.common.params.CommonAdminParams.ASYNC;
import static org.apache.solr.common.params.CommonParams.NAME;
import static org.apache.solr.common.util.StrUtils.formatString;
@ -132,7 +142,7 @@ import static org.apache.solr.common.util.Utils.makeMap;
* A {@link OverseerMessageHandler} that handles Collections API related
* overseer messages.
*/
public class OverseerCollectionMessageHandler implements OverseerMessageHandler {
public class OverseerCollectionMessageHandler implements OverseerMessageHandler , Closeable {
public static final String NUM_SLICES = "numShards";
@ -172,7 +182,7 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler
private Overseer overseer;
private ShardHandlerFactory shardHandlerFactory;
private String adminPath;
private ZkStateReader zkStateReader;
ZkStateReader zkStateReader;
private String myId;
private Overseer.Stats stats;
private OverseerNodePrioritizer overseerPrioritizer;
@ -181,6 +191,9 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler
// This is used for handling mutual exclusion of the tasks.
final private LockTree lockTree = new LockTree();
ExecutorService tpe = new ExecutorUtil.MDCAwareThreadPoolExecutor(5, 10, 0L, TimeUnit.MILLISECONDS,
new SynchronousQueue<>(),
new DefaultSolrThreadFactory("OverseerCollectionMessageHandlerThreadFactory"));
static final Random RANDOM;
static {
@ -193,6 +206,7 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler
RANDOM = new Random(seed.hashCode());
}
}
private final Map<CollectionParams.CollectionAction, Cmd> commandMap;
public OverseerCollectionMessageHandler(ZkStateReader zkStateReader, String myId,
final ShardHandlerFactory shardHandlerFactory,
@ -207,6 +221,11 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler
this.stats = stats;
this.overseer = overseer;
this.overseerPrioritizer = overseerPrioritizer;
commandMap = new ImmutableMap.Builder<CollectionParams.CollectionAction, Cmd>()
.put(REPLACENODE, new ReplaceNodeCmd(this))
.put(DELETENODE, new DeleteNodeCmd(this))
.build()
;
}
@Override
@ -244,7 +263,7 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler
createShard(zkStateReader.getClusterState(), message, results);
break;
case DELETEREPLICA:
deleteReplica(zkStateReader.getClusterState(), message, results);
deleteReplica(zkStateReader.getClusterState(), message, results, null);
break;
case MIGRATE:
migrate(zkStateReader.getClusterState(), message, results);
@ -256,7 +275,7 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler
processRoleCommand(message, operation);
break;
case ADDREPLICA:
addReplica(zkStateReader.getClusterState(), message, results);
addReplica(zkStateReader.getClusterState(), message, results, null);
break;
case OVERSEERSTATUS:
getOverseerStatus(message, results);
@ -294,9 +313,15 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler
results.add("MOCK_FINISHED", System.currentTimeMillis());
break;
}
default:
throw new SolrException(ErrorCode.BAD_REQUEST, "Unknown operation:"
+ operation);
default: {
Cmd command = commandMap.get(action);
if (command != null) {
command.call(zkStateReader.getClusterState(), message, results);
} else {
throw new SolrException(ErrorCode.BAD_REQUEST, "Unknown operation:"
+ operation);
}
}
}
} catch (Exception e) {
String collName = message.getStr("collection");
@ -590,12 +615,13 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler
}
@SuppressWarnings("unchecked")
private void deleteReplica(ClusterState clusterState, ZkNodeProps message, NamedList results)
void deleteReplica(ClusterState clusterState, ZkNodeProps message, NamedList results, Runnable onComplete)
throws KeeperException, InterruptedException {
checkRequired(message, COLLECTION_PROP, SHARD_ID_PROP, REPLICA_PROP);
String collectionName = message.getStr(COLLECTION_PROP);
String shard = message.getStr(SHARD_ID_PROP);
String replicaName = message.getStr(REPLICA_PROP);
boolean parallel = message.getBool("parallel", false);
DocCollection coll = clusterState.getCollection(collectionName);
Slice slice = coll.getSlice(shard);
@ -623,9 +649,9 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler
ShardHandler shardHandler = shardHandlerFactory.getShardHandler();
String core = replica.getStr(ZkStateReader.CORE_NAME_PROP);
String asyncId = message.getStr(ASYNC);
Map<String, String> requestMap = null;
AtomicReference<Map<String, String>> requestMap = new AtomicReference<>(null);
if (asyncId != null) {
requestMap = new HashMap<>(1, 1.0f);
requestMap.set(new HashMap<>(1, 1.0f));
}
ModifiableSolrParams params = new ModifiableSolrParams();
@ -636,19 +662,46 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler
params.set(CoreAdminParams.DELETE_INSTANCE_DIR, message.getBool(CoreAdminParams.DELETE_INSTANCE_DIR, true));
params.set(CoreAdminParams.DELETE_DATA_DIR, message.getBool(CoreAdminParams.DELETE_DATA_DIR, true));
sendShardRequest(replica.getNodeName(), params, shardHandler, asyncId, requestMap);
boolean isLive = zkStateReader.getClusterState().getLiveNodes().contains(replica.getNodeName());
if (isLive) {
sendShardRequest(replica.getNodeName(), params, shardHandler, asyncId, requestMap.get());
}
processResponses(results, shardHandler, false, null, asyncId, requestMap);
Callable<Boolean> callable = () -> {
try {
if (isLive) {
processResponses(results, shardHandler, false, null, asyncId, requestMap.get());
//check if the core unload removed the corenode zk entry
if (waitForCoreNodeGone(collectionName, shard, replicaName, 5000)) return;
//check if the core unload removed the corenode zk entry
if (waitForCoreNodeGone(collectionName, shard, replicaName, 5000)) return Boolean.TRUE;
}
// try and ensure core info is removed from cluster state
deleteCoreNode(collectionName, replicaName, replica, core);
if (waitForCoreNodeGone(collectionName, shard, replicaName, 30000)) return;
throw new SolrException(ErrorCode.SERVER_ERROR,
"Could not remove replica : " + collectionName + "/" + shard + "/" + replicaName);
// try and ensure core info is removed from cluster state
deleteCoreNode(collectionName, replicaName, replica, core);
if (waitForCoreNodeGone(collectionName, shard, replicaName, 30000)) return Boolean.TRUE;
return Boolean.FALSE;
} catch (Exception e) {
results.add("failure", "Could not complete delete " + e.getMessage());
throw e;
} finally {
if (onComplete != null) onComplete.run();
}
};
if (!parallel) {
try {
if (!callable.call())
throw new SolrException(ErrorCode.SERVER_ERROR,
"Could not remove replica : " + collectionName + "/" + shard + "/" + replicaName);
} catch (InterruptedException | KeeperException e) {
throw e;
} catch (Exception ex) {
throw new SolrException(ErrorCode.UNKNOWN, "Error waiting for corenode gone", ex);
}
} else {
tpe.submit(callable);
}
}
private boolean waitForCoreNodeGone(String collectionName, String shard, String replicaName, int timeoutms) throws InterruptedException {
@ -679,7 +732,7 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler
Overseer.getStateUpdateQueue(zkStateReader.getZkClient()).offer(Utils.toJSON(m));
}
private void checkRequired(ZkNodeProps message, String... props) {
void checkRequired(ZkNodeProps message, String... props) {
for (String prop : props) {
if(message.get(prop) == null){
throw new SolrException(ErrorCode.BAD_REQUEST, StrUtils.join(Arrays.asList(props),',') +" are required params" );
@ -1137,7 +1190,7 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler
if (asyncId != null) {
propMap.put(ASYNC, asyncId);
}
addReplica(clusterState, new ZkNodeProps(propMap), results);
addReplica(clusterState, new ZkNodeProps(propMap), results, null);
}
processResponses(results, shardHandler, true, "SPLITSHARD failed to create subshard leaders", asyncId, requestMap);
@ -1307,7 +1360,7 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler
// now actually create replica cores on sub shard nodes
for (Map<String, Object> replica : replicas) {
addReplica(clusterState, new ZkNodeProps(replica), results);
addReplica(clusterState, new ZkNodeProps(replica), results, null);
}
processResponses(results, shardHandler, true, "SPLITSHARD failed to create subshard replicas", asyncId, requestMap);
@ -1681,7 +1734,7 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler
if(asyncId != null) {
props.put(ASYNC, asyncId);
}
addReplica(clusterState, new ZkNodeProps(props), results);
addReplica(clusterState, new ZkNodeProps(props), results, null);
processResponses(results, shardHandler, true, "MIGRATE failed to create replica of " +
"temporary collection in target leader node.", asyncId, requestMap);
@ -2110,12 +2163,14 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler
}
}
private void addReplica(ClusterState clusterState, ZkNodeProps message, NamedList results)
ZkNodeProps addReplica(ClusterState clusterState, ZkNodeProps message, NamedList results, Runnable onComplete)
throws KeeperException, InterruptedException {
log.info("addReplica() : {}", Utils.toJSONString(message));
String collection = message.getStr(COLLECTION_PROP);
String node = message.getStr(CoreAdminParams.NODE);
String shard = message.getStr(SHARD_ID_PROP);
String coreName = message.getStr(CoreAdminParams.NAME);
boolean parallel = message.getBool("parallel", false);
if (StringUtils.isBlank(coreName)) {
coreName = message.getStr(CoreAdminParams.PROPERTY_PREFIX + CoreAdminParams.NAME);
}
@ -2138,7 +2193,7 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler
node = getNodesForNewReplicas(clusterState, collection, shard, 1, node,
overseer.getZkController().getCoreContainer()).get(0).nodeName;
}
log.info("Node not provided, Identified {} for creating new replica", node);
log.info("Node Identified {} for creating new replica", node);
if (!clusterState.liveNodesContain(node)) {
throw new SolrException(ErrorCode.BAD_REQUEST, "Node: " + node + " is not live");
@ -2161,10 +2216,14 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler
if (!Overseer.isLegacy(zkStateReader)) {
if (!skipCreateReplicaInClusterState) {
ZkNodeProps props = new ZkNodeProps(Overseer.QUEUE_OPERATION, ADDREPLICA.toLower(), ZkStateReader.COLLECTION_PROP,
collection, ZkStateReader.SHARD_ID_PROP, shard, ZkStateReader.CORE_NAME_PROP, coreName,
ZkStateReader.STATE_PROP, Replica.State.DOWN.toString(), ZkStateReader.BASE_URL_PROP,
zkStateReader.getBaseUrlForNodeName(node), ZkStateReader.NODE_NAME_PROP, node);
ZkNodeProps props = new ZkNodeProps(
Overseer.QUEUE_OPERATION, ADDREPLICA.toLower(),
ZkStateReader.COLLECTION_PROP, collection,
ZkStateReader.SHARD_ID_PROP, shard,
ZkStateReader.CORE_NAME_PROP, coreName,
ZkStateReader.STATE_PROP, Replica.State.DOWN.toString(),
ZkStateReader.BASE_URL_PROP, zkStateReader.getBaseUrlForNodeName(node),
ZkStateReader.NODE_NAME_PROP, node);
Overseer.getStateUpdateQueue(zkStateReader.getZkClient()).offer(Utils.toJSON(props));
}
params.set(CoreAdminParams.CORE_NODE_NAME,
@ -2204,9 +2263,28 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler
Map<String,String> requestMap = new HashMap<>();
sendShardRequest(node, params, shardHandler, asyncId, requestMap);
processResponses(results, shardHandler, true, "ADDREPLICA failed to create replica", asyncId, requestMap);
final String fnode = node;
final String fcoreName = coreName;
waitForCoreNodeName(collection, node, coreName);
Runnable runnable = () -> {
processResponses(results, shardHandler, true, "ADDREPLICA failed to create replica", asyncId, requestMap);
waitForCoreNodeName(collection, fnode, fcoreName);
if (onComplete != null) onComplete.run();
};
if (!parallel) {
runnable.run();
} else {
tpe.submit(runnable);
}
return new ZkNodeProps(
ZkStateReader.COLLECTION_PROP, collection,
ZkStateReader.SHARD_ID_PROP, shard,
ZkStateReader.CORE_NAME_PROP, coreName,
ZkStateReader.NODE_NAME_PROP, node
);
}
private void processBackupAction(ZkNodeProps message, NamedList results) throws IOException, KeeperException, InterruptedException {
@ -2394,7 +2472,7 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler
}
addPropertyParams(message, propMap);
addReplica(clusterState, new ZkNodeProps(propMap), new NamedList());
addReplica(clusterState, new ZkNodeProps(propMap), new NamedList(), null);
}
//refresh the location copy of collection state
@ -2443,7 +2521,7 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler
}
addPropertyParams(message, propMap);
addReplica(zkStateReader.getClusterState(), new ZkNodeProps(propMap), results);
addReplica(zkStateReader.getClusterState(), new ZkNodeProps(propMap), results, null);
}
}
}
@ -2503,7 +2581,7 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler
}
return configName;
}
private void validateConfigOrThrowSolrException(String configName) throws KeeperException, InterruptedException {
boolean isValid = zkStateReader.getZkClient().exists(ZkConfigManager.CONFIGS_ZKNODE + "/" + configName, true);
if(!isValid) {
@ -2723,4 +2801,19 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler
);
}
@Override
public void close() throws IOException {
if (tpe != null) {
if (!tpe.isShutdown()) {
ExecutorUtil.shutdownAndAwaitTermination(tpe);
}
}
}
interface Cmd {
void call(ClusterState state, ZkNodeProps message, NamedList results) throws Exception;
}
}

View File

@ -31,6 +31,7 @@ import java.util.concurrent.TimeUnit;
import java.util.function.Predicate;
import com.google.common.collect.ImmutableSet;
import org.apache.commons.io.IOUtils;
import org.apache.solr.client.solrj.SolrResponse;
import org.apache.solr.cloud.OverseerTaskQueue.QueueEvent;
import org.apache.solr.cloud.Overseer.LeaderStatus;
@ -115,7 +116,7 @@ public class OverseerTaskProcessor implements Runnable, Closeable {
private final Object waitLock = new Object();
private OverseerMessageHandlerSelector selector;
protected OverseerMessageHandlerSelector selector;
private OverseerNodePrioritizer prioritizer;
@ -328,6 +329,7 @@ public class OverseerTaskProcessor implements Runnable, Closeable {
ExecutorUtil.shutdownAndAwaitTermination(tpe);
}
}
IOUtils.closeQuietly(selector);
}
public static List<String> getSortedOverseerNodeNames(SolrZkClient zk) throws KeeperException, InterruptedException {
@ -588,7 +590,7 @@ public class OverseerTaskProcessor implements Runnable, Closeable {
* messages only) , or a different handler could be selected based on the
* contents of the message.
*/
public interface OverseerMessageHandlerSelector {
public interface OverseerMessageHandlerSelector extends Closeable {
OverseerMessageHandler selectOverseerMessageHandler(ZkNodeProps message);
}

View File

@ -0,0 +1,163 @@
/*
* 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.cloud;
import java.lang.invoke.MethodHandles;
import java.util.ArrayList;
import java.util.List;
import java.util.Locale;
import java.util.Map;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.cloud.ClusterState;
import org.apache.solr.common.cloud.DocCollection;
import org.apache.solr.common.cloud.Replica;
import org.apache.solr.common.cloud.Slice;
import org.apache.solr.common.cloud.ZkNodeProps;
import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.params.CoreAdminParams;
import org.apache.solr.common.util.NamedList;
import org.apache.zookeeper.KeeperException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import static org.apache.solr.common.cloud.ZkStateReader.COLLECTION_PROP;
import static org.apache.solr.common.cloud.ZkStateReader.SHARD_ID_PROP;
public class ReplaceNodeCmd implements OverseerCollectionMessageHandler.Cmd {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
private final OverseerCollectionMessageHandler ocmh;
public ReplaceNodeCmd(OverseerCollectionMessageHandler ocmh) {
this.ocmh = ocmh;
}
@Override
public void call(ClusterState state, ZkNodeProps message, NamedList results) throws Exception {
ZkStateReader zkStateReader = ocmh.zkStateReader;
ocmh.checkRequired(message, "source", "target");
String source = message.getStr("source");
String target = message.getStr("target");
boolean parallel = message.getBool("parallel", false);
ClusterState clusterState = zkStateReader.getClusterState();
if (!clusterState.liveNodesContain(source)) {
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Source Node: " + source + " is not live");
}
if (!clusterState.liveNodesContain(target)) {
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Target Node: " + target + " is not live");
}
List<ZkNodeProps> sourceReplicas = getReplicasOfNode(source, clusterState);
List<ZkNodeProps> createdReplicas = new ArrayList<>();
AtomicBoolean anyOneFailed = new AtomicBoolean(false);
CountDownLatch countDownLatch = new CountDownLatch(sourceReplicas.size());
for (ZkNodeProps sourceReplica : sourceReplicas) {
NamedList nl = new NamedList();
log.info("Going to create replica for collection={} shard={} on node={}", sourceReplica.getStr(COLLECTION_PROP), sourceReplica.getStr(SHARD_ID_PROP), target);
ZkNodeProps msg = sourceReplica.plus("parallel", String.valueOf(parallel)).plus(CoreAdminParams.NODE, target);
final ZkNodeProps addedReplica = ocmh.addReplica(clusterState,
msg, nl, () -> {
countDownLatch.countDown();
if (nl.get("failure") != null) {
String errorString = String.format(Locale.ROOT, "Failed to create replica for collection=%s shard=%s" +
" on node=%s", sourceReplica.getStr(COLLECTION_PROP), sourceReplica.getStr(SHARD_ID_PROP), target);
log.warn(errorString);
// one replica creation failed. Make the best attempt to
// delete all the replicas created so far in the target
// and exit
synchronized (results) {
results.add("failure", errorString);
anyOneFailed.set(true);
}
} else {
log.debug("Successfully created replica for collection={} shard={} on node={}",
sourceReplica.getStr(COLLECTION_PROP), sourceReplica.getStr(SHARD_ID_PROP), target);
}
});
if (addedReplica != null) {
createdReplicas.add(addedReplica);
}
}
log.debug("Waiting for replace node action to complete");
countDownLatch.await(5, TimeUnit.MINUTES);
log.debug("Finished waiting for replace node action to complete");
if (anyOneFailed.get()) {
log.info("Failed to create some replicas. Cleaning up all replicas on target node");
CountDownLatch cleanupLatch = new CountDownLatch(createdReplicas.size());
for (ZkNodeProps createdReplica : createdReplicas) {
NamedList deleteResult = new NamedList();
try {
ocmh.deleteReplica(zkStateReader.getClusterState(), createdReplica.plus("parallel", "true"), deleteResult, () -> {
cleanupLatch.countDown();
if (deleteResult.get("failure") != null) {
synchronized (results) {
results.add("failure", "Could not cleanup, because of : " + deleteResult.get("failure"));
}
}
});
} catch (KeeperException e) {
cleanupLatch.countDown();
log.warn("Error deleting replica ", e);
} catch (Exception e) {
log.warn("Error deleting replica ", e);
cleanupLatch.countDown();
throw e;
}
}
cleanupLatch.await(5, TimeUnit.MINUTES);
}
// we have reached this far means all replicas could be recreated
//now cleanup the replicas in the source node
DeleteNodeCmd.cleanupReplicas(results, state, sourceReplicas, ocmh, source);
results.add("success", "REPLACENODE action completed successfully from : " + source + " to : " + target);
}
static List<ZkNodeProps> getReplicasOfNode(String source, ClusterState state) {
List<ZkNodeProps> sourceReplicas = new ArrayList<>();
for (Map.Entry<String, DocCollection> e : state.getCollectionsMap().entrySet()) {
for (Slice slice : e.getValue().getSlices()) {
for (Replica replica : slice.getReplicas()) {
if (source.equals(replica.getNodeName())) {
ZkNodeProps props = new ZkNodeProps(
COLLECTION_PROP, e.getKey(),
SHARD_ID_PROP, slice.getName(),
ZkStateReader.CORE_NAME_PROP, replica.getCoreName(),
ZkStateReader.REPLICA_PROP, replica.getName(),
CoreAdminParams.NODE, source);
sourceReplicas.add(props
);
}
}
}
}
return sourceReplicas;
}
}

View File

@ -777,7 +777,10 @@ public class CollectionsHandler extends RequestHandlerBase implements Permission
req.getParams().getAll(params, COLL_CONF, REPLICATION_FACTOR, MAX_SHARDS_PER_NODE, STATE_FORMAT, AUTO_ADD_REPLICAS);
copyPropertiesWithPrefix(req.getParams(), params, COLL_PROP_PREFIX);
return params;
});
}),
REPLACENODE_OP(REPLACENODE, (req, rsp, h) -> req.getParams().required().getAll(req.getParams().getAll(null, "parallel"), "source", "target")),
DELETENODE_OP(DELETENODE, (req, rsp, h) -> req.getParams().required().getAll(null, "node"));
public final CollectionOp fun;
CollectionAction action;
long timeOut;

View File

@ -38,5 +38,5 @@ public interface CacheRegenerator {
* @param oldVal the old value of the cache item
* @return true to continue with autowarming, false to stop
*/
public boolean regenerateItem(SolrIndexSearcher newSearcher, SolrCache newCache, SolrCache oldCache, Object oldKey, Object oldVal) throws IOException;
boolean regenerateItem(SolrIndexSearcher newSearcher, SolrCache newCache, SolrCache oldCache, Object oldKey, Object oldVal) throws IOException;
}

View File

@ -25,8 +25,6 @@ import com.carrotsearch.hppc.IntIntHashMap;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.search.FilterWeight;
import org.apache.lucene.search.Explanation;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.LeafCollector;
import org.apache.lucene.search.MatchAllDocsQuery;
@ -42,7 +40,6 @@ import org.apache.lucene.search.TopScoreDocCollector;
import org.apache.lucene.search.Weight;
import org.apache.lucene.util.BytesRef;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.params.CommonParams;
import org.apache.solr.common.params.SolrParams;
import org.apache.solr.handler.component.MergeStrategy;
import org.apache.solr.handler.component.QueryElevationComponent;
@ -91,10 +88,7 @@ public class ReRankQParserPlugin extends QParserPlugin {
double reRankWeight = localParams.getDouble(RERANK_WEIGHT, RERANK_WEIGHT_DEFAULT);
int start = params.getInt(CommonParams.START,CommonParams.START_DEFAULT);
int rows = params.getInt(CommonParams.ROWS,CommonParams.ROWS_DEFAULT);
int length = start+rows;
return new ReRankQuery(reRankQuery, reRankDocs, reRankWeight, length);
return new ReRankQuery(reRankQuery, reRankDocs, reRankWeight);
}
}
@ -121,7 +115,6 @@ public class ReRankQParserPlugin extends QParserPlugin {
private Query mainQuery = defaultQuery;
final private Query reRankQuery;
final private int reRankDocs;
final private int length;
final private double reRankWeight;
final private Rescorer reRankQueryRescorer;
private Map<BytesRef, Integer> boostedPriority;
@ -142,11 +135,10 @@ public class ReRankQParserPlugin extends QParserPlugin {
reRankDocs == rrq.reRankDocs;
}
public ReRankQuery(Query reRankQuery, int reRankDocs, double reRankWeight, int length) {
public ReRankQuery(Query reRankQuery, int reRankDocs, double reRankWeight) {
this.reRankQuery = reRankQuery;
this.reRankDocs = reRankDocs;
this.reRankWeight = reRankWeight;
this.length = length;
this.reRankQueryRescorer = new ReRankQueryRescorer(reRankQuery, reRankWeight);
}
@ -171,7 +163,7 @@ public class ReRankQParserPlugin extends QParserPlugin {
}
}
return new ReRankCollector(reRankDocs, length, reRankQueryRescorer, cmd, searcher, boostedPriority);
return new ReRankCollector(reRankDocs, len, reRankQueryRescorer, cmd, searcher, boostedPriority);
}
@Override
@ -188,29 +180,14 @@ public class ReRankQParserPlugin extends QParserPlugin {
public Query rewrite(IndexReader reader) throws IOException {
Query q = mainQuery.rewrite(reader);
if (q != mainQuery) {
return new ReRankQuery(reRankQuery, reRankDocs, reRankWeight, length).wrap(q);
return new ReRankQuery(reRankQuery, reRankDocs, reRankWeight).wrap(q);
}
return super.rewrite(reader);
}
public Weight createWeight(IndexSearcher searcher, boolean needsScores) throws IOException{
return new ReRankWeight(mainQuery, reRankQueryRescorer, searcher, needsScores);
}
}
private class ReRankWeight extends FilterWeight {
private IndexSearcher searcher;
final private Rescorer reRankQueryRescorer;
public ReRankWeight(Query mainQuery, Rescorer reRankQueryRescorer, IndexSearcher searcher, boolean needsScores) throws IOException {
super(mainQuery, mainQuery.createWeight(searcher, needsScores));
this.searcher = searcher;
this.reRankQueryRescorer = reRankQueryRescorer;
}
public Explanation explain(LeafReaderContext context, int doc) throws IOException {
Explanation mainExplain = in.explain(context, doc);
return reRankQueryRescorer.explain(searcher, mainExplain, context.docBase+doc);
final Weight mainWeight = mainQuery.createWeight(searcher, needsScores);
return new ReRankWeight(mainQuery, reRankQueryRescorer, searcher, mainWeight);
}
}

View File

@ -0,0 +1,48 @@
/*
* 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 org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.search.Explanation;
import org.apache.lucene.search.FilterWeight;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.Rescorer;
import org.apache.lucene.search.Weight;
/**
* A {@code Weight} used by reranking queries.
*/
public class ReRankWeight extends FilterWeight {
final private IndexSearcher searcher;
final private Rescorer reRankQueryRescorer;
public ReRankWeight(Query mainQuery, Rescorer reRankQueryRescorer, IndexSearcher searcher, Weight mainWeight) throws IOException {
super(mainQuery, mainWeight);
this.searcher = searcher;
this.reRankQueryRescorer = reRankQueryRescorer;
}
public Explanation explain(LeafReaderContext context, int doc) throws IOException {
final Explanation mainExplain = in.explain(context, doc);
return reRankQueryRescorer.explain(searcher, mainExplain, context.docBase+doc);
}
}

View File

@ -529,62 +529,50 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable, SolrI
//
public static void initRegenerators(SolrConfig solrConfig) {
if (solrConfig.fieldValueCacheConfig != null && solrConfig.fieldValueCacheConfig.getRegenerator() == null) {
solrConfig.fieldValueCacheConfig.setRegenerator(new CacheRegenerator() {
@Override
public boolean regenerateItem(SolrIndexSearcher newSearcher, SolrCache newCache, SolrCache oldCache,
Object oldKey, Object oldVal) throws IOException {
if (oldVal instanceof UnInvertedField) {
UnInvertedField.getUnInvertedField((String) oldKey, newSearcher);
}
return true;
solrConfig.fieldValueCacheConfig.setRegenerator((newSearcher, newCache, oldCache, oldKey, oldVal) -> {
if (oldVal instanceof UnInvertedField) {
UnInvertedField.getUnInvertedField((String) oldKey, newSearcher);
}
return true;
});
}
if (solrConfig.filterCacheConfig != null && solrConfig.filterCacheConfig.getRegenerator() == null) {
solrConfig.filterCacheConfig.setRegenerator(new CacheRegenerator() {
@Override
public boolean regenerateItem(SolrIndexSearcher newSearcher, SolrCache newCache, SolrCache oldCache,
Object oldKey, Object oldVal) throws IOException {
newSearcher.cacheDocSet((Query) oldKey, null, false);
return true;
}
solrConfig.filterCacheConfig.setRegenerator((newSearcher, newCache, oldCache, oldKey, oldVal) -> {
newSearcher.cacheDocSet((Query) oldKey, null, false);
return true;
});
}
if (solrConfig.queryResultCacheConfig != null && solrConfig.queryResultCacheConfig.getRegenerator() == null) {
final int queryResultWindowSize = solrConfig.queryResultWindowSize;
solrConfig.queryResultCacheConfig.setRegenerator(new CacheRegenerator() {
@Override
public boolean regenerateItem(SolrIndexSearcher newSearcher, SolrCache newCache, SolrCache oldCache,
Object oldKey, Object oldVal) throws IOException {
QueryResultKey key = (QueryResultKey) oldKey;
int nDocs = 1;
// request 1 doc and let caching round up to the next window size...
// unless the window size is <=1, in which case we will pick
// the minimum of the number of documents requested last time and
// a reasonable number such as 40.
// TODO: make more configurable later...
solrConfig.queryResultCacheConfig.setRegenerator((newSearcher, newCache, oldCache, oldKey, oldVal) -> {
QueryResultKey key = (QueryResultKey) oldKey;
int nDocs = 1;
// request 1 doc and let caching round up to the next window size...
// unless the window size is <=1, in which case we will pick
// the minimum of the number of documents requested last time and
// a reasonable number such as 40.
// TODO: make more configurable later...
if (queryResultWindowSize <= 1) {
DocList oldList = (DocList) oldVal;
int oldnDocs = oldList.offset() + oldList.size();
// 40 has factors of 2,4,5,10,20
nDocs = Math.min(oldnDocs, 40);
}
int flags = NO_CHECK_QCACHE | key.nc_flags;
QueryCommand qc = new QueryCommand();
qc.setQuery(key.query)
.setFilterList(key.filters)
.setSort(key.sort)
.setLen(nDocs)
.setSupersetMaxDoc(nDocs)
.setFlags(flags);
QueryResult qr = new QueryResult();
newSearcher.getDocListC(qr, qc);
return true;
if (queryResultWindowSize <= 1) {
DocList oldList = (DocList) oldVal;
int oldnDocs = oldList.offset() + oldList.size();
// 40 has factors of 2,4,5,10,20
nDocs = Math.min(oldnDocs, 40);
}
int flags = NO_CHECK_QCACHE | key.nc_flags;
QueryCommand qc = new QueryCommand();
qc.setQuery(key.query)
.setFilterList(key.filters)
.setSort(key.sort)
.setLen(nDocs)
.setSupersetMaxDoc(nDocs)
.setFlags(flags);
QueryResult qr = new QueryResult();
newSearcher.getDocListC(qr, qc);
return true;
});
}
}

View File

@ -0,0 +1,369 @@
/*
* 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.facet;
import java.io.IOException;
import java.util.HashMap;
import java.util.LinkedHashMap;
import java.util.Map;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.search.Query;
import org.apache.solr.common.util.SimpleOrderedMap;
import org.apache.solr.schema.SchemaField;
import org.apache.solr.search.DocSet;
/**
* Facet processing based on field values. (not range nor by query)
* @see FacetField
*/
abstract class FacetFieldProcessor extends FacetProcessor<FacetField> {
SchemaField sf;
SlotAcc indexOrderAcc;
int effectiveMincount;
Map<String,AggValueSource> deferredAggs; // null if none
// TODO: push any of this down to base class?
//
// For sort="x desc", collectAcc would point to "x", and sortAcc would also point to "x".
// collectAcc would be used to accumulate all buckets, and sortAcc would be used to sort those buckets.
//
SlotAcc collectAcc; // Accumulator to collect across entire domain (in addition to the countAcc). May be null.
SlotAcc sortAcc; // Accumulator to use for sorting *only* (i.e. not used for collection). May be an alias of countAcc, collectAcc, or indexOrderAcc
SlotAcc[] otherAccs; // Accumulators that do not need to be calculated across all buckets.
SpecialSlotAcc allBucketsAcc; // this can internally refer to otherAccs and/or collectAcc. setNextReader should be called on otherAccs directly if they exist.
FacetFieldProcessor(FacetContext fcontext, FacetField freq, SchemaField sf) {
super(fcontext, freq);
this.sf = sf;
this.effectiveMincount = (int)(fcontext.isShard() ? Math.min(1 , freq.mincount) : freq.mincount);
}
// This is used to create accs for second phase (or to create accs for all aggs)
@Override
protected void createAccs(int docCount, int slotCount) throws IOException {
if (accMap == null) {
accMap = new LinkedHashMap<>();
}
// allow a custom count acc to be used
if (countAcc == null) {
countAcc = new CountSlotArrAcc(fcontext, slotCount);
countAcc.key = "count";
}
if (accs != null) {
// reuse these accs, but reset them first
for (SlotAcc acc : accs) {
acc.reset();
}
return;
} else {
accs = new SlotAcc[ freq.getFacetStats().size() ];
}
int accIdx = 0;
for (Map.Entry<String,AggValueSource> entry : freq.getFacetStats().entrySet()) {
SlotAcc acc = null;
if (slotCount == 1) {
acc = accMap.get(entry.getKey());
if (acc != null) {
acc.reset();
}
}
if (acc == null) {
acc = entry.getValue().createSlotAcc(fcontext, docCount, slotCount);
acc.key = entry.getKey();
accMap.put(acc.key, acc);
}
accs[accIdx++] = acc;
}
}
void createCollectAcc(int numDocs, int numSlots) throws IOException {
accMap = new LinkedHashMap<>();
// we always count...
// allow a subclass to set a custom counter.
if (countAcc == null) {
countAcc = new CountSlotArrAcc(fcontext, numSlots);
}
if ("count".equals(freq.sortVariable)) {
sortAcc = countAcc;
deferredAggs = freq.getFacetStats();
} else if ("index".equals(freq.sortVariable)) {
// allow subclass to set indexOrderAcc first
if (indexOrderAcc == null) {
// This sorting accumulator just goes by the slot number, so does not need to be collected
// and hence does not need to find it's way into the accMap or accs array.
indexOrderAcc = new SortSlotAcc(fcontext);
}
sortAcc = indexOrderAcc;
deferredAggs = freq.getFacetStats();
} else {
AggValueSource sortAgg = freq.getFacetStats().get(freq.sortVariable);
if (sortAgg != null) {
collectAcc = sortAgg.createSlotAcc(fcontext, numDocs, numSlots);
collectAcc.key = freq.sortVariable; // TODO: improve this
}
sortAcc = collectAcc;
deferredAggs = new HashMap<>(freq.getFacetStats());
deferredAggs.remove(freq.sortVariable);
}
if (deferredAggs.size() == 0) {
deferredAggs = null;
}
boolean needOtherAccs = freq.allBuckets; // TODO: use for missing too...
if (!needOtherAccs) {
// we may need them later, but we don't want to create them now
// otherwise we won't know if we need to call setNextReader on them.
return;
}
// create the deferred aggs up front for use by allBuckets
createOtherAccs(numDocs, 1);
}
private void createOtherAccs(int numDocs, int numSlots) throws IOException {
if (otherAccs != null) {
// reuse existing accumulators
for (SlotAcc acc : otherAccs) {
acc.reset(); // todo - make reset take numDocs and numSlots?
}
return;
}
int numDeferred = deferredAggs == null ? 0 : deferredAggs.size();
if (numDeferred <= 0) return;
otherAccs = new SlotAcc[ numDeferred ];
int otherAccIdx = 0;
for (Map.Entry<String,AggValueSource> entry : deferredAggs.entrySet()) {
AggValueSource agg = entry.getValue();
SlotAcc acc = agg.createSlotAcc(fcontext, numDocs, numSlots);
acc.key = entry.getKey();
accMap.put(acc.key, acc);
otherAccs[otherAccIdx++] = acc;
}
if (numDeferred == freq.getFacetStats().size()) {
// accs and otherAccs are the same...
accs = otherAccs;
}
}
int collectFirstPhase(DocSet docs, int slot) throws IOException {
int num = -1;
if (collectAcc != null) {
num = collectAcc.collect(docs, slot);
}
if (allBucketsAcc != null) {
num = allBucketsAcc.collect(docs, slot);
}
return num >= 0 ? num : docs.size();
}
void collectFirstPhase(int segDoc, int slot) throws IOException {
if (collectAcc != null) {
collectAcc.collect(segDoc, slot);
}
if (allBucketsAcc != null) {
allBucketsAcc.collect(segDoc, slot);
}
}
void fillBucket(SimpleOrderedMap<Object> target, int count, int slotNum, DocSet subDomain, Query filter) throws IOException {
target.add("count", count);
if (count <= 0 && !freq.processEmpty) return;
if (collectAcc != null && slotNum >= 0) {
collectAcc.setValues(target, slotNum);
}
createOtherAccs(-1, 1);
if (otherAccs == null && freq.subFacets.isEmpty()) return;
if (subDomain == null) {
subDomain = fcontext.searcher.getDocSet(filter, fcontext.base);
}
// if no subFacets, we only need a DocSet
// otherwise we need more?
// TODO: save something generic like "slotNum" in the context and use that to implement things like filter exclusion if necessary?
// Hmmm, but we need to look up some stuff anyway (for the label?)
// have a method like "DocSet applyConstraint(facet context, DocSet parent)"
// that's needed for domain changing things like joins anyway???
if (otherAccs != null) {
// do acc at a time (traversing domain each time) or do all accs for each doc?
for (SlotAcc acc : otherAccs) {
acc.reset(); // TODO: only needed if we previously used for allBuckets or missing
acc.collect(subDomain, 0);
acc.setValues(target, 0);
}
}
processSubs(target, filter, subDomain);
}
@Override
protected void processStats(SimpleOrderedMap<Object> bucket, DocSet docs, int docCount) throws IOException {
if (docCount == 0 && !freq.processEmpty || freq.getFacetStats().size() == 0) {
bucket.add("count", docCount);
return;
}
createAccs(docCount, 1);
int collected = collect(docs, 0);
// countAcc.incrementCount(0, collected); // should we set the counton the acc instead of just passing it?
assert collected == docCount;
addStats(bucket, collected, 0);
}
// overrides but with different signature!
private void addStats(SimpleOrderedMap<Object> target, int count, int slotNum) throws IOException {
target.add("count", count);
if (count > 0 || freq.processEmpty) {
for (SlotAcc acc : accs) {
acc.setValues(target, slotNum);
}
}
}
@Override
void setNextReader(LeafReaderContext ctx) throws IOException {
// base class calls this (for missing bucket...) ... go over accs[] in that case
super.setNextReader(ctx);
}
void setNextReaderFirstPhase(LeafReaderContext ctx) throws IOException {
if (collectAcc != null) {
collectAcc.setNextReader(ctx);
}
if (otherAccs != null) {
for (SlotAcc acc : otherAccs) {
acc.setNextReader(ctx);
}
}
}
static class Slot {
int slot;
public int tiebreakCompare(int slotA, int slotB) {
return slotB - slotA;
}
}
static class SpecialSlotAcc extends SlotAcc {
SlotAcc collectAcc;
SlotAcc[] otherAccs;
int collectAccSlot;
int otherAccsSlot;
long count;
SpecialSlotAcc(FacetContext fcontext, SlotAcc collectAcc, int collectAccSlot, SlotAcc[] otherAccs, int otherAccsSlot) {
super(fcontext);
this.collectAcc = collectAcc;
this.collectAccSlot = collectAccSlot;
this.otherAccs = otherAccs;
this.otherAccsSlot = otherAccsSlot;
}
public int getCollectAccSlot() { return collectAccSlot; }
public int getOtherAccSlot() { return otherAccsSlot; }
long getSpecialCount() {
return count;
}
@Override
public void collect(int doc, int slot) throws IOException {
assert slot != collectAccSlot || slot < 0;
count++;
if (collectAcc != null) {
collectAcc.collect(doc, collectAccSlot);
}
if (otherAccs != null) {
for (SlotAcc otherAcc : otherAccs) {
otherAcc.collect(doc, otherAccsSlot);
}
}
}
@Override
public void setNextReader(LeafReaderContext readerContext) throws IOException {
// collectAcc and otherAccs will normally have setNextReader called directly on them.
// This, however, will be used when collect(DocSet,slot) variant is used on this Acc.
if (collectAcc != null) {
collectAcc.setNextReader(readerContext);
}
if (otherAccs != null) {
for (SlotAcc otherAcc : otherAccs) {
otherAcc.setNextReader(readerContext);
}
}
}
@Override
public int compare(int slotA, int slotB) {
throw new UnsupportedOperationException();
}
@Override
public Object getValue(int slotNum) throws IOException {
throw new UnsupportedOperationException();
}
@Override
public void setValues(SimpleOrderedMap<Object> bucket, int slotNum) throws IOException {
if (collectAcc != null) {
collectAcc.setValues(bucket, collectAccSlot);
}
if (otherAccs != null) {
for (SlotAcc otherAcc : otherAccs) {
otherAcc.setValues(bucket, otherAccsSlot);
}
}
}
@Override
public void reset() {
// reset should be called on underlying accs
// TODO: but in case something does need to be done here, should we require this method to be called but do nothing for now?
throw new UnsupportedOperationException();
}
@Override
public void resize(Resizer resizer) {
// someone else will call resize on collectAcc directly
if (collectAccSlot >= 0) {
collectAccSlot = resizer.getNewSlot(collectAccSlot);
}
}
}
}

View File

@ -0,0 +1,213 @@
/*
* 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.facet;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import org.apache.lucene.index.Term;
import org.apache.lucene.search.TermQuery;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefBuilder;
import org.apache.lucene.util.PriorityQueue;
import org.apache.solr.common.util.SimpleOrderedMap;
import org.apache.solr.schema.SchemaField;
/**
* Base class for DV/UIF accumulating counts into an array by ordinal.
* It can handle terms (strings), not numbers directly but those encoded as terms, and is multi-valued capable.
*/
abstract class FacetFieldProcessorByArray extends FacetFieldProcessor {
BytesRefBuilder prefixRef;
int startTermIndex;
int endTermIndex;
int nTerms;
int nDocs;
int maxSlots;
int allBucketsSlot = -1; // slot for the primary Accs (countAcc, collectAcc)
FacetFieldProcessorByArray(FacetContext fcontext, FacetField freq, SchemaField sf) {
super(fcontext, freq, sf);
}
abstract protected void findStartAndEndOrds() throws IOException;
abstract protected void collectDocs() throws IOException;
/** this BytesRef may be shared across calls and should be deep-cloned if necessary */
abstract protected BytesRef lookupOrd(int ord) throws IOException;
@Override
public void process() throws IOException {
super.process();
sf = fcontext.searcher.getSchema().getField(freq.field);
response = getFieldCacheCounts();
}
private SimpleOrderedMap<Object> getFieldCacheCounts() throws IOException {
String prefix = freq.prefix;
if (prefix == null || prefix.length() == 0) {
prefixRef = null;
} else {
prefixRef = new BytesRefBuilder();
prefixRef.copyChars(prefix);
}
findStartAndEndOrds();
maxSlots = nTerms;
if (freq.allBuckets) {
allBucketsSlot = maxSlots++;
}
createCollectAcc(nDocs, maxSlots);
if (freq.allBuckets) {
allBucketsAcc = new SpecialSlotAcc(fcontext, collectAcc, allBucketsSlot, otherAccs, 0);
}
collectDocs();
return findTopSlots();
}
private SimpleOrderedMap<Object> findTopSlots() throws IOException {
SimpleOrderedMap<Object> res = new SimpleOrderedMap<>();
int numBuckets = 0;
List<Object> bucketVals = null;
if (freq.numBuckets && fcontext.isShard()) {
bucketVals = new ArrayList<>(100);
}
int off = fcontext.isShard() ? 0 : (int) freq.offset;
// add a modest amount of over-request if this is a shard request
int lim = freq.limit >= 0 ? (fcontext.isShard() ? (int)(freq.limit*1.1+4) : (int)freq.limit) : Integer.MAX_VALUE;
int maxsize = (int)(freq.limit >= 0 ? freq.offset + lim : Integer.MAX_VALUE - 1);
maxsize = Math.min(maxsize, nTerms);
final int sortMul = freq.sortDirection.getMultiplier();
final SlotAcc sortAcc = this.sortAcc;
PriorityQueue<Slot> queue = new PriorityQueue<Slot>(maxsize) {
@Override
protected boolean lessThan(Slot a, Slot b) {
int cmp = sortAcc.compare(a.slot, b.slot) * sortMul;
return cmp == 0 ? b.slot < a.slot : cmp < 0;
}
};
Slot bottom = null;
for (int i = 0; i < nTerms; i++) {
// screen out buckets not matching mincount immediately (i.e. don't even increment numBuckets)
if (effectiveMincount > 0 && countAcc.getCount(i) < effectiveMincount) {
continue;
}
numBuckets++;
if (bucketVals != null && bucketVals.size()<100) {
int ord = startTermIndex + i;
BytesRef br = lookupOrd(ord);
Object val = sf.getType().toObject(sf, br);
bucketVals.add(val);
}
if (bottom != null) {
if (sortAcc.compare(bottom.slot, i) * sortMul < 0) {
bottom.slot = i;
bottom = queue.updateTop();
}
} else if (lim > 0) {
// queue not full
Slot s = new Slot();
s.slot = i;
queue.add(s);
if (queue.size() >= maxsize) {
bottom = queue.top();
}
}
}
if (freq.numBuckets) {
if (!fcontext.isShard()) {
res.add("numBuckets", numBuckets);
} else {
SimpleOrderedMap<Object> map = new SimpleOrderedMap<>(2);
map.add("numBuckets", numBuckets);
map.add("vals", bucketVals);
res.add("numBuckets", map);
}
}
FacetDebugInfo fdebug = fcontext.getDebugInfo();
if (fdebug != null) fdebug.putInfoItem("numBuckets", (long) numBuckets);
// if we are deep paging, we don't have to order the highest "offset" counts.
int collectCount = Math.max(0, queue.size() - off);
assert collectCount <= lim;
int[] sortedSlots = new int[collectCount];
for (int i = collectCount - 1; i >= 0; i--) {
sortedSlots[i] = queue.pop().slot;
}
if (freq.allBuckets) {
SimpleOrderedMap<Object> allBuckets = new SimpleOrderedMap<>();
allBuckets.add("count", allBucketsAcc.getSpecialCount());
if (allBucketsAcc != null) {
allBucketsAcc.setValues(allBuckets, allBucketsSlot);
}
res.add("allBuckets", allBuckets);
}
ArrayList<SimpleOrderedMap<Object>> bucketList = new ArrayList<>(collectCount);
res.add("buckets", bucketList);
// TODO: do this with a callback instead?
boolean needFilter = deferredAggs != null || freq.getSubFacets().size() > 0;
for (int slotNum : sortedSlots) {
SimpleOrderedMap<Object> bucket = new SimpleOrderedMap<>();
// get the ord of the slot...
int ord = startTermIndex + slotNum;
BytesRef br = lookupOrd(ord);
Object val = sf.getType().toObject(sf, br);
bucket.add("val", val);
TermQuery filter = needFilter ? new TermQuery(new Term(sf.getName(), br)) : null;
fillBucket(bucket, countAcc.getCount(slotNum), slotNum, null, filter);
bucketList.add(bucket);
}
if (freq.missing) {
SimpleOrderedMap<Object> missingBucket = new SimpleOrderedMap<>();
fillBucket(missingBucket, getFieldMissingQuery(fcontext.searcher, freq.field), null);
res.add("missing", missingBucket);
}
return res;
}
}

View File

@ -34,23 +34,22 @@ import org.apache.solr.common.SolrException;
import org.apache.solr.schema.SchemaField;
import org.apache.solr.search.Filter;
class FacetFieldProcessorDV extends FacetFieldProcessorFCBase {
/**
* Grabs values from {@link DocValues}.
*/
class FacetFieldProcessorByArrayDV extends FacetFieldProcessorByArray {
static boolean unwrap_singleValued_multiDv = true; // only set to false for test coverage
boolean multiValuedField;
SortedSetDocValues si; // only used for term lookups (for both single and multi-valued)
MultiDocValues.OrdinalMap ordinalMap = null; // maps per-segment ords to global ords
public FacetFieldProcessorDV(FacetContext fcontext, FacetField freq, SchemaField sf) {
FacetFieldProcessorByArrayDV(FacetContext fcontext, FacetField freq, SchemaField sf) {
super(fcontext, freq, sf);
multiValuedField = sf.multiValued() || sf.getType().multiValuedFieldCache();
}
protected BytesRef lookupOrd(int ord) throws IOException {
return si.lookupOrd(ord);
}
@Override
protected void findStartAndEndOrds() throws IOException {
if (multiValuedField) {
si = FieldUtil.getSortedSetDocValues(fcontext.qcontext, sf, null);
@ -175,16 +174,9 @@ class FacetFieldProcessorDV extends FacetFieldProcessorFCBase {
reuse = null; // better GC
}
private int[] reuse;
private int[] getCountArr(int maxNeeded) {
if (reuse == null) {
// make the count array large enough for any segment
// FUTURE: (optionally) directly use the array of the CountAcc for an optimized index..
reuse = new int[(int) si.getValueCount() + 1];
} else {
Arrays.fill(reuse, 0, maxNeeded, 0);
}
return reuse;
@Override
protected BytesRef lookupOrd(int ord) throws IOException {
return si.lookupOrd(ord);
}
private void collectPerSeg(SortedDocValues singleDv, DocIdSetIterator disi, LongValues toGlobal) throws IOException {
@ -205,7 +197,6 @@ class FacetFieldProcessorDV extends FacetFieldProcessorFCBase {
}
}
private void collectPerSeg(SortedSetDocValues multiDv, DocIdSetIterator disi, LongValues toGlobal) throws IOException {
int segMax = (int)multiDv.getValueCount();
final int[] counts = getCountArr( segMax );
@ -229,6 +220,18 @@ class FacetFieldProcessorDV extends FacetFieldProcessorFCBase {
}
}
private int[] reuse;
private int[] getCountArr(int maxNeeded) {
if (reuse == null) {
// make the count array large enough for any segment
// FUTURE: (optionally) directly use the array of the CountAcc for an optimized index..
reuse = new int[(int) si.getValueCount() + 1];
} else {
Arrays.fill(reuse, 0, maxNeeded, 0);
}
return reuse;
}
private void collectDocs(SortedDocValues singleDv, DocIdSetIterator disi, LongValues toGlobal) throws IOException {
int doc;
while ((doc = disi.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {

View File

@ -0,0 +1,71 @@
/*
* 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.facet;
import java.io.IOException;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.UnicodeUtil;
import org.apache.solr.schema.SchemaField;
/** {@link UnInvertedField} implementation of field faceting.
* It's a top-level term cache. */
class FacetFieldProcessorByArrayUIF extends FacetFieldProcessorByArray {
UnInvertedField uif;
TermsEnum te;
FacetFieldProcessorByArrayUIF(FacetContext fcontext, FacetField freq, SchemaField sf) {
super(fcontext, freq, sf);
}
@Override
protected void findStartAndEndOrds() throws IOException {
uif = UnInvertedField.getUnInvertedField(freq.field, fcontext.searcher);
te = uif.getOrdTermsEnum( fcontext.searcher.getLeafReader() ); // "te" can be null
startTermIndex = 0;
endTermIndex = uif.numTerms(); // one past the end
if (prefixRef != null && te != null) {
if (te.seekCeil(prefixRef.get()) == TermsEnum.SeekStatus.END) {
startTermIndex = uif.numTerms();
} else {
startTermIndex = (int) te.ord();
}
prefixRef.append(UnicodeUtil.BIG_TERM);
if (te.seekCeil(prefixRef.get()) == TermsEnum.SeekStatus.END) {
endTermIndex = uif.numTerms();
} else {
endTermIndex = (int) te.ord();
}
}
nTerms = endTermIndex - startTermIndex;
}
@Override
protected void collectDocs() throws IOException {
uif.collectDocs(this);
}
@Override
protected BytesRef lookupOrd(int ord) throws IOException {
return uif.getTermValue(te, ord);
}
}

View File

@ -0,0 +1,356 @@
/*
* 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.facet;
import java.io.Closeable;
import java.io.IOException;
import java.util.Iterator;
import java.util.List;
import org.apache.lucene.index.Fields;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.MultiPostingsEnum;
import org.apache.lucene.index.PostingsEnum;
import org.apache.lucene.index.Term;
import org.apache.lucene.index.Terms;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.search.DocIdSetIterator;
import org.apache.lucene.search.TermQuery;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.StringHelper;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.util.SimpleOrderedMap;
import org.apache.solr.schema.SchemaField;
import org.apache.solr.schema.TrieField;
import org.apache.solr.search.DocSet;
import org.apache.solr.search.HashDocSet;
import org.apache.solr.search.SolrIndexSearcher;
import org.apache.solr.search.SortedIntDocSet;
/**
* Enumerates indexed terms in order in a streaming fashion.
* It's able to stream since no data needs to be accumulated so long as it's index order.
*/
class FacetFieldProcessorByEnumTermsStream extends FacetFieldProcessor implements Closeable {
long bucketsToSkip;
long bucketsReturned;
boolean closed;
boolean countOnly;
boolean hasSubFacets; // true if there are subfacets
int minDfFilterCache;
DocSet docs;
DocSet fastForRandomSet;
TermsEnum termsEnum = null;
SolrIndexSearcher.DocsEnumState deState = null;
PostingsEnum postingsEnum;
BytesRef startTermBytes;
BytesRef term;
LeafReaderContext[] leaves;
FacetFieldProcessorByEnumTermsStream(FacetContext fcontext, FacetField freq, SchemaField sf) {
super(fcontext, freq, sf);
}
@Override
public void close() throws IOException {
if (!closed) {
closed = true;
// fcontext.base.decref(); // OFF-HEAP
}
}
@Override
public void process() throws IOException {
super.process();
// We need to keep the fcontext open after processing is done (since we will be streaming in the response writer).
// But if the connection is broken, we want to clean up.
// fcontext.base.incref(); // OFF-HEAP
fcontext.qcontext.addCloseHook(this);
setup();
response = new SimpleOrderedMap<>();
response.add("buckets", new Iterator() {
boolean retrieveNext = true;
Object val;
@Override
public boolean hasNext() {
if (retrieveNext) {
val = nextBucket();
}
retrieveNext = false;
return val != null;
}
@Override
public Object next() {
if (retrieveNext) {
val = nextBucket();
}
retrieveNext = true;
if (val == null) {
// Last value, so clean up. In the case that we are doing streaming facets within streaming facets,
// the number of close hooks could grow very large, so we want to remove ourselves.
boolean removed = fcontext.qcontext.removeCloseHook(FacetFieldProcessorByEnumTermsStream.this);
assert removed;
try {
close();
} catch (IOException e) {
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Error during facet streaming close", e);
}
}
return val;
}
@Override
public void remove() {
throw new UnsupportedOperationException();
}
});
}
private void setup() throws IOException {
countOnly = freq.facetStats.size() == 0 || freq.facetStats.values().iterator().next() instanceof CountAgg;
hasSubFacets = freq.subFacets.size() > 0;
bucketsToSkip = freq.offset;
createAccs(-1, 1);
// Minimum term docFreq in order to use the filterCache for that term.
if (freq.cacheDf == -1) { // -1 means never cache
minDfFilterCache = Integer.MAX_VALUE;
} else if (freq.cacheDf == 0) { // default; compute as fraction of maxDoc
minDfFilterCache = Math.max(fcontext.searcher.maxDoc() >> 4, 3); // (minimum of 3 is for test coverage purposes)
} else {
minDfFilterCache = freq.cacheDf;
}
docs = fcontext.base;
fastForRandomSet = null;
if (freq.prefix != null) {
String indexedPrefix = sf.getType().toInternal(freq.prefix);
startTermBytes = new BytesRef(indexedPrefix);
} else if (sf.getType().getNumericType() != null) {
String triePrefix = TrieField.getMainValuePrefix(sf.getType());
if (triePrefix != null) {
startTermBytes = new BytesRef(triePrefix);
}
}
Fields fields = fcontext.searcher.getLeafReader().fields();
Terms terms = fields == null ? null : fields.terms(sf.getName());
termsEnum = null;
deState = null;
term = null;
if (terms != null) {
termsEnum = terms.iterator();
// TODO: OPT: if seek(ord) is supported for this termsEnum, then we could use it for
// facet.offset when sorting by index order.
if (startTermBytes != null) {
if (termsEnum.seekCeil(startTermBytes) == TermsEnum.SeekStatus.END) {
termsEnum = null;
} else {
term = termsEnum.term();
}
} else {
// position termsEnum on first term
term = termsEnum.next();
}
}
List<LeafReaderContext> leafList = fcontext.searcher.getTopReaderContext().leaves();
leaves = leafList.toArray( new LeafReaderContext[ leafList.size() ]);
}
private SimpleOrderedMap<Object> nextBucket() {
try {
return _nextBucket();
} catch (Exception e) {
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Error during facet streaming", e);
}
}
private SimpleOrderedMap<Object> _nextBucket() throws IOException {
DocSet termSet = null;
try {
while (term != null) {
if (startTermBytes != null && !StringHelper.startsWith(term, startTermBytes)) {
break;
}
int df = termsEnum.docFreq();
if (df < effectiveMincount) {
term = termsEnum.next();
continue;
}
if (termSet != null) {
// termSet.decref(); // OFF-HEAP
termSet = null;
}
int c = 0;
if (hasSubFacets || df >= minDfFilterCache) {
// use the filter cache
if (deState == null) {
deState = new SolrIndexSearcher.DocsEnumState();
deState.fieldName = sf.getName();
deState.liveDocs = fcontext.searcher.getLeafReader().getLiveDocs();
deState.termsEnum = termsEnum;
deState.postingsEnum = postingsEnum;
deState.minSetSizeCached = minDfFilterCache;
}
if (hasSubFacets || !countOnly) {
DocSet termsAll = fcontext.searcher.getDocSet(deState);
termSet = docs.intersection(termsAll);
// termsAll.decref(); // OFF-HEAP
c = termSet.size();
} else {
c = fcontext.searcher.numDocs(docs, deState);
}
postingsEnum = deState.postingsEnum;
resetStats();
if (!countOnly) {
collect(termSet, 0);
}
} else {
// We don't need the docset here (meaning no sub-facets).
// if countOnly, then we are calculating some other stats...
resetStats();
// lazy convert to fastForRandomSet
if (fastForRandomSet == null) {
fastForRandomSet = docs;
if (docs instanceof SortedIntDocSet) { // OFF-HEAP todo: also check for native version
SortedIntDocSet sset = (SortedIntDocSet) docs;
fastForRandomSet = new HashDocSet(sset.getDocs(), 0, sset.size());
}
}
// iterate over TermDocs to calculate the intersection
postingsEnum = termsEnum.postings(postingsEnum, PostingsEnum.NONE);
if (postingsEnum instanceof MultiPostingsEnum) {
MultiPostingsEnum.EnumWithSlice[] subs = ((MultiPostingsEnum) postingsEnum).getSubs();
int numSubs = ((MultiPostingsEnum) postingsEnum).getNumSubs();
for (int subindex = 0; subindex < numSubs; subindex++) {
MultiPostingsEnum.EnumWithSlice sub = subs[subindex];
if (sub.postingsEnum == null) continue;
int base = sub.slice.start;
int docid;
if (countOnly) {
while ((docid = sub.postingsEnum.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
if (fastForRandomSet.exists(docid + base)) c++;
}
} else {
setNextReader(leaves[sub.slice.readerIndex]);
while ((docid = sub.postingsEnum.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
if (fastForRandomSet.exists(docid + base)) {
c++;
collect(docid, 0);
}
}
}
}
} else {
int docid;
if (countOnly) {
while ((docid = postingsEnum.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
if (fastForRandomSet.exists(docid)) c++;
}
} else {
setNextReader(leaves[0]);
while ((docid = postingsEnum.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
if (fastForRandomSet.exists(docid)) {
c++;
collect(docid, 0);
}
}
}
}
}
if (c < effectiveMincount) {
term = termsEnum.next();
continue;
}
// handle offset and limit
if (bucketsToSkip > 0) {
bucketsToSkip--;
term = termsEnum.next();
continue;
}
if (freq.limit >= 0 && ++bucketsReturned > freq.limit) {
return null;
}
// set count in case other stats depend on it
countAcc.incrementCount(0, c);
// OK, we have a good bucket to return... first get bucket value before moving to next term
Object bucketVal = sf.getType().toObject(sf, term);
TermQuery bucketQuery = hasSubFacets ? new TermQuery(new Term(freq.field, term)) : null;
term = termsEnum.next();
SimpleOrderedMap<Object> bucket = new SimpleOrderedMap<>();
bucket.add("val", bucketVal);
addStats(bucket, 0);
if (hasSubFacets) {
processSubs(bucket, bucketQuery, termSet);
}
// TODO... termSet needs to stick around for streaming sub-facets?
return bucket;
}
} finally {
if (termSet != null) {
// termSet.decref(); // OFF-HEAP
termSet = null;
}
}
// end of the iteration
return null;
}
}

View File

@ -32,10 +32,15 @@ import org.apache.solr.common.util.SimpleOrderedMap;
import org.apache.solr.schema.SchemaField;
import org.apache.solr.search.DocIterator;
class FacetFieldProcessorNumeric extends FacetFieldProcessor {
/**
* Facets numbers into a hash table.
* It currently only works with {@link NumericDocValues} (single-valued).
*/
class FacetFieldProcessorByHashNumeric extends FacetFieldProcessor {
static int MAXIMUM_STARTING_TABLE_SIZE=1024; // must be a power of two, non-final to support setting by tests
static class LongCounts {
/** a hash table with long keys (what we're counting) and integer values (counts) */
private static class LongCounts {
static final float LOAD_FACTOR = 0.7f;
@ -55,7 +60,7 @@ class FacetFieldProcessorNumeric extends FacetFieldProcessor {
}
/** Current number of slots in the hash table */
public int numSlots() {
int numSlots() {
return vals.length;
}
@ -130,69 +135,22 @@ class FacetFieldProcessorNumeric extends FacetFieldProcessor {
}
int allBucketsSlot = -1;
FacetFieldProcessorNumeric(FacetContext fcontext, FacetField freq, SchemaField sf) {
FacetFieldProcessorByHashNumeric(FacetContext fcontext, FacetField freq, SchemaField sf) {
super(fcontext, freq, sf);
}
int allBucketsSlot = -1;
@Override
public void process() throws IOException {
super.process();
response = calcFacets();
}
private void doRehash(LongCounts table) {
if (collectAcc == null && allBucketsAcc == null) return;
// Our "count" acc is backed by the hash table and will already be rehashed
// otherAccs don't need to be rehashed
int newTableSize = table.numSlots();
int numSlots = newTableSize;
final int oldAllBucketsSlot = allBucketsSlot;
if (oldAllBucketsSlot >= 0) {
allBucketsSlot = numSlots++;
}
final int finalNumSlots = numSlots;
final int[] mapping = table.oldToNewMapping;
SlotAcc.Resizer resizer = new SlotAcc.Resizer() {
@Override
public int getNewSize() {
return finalNumSlots;
}
@Override
public int getNewSlot(int oldSlot) {
if (oldSlot < mapping.length) {
return mapping[oldSlot];
}
if (oldSlot == oldAllBucketsSlot) {
return allBucketsSlot;
}
return -1;
}
};
// NOTE: resizing isn't strictly necessary for missing/allBuckets... we could just set the new slot directly
if (collectAcc != null) {
collectAcc.resize(resizer);
}
if (allBucketsAcc != null) {
allBucketsAcc.resize(resizer);
}
}
public SimpleOrderedMap<Object> calcFacets() throws IOException {
private SimpleOrderedMap<Object> calcFacets() throws IOException {
final FacetRangeProcessor.Calc calc = FacetRangeProcessor.getNumericCalc(sf);
// TODO: it would be really nice to know the number of unique values!!!!
int possibleValues = fcontext.base.size();
@ -212,7 +170,6 @@ class FacetFieldProcessorNumeric extends FacetFieldProcessor {
int numMissing = 0;
if (freq.allBuckets) {
allBucketsSlot = numSlots++;
}
@ -325,7 +282,6 @@ class FacetFieldProcessorNumeric extends FacetFieldProcessor {
}
}
//
// collection done, time to find the top slots
//
@ -333,7 +289,7 @@ class FacetFieldProcessorNumeric extends FacetFieldProcessor {
int numBuckets = 0;
List<Object> bucketVals = null;
if (freq.numBuckets && fcontext.isShard()) {
bucketVals = new ArrayList(100);
bucketVals = new ArrayList<>(100);
}
int off = fcontext.isShard() ? 0 : (int) freq.offset;
@ -378,13 +334,12 @@ class FacetFieldProcessorNumeric extends FacetFieldProcessor {
bottom = queue.insertWithOverflow(bottom);
}
SimpleOrderedMap res = new SimpleOrderedMap();
SimpleOrderedMap<Object> res = new SimpleOrderedMap<>();
if (freq.numBuckets) {
if (!fcontext.isShard()) {
res.add("numBuckets", numBuckets);
} else {
SimpleOrderedMap map = new SimpleOrderedMap(2);
SimpleOrderedMap<Object> map = new SimpleOrderedMap<>(2);
map.add("numBuckets", numBuckets);
map.add("vals", bucketVals);
res.add("numBuckets", map);
@ -392,7 +347,7 @@ class FacetFieldProcessorNumeric extends FacetFieldProcessor {
}
FacetDebugInfo fdebug = fcontext.getDebugInfo();
if (fdebug != null) fdebug.putInfoItem("numBuckets", new Long(numBuckets));
if (fdebug != null) fdebug.putInfoItem("numBuckets", (long) numBuckets);
if (freq.allBuckets) {
SimpleOrderedMap<Object> allBuckets = new SimpleOrderedMap<>();
@ -419,7 +374,7 @@ class FacetFieldProcessorNumeric extends FacetFieldProcessor {
sortedSlots[i] = queue.pop().slot;
}
ArrayList bucketList = new ArrayList(collectCount);
ArrayList<SimpleOrderedMap> bucketList = new ArrayList<>(collectCount);
res.add("buckets", bucketList);
boolean needFilter = deferredAggs != null || freq.getSubFacets().size() > 0;
@ -436,8 +391,49 @@ class FacetFieldProcessorNumeric extends FacetFieldProcessor {
bucketList.add(bucket);
}
return res;
}
private void doRehash(LongCounts table) {
if (collectAcc == null && allBucketsAcc == null) return;
// Our "count" acc is backed by the hash table and will already be rehashed
// otherAccs don't need to be rehashed
int newTableSize = table.numSlots();
int numSlots = newTableSize;
final int oldAllBucketsSlot = allBucketsSlot;
if (oldAllBucketsSlot >= 0) {
allBucketsSlot = numSlots++;
}
final int finalNumSlots = numSlots;
final int[] mapping = table.oldToNewMapping;
SlotAcc.Resizer resizer = new SlotAcc.Resizer() {
@Override
public int getNewSize() {
return finalNumSlots;
}
@Override
public int getNewSlot(int oldSlot) {
if (oldSlot < mapping.length) {
return mapping[oldSlot];
}
if (oldSlot == oldAllBucketsSlot) {
return allBucketsSlot;
}
return -1;
}
};
// NOTE: resizing isn't strictly necessary for missing/allBuckets... we could just set the new slot directly
if (collectAcc != null) {
collectAcc.resize(resizer);
}
if (allBucketsAcc != null) {
allBucketsAcc.resize(resizer);
}
}
}

View File

@ -45,27 +45,18 @@ import org.apache.solr.search.SolrIndexSearcher;
import org.apache.solr.search.SyntaxError;
import org.apache.solr.util.RTimer;
public class FacetProcessor<FacetRequestT extends FacetRequest> {
protected SimpleOrderedMap<Object> response;
protected FacetContext fcontext;
protected FacetRequestT freq;
public abstract class FacetProcessor<FacetRequestT extends FacetRequest> {
SimpleOrderedMap<Object> response;
FacetContext fcontext;
FacetRequestT freq;
LinkedHashMap<String,SlotAcc> accMap;
protected SlotAcc[] accs;
protected CountSlotAcc countAcc;
SlotAcc[] accs;
CountSlotAcc countAcc;
FacetProcessor(FacetContext fcontext, FacetRequestT freq) {
this.fcontext = fcontext;
this.freq = freq;
}
public void process() throws IOException {
handleDomainChanges();
}
/** factory method for invoking json facet framework as whole */
public static FacetProcessor<?> createProcessor(SolrQueryRequest req,
Map<String, Object> params, DocSet docs){
public static FacetProcessor<?> createProcessor(SolrQueryRequest req,
Map<String, Object> params, DocSet docs){
FacetParser parser = new FacetTopParser(req);
FacetRequest facetRequest = null;
try {
@ -83,39 +74,25 @@ public class FacetProcessor<FacetRequestT extends FacetRequest> {
return facetRequest.createFacetProcessor(fcontext);
}
protected void handleDomainChanges() throws IOException {
FacetProcessor(FacetContext fcontext, FacetRequestT freq) {
this.fcontext = fcontext;
this.freq = freq;
}
public Object getResponse() {
return response;
}
public void process() throws IOException {
handleDomainChanges();
}
private void handleDomainChanges() throws IOException {
if (freq.domain == null) return;
handleFilterExclusions();
handleBlockJoin();
}
private void handleBlockJoin() throws IOException {
if (!(freq.domain.toChildren || freq.domain.toParent)) return;
// TODO: avoid query parsing per-bucket somehow...
String parentStr = freq.domain.parents;
Query parentQuery;
try {
QParser parser = QParser.getParser(parentStr, fcontext.req);
parentQuery = parser.getQuery();
} catch (SyntaxError err) {
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Error parsing block join parent specification: " + parentStr);
}
BitDocSet parents = fcontext.searcher.getDocSetBits(parentQuery);
DocSet input = fcontext.base;
DocSet result;
if (freq.domain.toChildren) {
DocSet filt = fcontext.searcher.getDocSetBits( new MatchAllDocsQuery() );
result = BlockJoin.toChildren(input, parents, filt, fcontext.qcontext);
} else {
result = BlockJoin.toParents(input, parents, fcontext.qcontext);
}
fcontext.base = result;
}
private void handleFilterExclusions() throws IOException {
List<String> excludeTags = freq.domain.excludeTags;
@ -177,11 +154,44 @@ public class FacetProcessor<FacetRequestT extends FacetRequest> {
fcontext.base = fcontext.searcher.getDocSet(qlist);
}
private void handleBlockJoin() throws IOException {
if (!(freq.domain.toChildren || freq.domain.toParent)) return;
public Object getResponse() {
return null;
// TODO: avoid query parsing per-bucket somehow...
String parentStr = freq.domain.parents;
Query parentQuery;
try {
QParser parser = QParser.getParser(parentStr, fcontext.req);
parentQuery = parser.getQuery();
} catch (SyntaxError err) {
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Error parsing block join parent specification: " + parentStr);
}
BitDocSet parents = fcontext.searcher.getDocSetBits(parentQuery);
DocSet input = fcontext.base;
DocSet result;
if (freq.domain.toChildren) {
DocSet filt = fcontext.searcher.getDocSetBits( new MatchAllDocsQuery() );
result = BlockJoin.toChildren(input, parents, filt, fcontext.qcontext);
} else {
result = BlockJoin.toParents(input, parents, fcontext.qcontext);
}
fcontext.base = result;
}
protected void processStats(SimpleOrderedMap<Object> bucket, DocSet docs, int docCount) throws IOException {
if (docCount == 0 && !freq.processEmpty || freq.getFacetStats().size() == 0) {
bucket.add("count", docCount);
return;
}
createAccs(docCount, 1);
int collected = collect(docs, 0);
countAcc.incrementCount(0, collected);
assert collected == docCount;
addStats(bucket, 0);
}
protected void createAccs(int docCount, int slotCount) throws IOException {
accMap = new LinkedHashMap<>();
@ -198,7 +208,6 @@ public class FacetProcessor<FacetRequestT extends FacetRequest> {
accMap.put(acc.key, acc);
}
accs = new SlotAcc[accMap.size()];
int i=0;
for (SlotAcc acc : accMap.values()) {
@ -206,63 +215,14 @@ public class FacetProcessor<FacetRequestT extends FacetRequest> {
}
}
protected void resetStats() {
// note: only called by enum/stream prior to collect
void resetStats() {
countAcc.reset();
for (SlotAcc acc : accs) {
acc.reset();
}
}
protected void processStats(SimpleOrderedMap<Object> bucket, DocSet docs, int docCount) throws IOException {
if (docCount == 0 && !freq.processEmpty || freq.getFacetStats().size() == 0) {
bucket.add("count", docCount);
return;
}
createAccs(docCount, 1);
int collected = collect(docs, 0);
countAcc.incrementCount(0, collected);
assert collected == docCount;
addStats(bucket, 0);
}
protected void processSubs(SimpleOrderedMap<Object> response, Query filter, DocSet domain) throws IOException {
// TODO: what if a zero bucket has a sub-facet with an exclusion that would yield results?
// should we check for domain-altering exclusions, or even ask the sub-facet for
// it's domain and then only skip it if it's 0?
if (domain == null || domain.size() == 0 && !freq.processEmpty) {
return;
}
for (Map.Entry<String,FacetRequest> sub : freq.getSubFacets().entrySet()) {
// make a new context for each sub-facet since they can change the domain
FacetContext subContext = fcontext.sub(filter, domain);
FacetProcessor subProcessor = sub.getValue().createFacetProcessor(subContext);
if (fcontext.getDebugInfo() != null) { // if fcontext.debugInfo != null, it means rb.debug() == true
FacetDebugInfo fdebug = new FacetDebugInfo();
subContext.setDebugInfo(fdebug);
fcontext.getDebugInfo().addChild(fdebug);
fdebug.setReqDescription(sub.getValue().getFacetDescription());
fdebug.setProcessor(subProcessor.getClass().getSimpleName());
if (subContext.filter != null) fdebug.setFilter(subContext.filter.toString());
final RTimer timer = new RTimer();
subProcessor.process();
long timeElapsed = (long) timer.getTime();
fdebug.setElapse(timeElapsed);
fdebug.putInfoItem("domainSize", (long)subContext.base.size());
} else {
subProcessor.process();
}
response.add( sub.getKey(), subProcessor.getResponse() );
}
}
int collect(DocSet docs, int slot) throws IOException {
int count = 0;
SolrIndexSearcher searcher = fcontext.searcher;
@ -310,7 +270,6 @@ public class FacetProcessor<FacetRequestT extends FacetRequest> {
}
}
void addStats(SimpleOrderedMap<Object> target, int slotNum) throws IOException {
int count = countAcc.getCount(slotNum);
target.add("count", count);
@ -321,8 +280,7 @@ public class FacetProcessor<FacetRequestT extends FacetRequest> {
}
}
public void fillBucket(SimpleOrderedMap<Object> bucket, Query q, DocSet result) throws IOException {
void fillBucket(SimpleOrderedMap<Object> bucket, Query q, DocSet result) throws IOException {
boolean needDocSet = freq.getFacetStats().size() > 0 || freq.getSubFacets().size() > 0;
// TODO: always collect counts or not???
@ -348,7 +306,7 @@ public class FacetProcessor<FacetRequestT extends FacetRequest> {
}
try {
processStats(bucket, result, (int) count);
processStats(bucket, result, count);
processSubs(bucket, q, result);
} finally {
if (result != null) {
@ -358,7 +316,44 @@ public class FacetProcessor<FacetRequestT extends FacetRequest> {
}
}
public static DocSet getFieldMissing(SolrIndexSearcher searcher, DocSet docs, String fieldName) throws IOException {
void processSubs(SimpleOrderedMap<Object> response, Query filter, DocSet domain) throws IOException {
// TODO: what if a zero bucket has a sub-facet with an exclusion that would yield results?
// should we check for domain-altering exclusions, or even ask the sub-facet for
// it's domain and then only skip it if it's 0?
if (domain == null || domain.size() == 0 && !freq.processEmpty) {
return;
}
for (Map.Entry<String,FacetRequest> sub : freq.getSubFacets().entrySet()) {
// make a new context for each sub-facet since they can change the domain
FacetContext subContext = fcontext.sub(filter, domain);
FacetProcessor subProcessor = sub.getValue().createFacetProcessor(subContext);
if (fcontext.getDebugInfo() != null) { // if fcontext.debugInfo != null, it means rb.debug() == true
FacetDebugInfo fdebug = new FacetDebugInfo();
subContext.setDebugInfo(fdebug);
fcontext.getDebugInfo().addChild(fdebug);
fdebug.setReqDescription(sub.getValue().getFacetDescription());
fdebug.setProcessor(subProcessor.getClass().getSimpleName());
if (subContext.filter != null) fdebug.setFilter(subContext.filter.toString());
final RTimer timer = new RTimer();
subProcessor.process();
long timeElapsed = (long) timer.getTime();
fdebug.setElapse(timeElapsed);
fdebug.putInfoItem("domainSize", (long)subContext.base.size());
} else {
subProcessor.process();
}
response.add( sub.getKey(), subProcessor.getResponse() );
}
}
@SuppressWarnings("unused")
static DocSet getFieldMissing(SolrIndexSearcher searcher, DocSet docs, String fieldName) throws IOException {
SchemaField sf = searcher.getSchema().getField(fieldName);
DocSet hasVal = searcher.getDocSet(sf.getType().getRangeQuery(null, sf, null, null, false, false));
DocSet answer = docs.andNot(hasVal);
@ -366,7 +361,7 @@ public class FacetProcessor<FacetRequestT extends FacetRequest> {
return answer;
}
public static Query getFieldMissingQuery(SolrIndexSearcher searcher, String fieldName) throws IOException {
static Query getFieldMissingQuery(SolrIndexSearcher searcher, String fieldName) throws IOException {
SchemaField sf = searcher.getSchema().getField(fieldName);
Query hasVal = sf.getType().getRangeQuery(null, sf, null, null, false, false);
BooleanQuery.Builder noVal = new BooleanQuery.Builder();

View File

@ -53,11 +53,6 @@ class FacetQueryProcessor extends FacetProcessor<FacetQuery> {
super(fcontext, freq);
}
@Override
public Object getResponse() {
return response;
}
@Override
public void process() throws IOException {
super.process();

View File

@ -93,11 +93,6 @@ class FacetRangeProcessor extends FacetProcessor<FacetRange> {
response = getRangeCounts();
}
@Override
public Object getResponse() {
return response;
}
private static class Range {
Object label;
Comparable low;

View File

@ -305,7 +305,7 @@ public class UnInvertedField extends DocTermOrds {
private void getCounts(FacetFieldProcessorUIF processor, CountSlotAcc counts) throws IOException {
private void getCounts(FacetFieldProcessorByArrayUIF processor, CountSlotAcc counts) throws IOException {
DocSet docs = processor.fcontext.base;
int baseSize = docs.size();
int maxDoc = searcher.maxDoc();
@ -397,7 +397,7 @@ public class UnInvertedField extends DocTermOrds {
public void collectDocs(FacetFieldProcessorUIF processor) throws IOException {
public void collectDocs(FacetFieldProcessorByArrayUIF processor) throws IOException {
if (processor.collectAcc==null && processor.allBucketsAcc == null && processor.startTermIndex == 0 && processor.endTermIndex >= numTermsInField) {
getCounts(processor, processor.countAcc);
return;
@ -408,7 +408,7 @@ public class UnInvertedField extends DocTermOrds {
// called from FieldFacetProcessor
// TODO: do a callback version that can be specialized!
public void collectDocsGeneric(FacetFieldProcessorUIF processor) throws IOException {
public void collectDocsGeneric(FacetFieldProcessorByArrayUIF processor) throws IOException {
use.incrementAndGet();
int startTermIndex = processor.startTermIndex;

View File

@ -0,0 +1,75 @@
/*
* 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.cloud;
import java.lang.invoke.MethodHandles;
import java.util.ArrayList;
import java.util.Collections;
import java.util.Set;
import org.apache.solr.client.solrj.impl.CloudSolrClient;
import org.apache.solr.client.solrj.request.CollectionAdminRequest;
import org.apache.solr.client.solrj.response.RequestStatusState;
import org.apache.solr.common.util.StrUtils;
import org.junit.BeforeClass;
import org.junit.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
public class DeleteNodeTest extends SolrCloudTestCase {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
@BeforeClass
public static void setupCluster() throws Exception {
configureCluster(6)
.addConfig("conf1", TEST_PATH().resolve("configsets").resolve("cloud-dynamic").resolve("conf"))
.configure();
}
protected String getSolrXml() {
return "solr.xml";
}
@Test
public void test() throws Exception {
cluster.waitForAllNodes(5000);
CloudSolrClient cloudClient = cluster.getSolrClient();
String coll = "deletenodetest_coll";
Set<String> liveNodes = cloudClient.getZkStateReader().getClusterState().getLiveNodes();
ArrayList<String> l = new ArrayList<>(liveNodes);
Collections.shuffle(l, random());
CollectionAdminRequest.Create create = CollectionAdminRequest.createCollection(coll, "conf1", 5, 2);
create.setCreateNodeSet(StrUtils.join(l, ',')).setMaxShardsPerNode(3);
cloudClient.request(create);
String node2bdecommissioned = l.get(0);
new CollectionAdminRequest.DeleteNode(node2bdecommissioned).processAsync("003", cloudClient);
CollectionAdminRequest.RequestStatus requestStatus = CollectionAdminRequest.requestStatus("003");
boolean success = false;
for (int i = 0; i < 200; i++) {
CollectionAdminRequest.RequestStatusResponse rsp = requestStatus.process(cloudClient);
if (rsp.getRequestStatus() == RequestStatusState.COMPLETED) {
success = true;
break;
}
assertFalse(rsp.getRequestStatus() == RequestStatusState.FAILED);
Thread.sleep(50);
}
assertTrue(success);
}
}

View File

@ -0,0 +1,104 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.solr.cloud;
import java.lang.invoke.MethodHandles;
import java.util.ArrayList;
import java.util.Collections;
import java.util.Set;
import org.apache.solr.client.solrj.impl.CloudSolrClient;
import org.apache.solr.client.solrj.impl.HttpSolrClient;
import org.apache.solr.client.solrj.request.CollectionAdminRequest;
import org.apache.solr.client.solrj.request.CoreAdminRequest;
import org.apache.solr.client.solrj.response.CoreAdminResponse;
import org.apache.solr.client.solrj.response.RequestStatusState;
import org.apache.solr.common.util.StrUtils;
import org.junit.BeforeClass;
import org.junit.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
public class ReplaceNodeTest extends SolrCloudTestCase {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
@BeforeClass
public static void setupCluster() throws Exception {
configureCluster(6)
.addConfig("conf1", TEST_PATH().resolve("configsets").resolve("cloud-dynamic").resolve("conf"))
.configure();
}
protected String getSolrXml() {
return "solr.xml";
}
@Test
public void test() throws Exception {
cluster.waitForAllNodes(5000);
String coll = "replacenodetest_coll";
log.info("total_jettys: " + cluster.getJettySolrRunners().size());
CloudSolrClient cloudClient = cluster.getSolrClient();
Set<String> liveNodes = cloudClient.getZkStateReader().getClusterState().getLiveNodes();
ArrayList<String> l = new ArrayList<>(liveNodes);
Collections.shuffle(l, random());
String emptyNode = l.remove(0);
String node2bdecommissioned = l.get(0);
CollectionAdminRequest.Create create = CollectionAdminRequest.createCollection(coll, "conf1", 5, 2);
create.setCreateNodeSet(StrUtils.join(l, ',')).setMaxShardsPerNode(3);
cloudClient.request(create);
log.info("excluded_node : {} ", emptyNode);
new CollectionAdminRequest.ReplaceNode(node2bdecommissioned, emptyNode).processAsync("000", cloudClient);
CollectionAdminRequest.RequestStatus requestStatus = CollectionAdminRequest.requestStatus("000");
boolean success = false;
for (int i = 0; i < 200; i++) {
CollectionAdminRequest.RequestStatusResponse rsp = requestStatus.process(cloudClient);
if (rsp.getRequestStatus() == RequestStatusState.COMPLETED) {
success = true;
break;
}
assertFalse(rsp.getRequestStatus() == RequestStatusState.FAILED);
Thread.sleep(50);
}
assertTrue(success);
try (HttpSolrClient coreclient = getHttpSolrClient(cloudClient.getZkStateReader().getBaseUrlForNodeName(node2bdecommissioned))) {
CoreAdminResponse status = CoreAdminRequest.getStatus(null, coreclient);
assertTrue(status.getCoreStatus().size() == 0);
}
//let's do it back
new CollectionAdminRequest.ReplaceNode(emptyNode, node2bdecommissioned).setParallel(Boolean.TRUE).processAsync("001", cloudClient);
requestStatus = CollectionAdminRequest.requestStatus("001");
for (int i = 0; i < 200; i++) {
CollectionAdminRequest.RequestStatusResponse rsp = requestStatus.process(cloudClient);
if (rsp.getRequestStatus() == RequestStatusState.COMPLETED) {
success = true;
break;
}
assertFalse(rsp.getRequestStatus() == RequestStatusState.FAILED);
Thread.sleep(50);
}
assertTrue(success);
try (HttpSolrClient coreclient = getHttpSolrClient(cloudClient.getZkStateReader().getBaseUrlForNodeName(emptyNode))) {
CoreAdminResponse status = CoreAdminRequest.getStatus(null, coreclient);
assertTrue(status.getCoreStatus().size() == 0);
}
}
}

View File

@ -47,8 +47,8 @@ public class TestJsonFacets extends SolrTestCaseHS {
@BeforeClass
public static void beforeTests() throws Exception {
JSONTestUtil.failRepeatedKeys = true;
origTableSize = FacetFieldProcessorNumeric.MAXIMUM_STARTING_TABLE_SIZE;
FacetFieldProcessorNumeric.MAXIMUM_STARTING_TABLE_SIZE=2; // stress test resizing
origTableSize = FacetFieldProcessorByHashNumeric.MAXIMUM_STARTING_TABLE_SIZE;
FacetFieldProcessorByHashNumeric.MAXIMUM_STARTING_TABLE_SIZE=2; // stress test resizing
initCore("solrconfig-tlog.xml","schema_latest.xml");
}
@ -61,7 +61,7 @@ public class TestJsonFacets extends SolrTestCaseHS {
@AfterClass
public static void afterTests() throws Exception {
JSONTestUtil.failRepeatedKeys = false;
FacetFieldProcessorNumeric.MAXIMUM_STARTING_TABLE_SIZE=origTableSize;
FacetFieldProcessorByHashNumeric.MAXIMUM_STARTING_TABLE_SIZE=origTableSize;
if (servers != null) {
servers.stop();
servers = null;
@ -349,11 +349,11 @@ public class TestJsonFacets extends SolrTestCaseHS {
doStatsTemplated(client, params(p, "rows","0", "noexist","noexist_sd", "cat_s","cat_sd", "where_s","where_sd", "num_d","num_dd", "num_i","num_id", "num_is","num_lds", "num_fs","num_dds", "super_s","super_sd", "val_b","val_b", "date","date_dtd", "sparse_s","sparse_sd" ,"multi_ss","multi_sds") );
// multi-valued docvalues
FacetFieldProcessorDV.unwrap_singleValued_multiDv = false; // better multi-valued coverage
FacetFieldProcessorByArrayDV.unwrap_singleValued_multiDv = false; // better multi-valued coverage
doStatsTemplated(client, params(p, "rows","0", "noexist","noexist_sds", "cat_s","cat_sds", "where_s","where_sds", "num_d","num_d", "num_i","num_i", "num_is","num_ids", "num_fs","num_fds", "super_s","super_sds", "val_b","val_b", "date","date_dtds", "sparse_s","sparse_sds" ,"multi_ss","multi_sds") );
// multi-valued docvalues
FacetFieldProcessorDV.unwrap_singleValued_multiDv = true;
FacetFieldProcessorByArrayDV.unwrap_singleValued_multiDv = true;
doStatsTemplated(client, params(p, "rows","0", "noexist","noexist_sds", "cat_s","cat_sds", "where_s","where_sds", "num_d","num_d", "num_i","num_i", "num_is","num_ids", "num_fs","num_fds", "super_s","super_sds", "val_b","val_b", "date","date_dtds", "sparse_s","sparse_sds" ,"multi_ss","multi_sds") );
}

View File

@ -1,6 +1,6 @@
# System Requirements
Apache Solr runs of Java 8 or greater.
Apache Solr runs on Java 8 or greater.
It is also recommended to always use the latest update version of your
Java VM, because bugs may affect Solr. An overview of known JVM bugs

View File

@ -112,7 +112,7 @@ public abstract class CollectionAdminRequest<T extends CollectionAdminResponse>
* @deprecated Use {@link #processAsync(String, SolrClient)} or {@link #processAsync(SolrClient)}
*/
@Deprecated
public abstract AsyncCollectionAdminRequest setAsyncId(String id);
public AsyncCollectionAdminRequest setAsyncId(String id){return this;};
/**
* Process this request asynchronously, generating and returning a request id
@ -491,6 +491,56 @@ public abstract class CollectionAdminRequest<T extends CollectionAdminResponse>
}
}
public static class DeleteNode extends AsyncCollectionAdminRequest {
String node;
/**
* @param node The node to be deleted
*/
public DeleteNode(String node) {
super(CollectionAction.DELETENODE);
this.node = node;
}
@Override
public SolrParams getParams() {
ModifiableSolrParams params = (ModifiableSolrParams) super.getParams();
params.set("node", node);
return params;
}
}
public static class ReplaceNode extends AsyncCollectionAdminRequest {
String source, target;
Boolean parallel;
/**
* @param source node to be cleaned up
* @param target node where the new replicas are to be created
*/
public ReplaceNode(String source, String target) {
super(CollectionAction.REPLACENODE);
this.source = source;
this.target = target;
}
public ReplaceNode setParallel(Boolean flag) {
this.parallel = flag;
return this;
}
@Override
public SolrParams getParams() {
ModifiableSolrParams params = (ModifiableSolrParams) super.getParams();
params.set("source", source);
params.set("target", target);
if (parallel != null) params.set("parallel", parallel.toString());
return params;
}
}
/*
* Returns a RebalanceLeaders object to rebalance leaders for a collection
*/

View File

@ -20,6 +20,7 @@ import org.apache.solr.common.util.Utils;
import org.noggit.JSONUtil;
import org.noggit.JSONWriter;
import java.util.Collections;
import java.util.LinkedHashMap;
import java.util.Map;
import java.util.Set;
@ -40,6 +41,17 @@ public class ZkNodeProps implements JSONWriter.Writable {
// Always wrapping introduces a memory leak.
}
public ZkNodeProps plus(String key , Object val) {
return plus(Collections.singletonMap(key,val));
}
public ZkNodeProps plus(Map<String, Object> newVals) {
LinkedHashMap<String, Object> copy = new LinkedHashMap<>(propMap);
if (newVals == null || newVals.isEmpty()) return new ZkNodeProps(copy);
copy.putAll(newVals);
return new ZkNodeProps(copy);
}
/**
* Constructor that populates the from array of Strings in form key1, value1,

View File

@ -147,7 +147,7 @@ public class ZkStateReader implements Closeable {
private class CollectionWatch {
int coreRefCount = 0;
Set<CollectionStateWatcher> stateWatchers = new HashSet<>();
Set<CollectionStateWatcher> stateWatchers = ConcurrentHashMap.newKeySet();
public boolean canBeRemoved() {
return coreRefCount + stateWatchers.size() == 0;
@ -1273,10 +1273,14 @@ public class ZkStateReader implements Closeable {
/* package-private for testing */
Set<CollectionStateWatcher> getStateWatchers(String collection) {
CollectionWatch watch = collectionWatches.get(collection);
if (watch == null)
return null;
return new HashSet<>(watch.stateWatchers);
final Set<CollectionStateWatcher> watchers = new HashSet<>();
collectionWatches.compute(collection, (k, v) -> {
if (v != null) {
watchers.addAll(v.stateWatchers);
}
return v;
});
return watchers;
}
// returns true if the state has changed

View File

@ -96,6 +96,9 @@ public interface CollectionParams {
// but the overseer is aware of these tasks
MOCK_COLL_TASK(false, LockLevel.COLLECTION),
MOCK_SHARD_TASK(false, LockLevel.SHARD),
//TODO when we have a node level lock use it here
REPLACENODE(true, LockLevel.NONE),
DELETENODE(true, LockLevel.NONE),
MOCK_REPLICA_TASK(false, LockLevel.REPLICA)
;
public final boolean isWrite;

View File

@ -19,8 +19,9 @@ package org.apache.solr.common.cloud;
import java.lang.invoke.MethodHandles;
import java.util.HashMap;
import java.util.Set;
import java.util.concurrent.Callable;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Future;
import java.util.concurrent.TimeUnit;
@ -81,6 +82,31 @@ public class TestCollectionStateWatchers extends SolrCloudTestCase {
});
}
private static void waitFor(String message, long timeout, TimeUnit unit, Callable<Boolean> predicate)
throws InterruptedException, ExecutionException {
Future<Boolean> future = executor.submit(() -> {
try {
while (true) {
if (predicate.call())
return true;
TimeUnit.MILLISECONDS.sleep(10);
}
}
catch (InterruptedException e) {
return false;
}
});
try {
if (future.get(timeout, unit) == true) {
return;
}
}
catch (TimeoutException e) {
// pass failure message on
}
future.cancel(true);
fail(message);
}
@Test
public void testSimpleCollectionWatch() throws Exception {
@ -113,9 +139,8 @@ public class TestCollectionStateWatchers extends SolrCloudTestCase {
cluster.stopJettySolrRunner(random().nextInt(cluster.getJettySolrRunners().size()));
assertTrue("CollectionStateWatcher was never notified of cluster change", latch.await(MAX_WAIT_TIMEOUT, TimeUnit.SECONDS));
Set<CollectionStateWatcher> watchers = client.getZkStateReader().getStateWatchers("testcollection");
assertTrue("CollectionStateWatcher wasn't cleared after completion",
watchers == null || watchers.size() == 0);
waitFor("CollectionStateWatcher wasn't cleared after completion", 1, TimeUnit.SECONDS,
() -> client.getZkStateReader().getStateWatchers("testcollection").isEmpty());
}
@ -144,8 +169,8 @@ public class TestCollectionStateWatchers extends SolrCloudTestCase {
assertTrue("CollectionStateWatcher isn't called when registering for already-watched collection",
latch.await(MAX_WAIT_TIMEOUT, TimeUnit.SECONDS));
assertEquals("CollectionStateWatcher should be removed",
1, client.getZkStateReader().getStateWatchers("currentstate").size());
waitFor("CollectionStateWatcher should be removed", 1, TimeUnit.SECONDS,
() -> client.getZkStateReader().getStateWatchers("currentstate").size() == 1);
}
@Test
@ -189,9 +214,8 @@ public class TestCollectionStateWatchers extends SolrCloudTestCase {
expectThrows(TimeoutException.class, () -> {
client.waitForState("nosuchcollection", 1, TimeUnit.SECONDS, ((liveNodes, collectionState) -> false));
});
Set<CollectionStateWatcher> watchers = client.getZkStateReader().getStateWatchers("nosuchcollection");
assertTrue("Watchers for collection should be removed after timeout",
watchers == null || watchers.size() == 0);
waitFor("Watchers for collection should be removed after timeout", 1, TimeUnit.SECONDS,
() -> client.getZkStateReader().getStateWatchers("nosuchcollection").isEmpty());
}
@ -229,18 +253,17 @@ public class TestCollectionStateWatchers extends SolrCloudTestCase {
}
@Test
public void testWatcherIsRemovedAfterTimeout() {
public void testWatcherIsRemovedAfterTimeout() throws Exception {
CloudSolrClient client = cluster.getSolrClient();
assertTrue("There should be no watchers for a non-existent collection!",
client.getZkStateReader().getStateWatchers("no-such-collection") == null);
client.getZkStateReader().getStateWatchers("no-such-collection").isEmpty());
expectThrows(TimeoutException.class, () -> {
client.waitForState("no-such-collection", 10, TimeUnit.MILLISECONDS, (n, c) -> DocCollection.isFullyActive(n, c, 1, 1));
});
Set<CollectionStateWatcher> watchers = client.getZkStateReader().getStateWatchers("no-such-collection");
assertTrue("Watchers for collection should be removed after timeout",
watchers == null || watchers.size() == 0);
waitFor("Watchers for collection should be removed after timeout", 1, TimeUnit.SECONDS,
() -> client.getZkStateReader().getStateWatchers("no-such-collection").isEmpty());
}

View File

@ -110,6 +110,17 @@ limitations under the License.
width: 40%;
}
#content #index .data
{
padding-bottom: 12px;
overflow: hidden;
}
#content #index .data:hover
{
overflow-x: auto;
}
#content #index .data li
{
padding-top: 3px;
@ -127,7 +138,6 @@ limitations under the License.
{
float: right;
text-overflow: ellipsis;
overflow: hidden;
white-space: nowrap;
width: 80%
}

View File

@ -111,8 +111,11 @@ solrAdminApp.controller('CollectionsController',
}
$scope.createAlias = function() {
var collections = $scope.aliasCollections.join(",");
Collections.createAlias({name: $scope.aliasToCreate, collections: collections}, function(data) {
var collections = [];
for (var i in $scope.aliasCollections) {
collections.push($scope.aliasCollections[i].name);
}
Collections.createAlias({name: $scope.aliasToCreate, collections: collections.join(",")}, function(data) {
$scope.hideAll();
});
}

View File

@ -17,7 +17,7 @@
// @todo test optimize (delete stuff, watch button appear, test button/form)
solrAdminApp.controller('CoreAdminController',
function($scope, $routeParams, $location, $timeout, Cores, Update, Constants){
function($scope, $routeParams, $location, $timeout, $route, Cores, Update, Constants){
$scope.resetMenu("cores", Constants.IS_ROOT_PAGE);
$scope.selectedCore = $routeParams.corename; // use 'corename' not 'core' to distinguish from /solr/:core/
$scope.refresh = function() {
@ -129,15 +129,15 @@ solrAdminApp.controller('CoreAdminController',
};
$scope.swapCores = function() {
if ($scope.swapOther) {
$swapMessage = "Please select a core to swap with";
if (!$scope.swapOther) {
$scope.swapMessage = "Please select a core to swap with";
} else if ($scope.swapOther == $scope.selectedCore) {
$swapMessage = "Cannot swap with the same core";
$scope.swapMessage = "Cannot swap with the same core";
} else {
Cores.swap({core: $scope.selectedCore, other: $scope.swapOther}, function(data) {
$location.path("/~cores/" + $scope.swapOther);
delete $scope.swapOther;
$scope.cancelSwap();
$scope.cancelSwapCores();
});
}
};

View File

@ -39,24 +39,27 @@ solrAdminApp.controller('DataImportController',
}
});
DataImport.config({core: $routeParams.core}, function (data) {
try {
var xml = $.parseXML(data.config);
} catch (err) {
$scope.hasHandlers = false;
return;
}
$scope.config = data.config;
$scope.entities = [];
$('document > entity', xml).each(function (i, element) {
$scope.entities.push($(element).attr('name'));
$scope.handler = $routeParams.handler;
if ($scope.handler && $scope.handler[0]=="/") {
$scope.handler = $scope.handler.substr(1);
}
if ($scope.handler) {
DataImport.config({core: $routeParams.core, name: $scope.handler}, function (data) {
try {
$scope.config = data.config;
var xml = $.parseXML(data.config);
$scope.entities = [];
$('document > entity', xml).each(function (i, element) {
$scope.entities.push($(element).attr('name'));
});
$scope.refreshStatus();
} catch (err) {
console.log(err);
}
});
});
}
$scope.lastUpdate = "unknown";
$scope.lastUpdateUTC = "";
$scope.refreshStatus();
};
$scope.toggleDebug = function () {
@ -81,7 +84,7 @@ solrAdminApp.controller('DataImportController',
}
$scope.reload = function () {
DataImport.reload({core: $routeParams.core}, function () {
DataImport.reload({core: $routeParams.core, name: $scope.handler}, function () {
$scope.reloaded = true;
$timeout(function () {
$scope.reloaded = false;
@ -126,6 +129,7 @@ solrAdminApp.controller('DataImportController',
}
params.core = $routeParams.core;
params.name = $scope.handler;
DataImport.post(params, function (data) {
$scope.rawResponse = JSON.stringify(data, null, 2);
@ -135,7 +139,7 @@ solrAdminApp.controller('DataImportController',
$scope.abort = function () {
$scope.isAborting = true;
DataImport.abort({core: $routeParams.core}, function () {
DataImport.abort({core: $routeParams.core, name: $scope.handler}, function () {
$timeout(function () {
$scope.isAborting = false;
$scope.refreshStatus();
@ -148,7 +152,7 @@ solrAdminApp.controller('DataImportController',
console.log("Refresh Status");
$scope.isStatusLoading = true;
DataImport.status({core: $routeParams.core}, function (data) {
DataImport.status({core: $routeParams.core, name: $scope.handler}, function (data) {
if (data[0] == "<") {
$scope.hasHandlers = false;
return;

View File

@ -16,7 +16,7 @@
*/
var contentTypeMap = { xml : 'text/xml', html : 'text/html', js : 'text/javascript', json : 'application/json', 'css' : 'text/css' };
var languages = {js: "javascript", xml:"xml", xsl:"xml", vm: "xml", html: "xml", json: "text", css: "css"};
var languages = {js: "javascript", xml:"xml", xsl:"xml", vm: "xml", html: "xml", json: "json", css: "css"};
solrAdminApp.controller('FilesController',
function($scope, $rootScope, $routeParams, $location, Files, Constants) {
@ -82,10 +82,10 @@ solrAdminApp.controller('FilesController',
Files.get({core: $routeParams.core, file: $scope.file, contentType: contentType}, function(data) {
$scope.content = data.data;
$scope.url = $scope.baseurl + data.config.url + "?" + $.param(data.config.params);
if (contentType.indexOf("text/plain") && data.data.indexOf("<?xml") || data.data.indexOf("<!--")) {
if (contentType.indexOf("text/plain") && (data.data.indexOf("<?xml")>=0) || data.data.indexOf("<!--")>=0) {
$scope.lang = "xml";
} else {
$scope.lang = languages[extension] || "text";
$scope.lang = languages[extension] || "txt";
}
});
}

View File

@ -477,6 +477,10 @@ var getFieldProperties = function(data, core, is, field) {
var row = display.rows[i];
row.cells = [];
if (!row.flags) {
continue; // Match the special case in the LukeRequestHandler
}
for (var j in display.columns) {
var flag = display.columns[j].key;
row.cells.push({key: flag, value: row.flags.indexOf(flag)>=0});

View File

@ -47,7 +47,7 @@ solrAdminServices.factory('System',
"add": {params:{action: "CREATE"}},
"unload": {params:{action: "UNLOAD", core: "@core"}},
"rename": {params:{action: "RENAME"}},
"swap": {params:{}},
"swap": {params:{action: "SWAP"}},
"reload": {method: "GET", params:{action:"RELOAD", core: "@core"}, headers:{doNotIntercept: "true"}},
"optimize": {params:{}}
});
@ -162,7 +162,7 @@ solrAdminServices.factory('System',
}])
.factory('DataImport',
['$resource', function($resource) {
return $resource('/solr/:core/dataimport', {core: '@core', indent:'on', wt:'json', _:Date.now()}, {
return $resource('/solr/:core/:name', {core: '@core', name: '@name', indent:'on', wt:'json', _:Date.now()}, {
"config": {params: {command: "show-config"}, headers: {doNotIntercept: "true"},
transformResponse: function(data) {
return {config: data};
@ -207,6 +207,8 @@ solrAdminServices.factory('System',
"list": {},
"get": {method: "GET", interceptor: {
response: function(config) {return config;}
}, transformResponse: function(data) {
return data;
}}
});
}])