dvProducersGens) throws IOException {
try {
dvp.decRef();
} catch (Throwable th) {
- if (t != null) {
+ if (t == null) {
t = th;
}
}
diff --git a/lucene/core/src/java/org/apache/lucene/index/SegmentReader.java b/lucene/core/src/java/org/apache/lucene/index/SegmentReader.java
index ccbcdf934e48..930340c70297 100644
--- a/lucene/core/src/java/org/apache/lucene/index/SegmentReader.java
+++ b/lucene/core/src/java/org/apache/lucene/index/SegmentReader.java
@@ -294,7 +294,7 @@ void notifyReaderClosedListeners(Throwable th) throws IOException {
synchronized(readerClosedListeners) {
for(ClosedListener listener : readerClosedListeners) {
try {
- listener.onClose(cacheHelper.getKey());
+ listener.onClose(readerCacheHelper.getKey());
} catch (Throwable t) {
if (th == null) {
th = t;
@@ -307,7 +307,7 @@ void notifyReaderClosedListeners(Throwable th) throws IOException {
}
}
- private final IndexReader.CacheHelper cacheHelper = new IndexReader.CacheHelper() {
+ private final IndexReader.CacheHelper readerCacheHelper = new IndexReader.CacheHelper() {
private final IndexReader.CacheKey cacheKey = new IndexReader.CacheKey();
@Override
@@ -317,18 +317,35 @@ public CacheKey getKey() {
@Override
public void addClosedListener(ClosedListener listener) {
+ ensureOpen();
readerClosedListeners.add(listener);
}
};
@Override
public CacheHelper getReaderCacheHelper() {
- return cacheHelper;
+ return readerCacheHelper;
}
+ /** Wrap the cache helper of the core to add ensureOpen() calls that make
+ * sure users do not register closed listeners on closed indices. */
+ private final IndexReader.CacheHelper coreCacheHelper = new IndexReader.CacheHelper() {
+
+ @Override
+ public CacheKey getKey() {
+ return core.getCacheHelper().getKey();
+ }
+
+ @Override
+ public void addClosedListener(ClosedListener listener) {
+ ensureOpen();
+ core.getCacheHelper().addClosedListener(listener);
+ }
+ };
+
@Override
public CacheHelper getCoreCacheHelper() {
- return core.getCacheHelper();
+ return coreCacheHelper;
}
@Override
diff --git a/lucene/core/src/java/org/apache/lucene/index/SortedDocValues.java b/lucene/core/src/java/org/apache/lucene/index/SortedDocValues.java
index 087e4871ed45..bd234159179a 100644
--- a/lucene/core/src/java/org/apache/lucene/index/SortedDocValues.java
+++ b/lucene/core/src/java/org/apache/lucene/index/SortedDocValues.java
@@ -46,7 +46,7 @@ protected SortedDocValues() {}
* @return ordinal for the document: this is dense, starts at 0, then
* increments by 1 for the next value in sorted order.
*/
- public abstract int ordValue();
+ public abstract int ordValue() throws IOException;
/** Retrieves the value for the specified ordinal. The returned
* {@link BytesRef} may be re-used across calls to {@link #lookupOrd(int)}
diff --git a/lucene/core/src/java/org/apache/lucene/index/StandardDirectoryReader.java b/lucene/core/src/java/org/apache/lucene/index/StandardDirectoryReader.java
index f0e7e9848251..bedf17ed7f40 100644
--- a/lucene/core/src/java/org/apache/lucene/index/StandardDirectoryReader.java
+++ b/lucene/core/src/java/org/apache/lucene/index/StandardDirectoryReader.java
@@ -484,7 +484,8 @@ public CacheKey getKey() {
@Override
public void addClosedListener(ClosedListener listener) {
- readerClosedListeners.add(listener);
+ ensureOpen();
+ readerClosedListeners.add(listener);
}
};
diff --git a/lucene/core/src/java/org/apache/lucene/search/FieldComparator.java b/lucene/core/src/java/org/apache/lucene/search/FieldComparator.java
index 82162015d0f7..3fe0e9333491 100644
--- a/lucene/core/src/java/org/apache/lucene/search/FieldComparator.java
+++ b/lucene/core/src/java/org/apache/lucene/search/FieldComparator.java
@@ -883,14 +883,6 @@ protected BinaryDocValues getBinaryDocValues(LeafReaderContext context, String f
return DocValues.getBinary(context.reader(), field);
}
- /** Check whether the given value represents null. This can be
- * useful if the {@link BinaryDocValues} returned by {@link #getBinaryDocValues}
- * use a special value as a sentinel.
- * NOTE: The null value can only be an EMPTY {@link BytesRef}. */
- protected boolean isNull(int doc, BytesRef term) throws IOException {
- return getValueForDoc(doc) == null;
- }
-
@Override
public LeafFieldComparator getLeafComparator(LeafReaderContext context) throws IOException {
docTerms = getBinaryDocValues(context, field);
diff --git a/lucene/core/src/java/org/apache/lucene/search/TopFieldCollector.java b/lucene/core/src/java/org/apache/lucene/search/TopFieldCollector.java
index 1ec322fe5d9d..97589c430dee 100644
--- a/lucene/core/src/java/org/apache/lucene/search/TopFieldCollector.java
+++ b/lucene/core/src/java/org/apache/lucene/search/TopFieldCollector.java
@@ -320,11 +320,9 @@ public boolean needsScores() {
* trackDocScores
to true as well.
* @return a {@link TopFieldCollector} instance which will sort the results by
* the sort criteria.
- * @throws IOException if there is a low-level I/O error
*/
public static TopFieldCollector create(Sort sort, int numHits,
- boolean fillFields, boolean trackDocScores, boolean trackMaxScore)
- throws IOException {
+ boolean fillFields, boolean trackDocScores, boolean trackMaxScore) {
return create(sort, numHits, null, fillFields, trackDocScores, trackMaxScore);
}
diff --git a/lucene/core/src/java/org/apache/lucene/store/FSDirectory.java b/lucene/core/src/java/org/apache/lucene/store/FSDirectory.java
index 50b52ceb07ac..158a2e230343 100644
--- a/lucene/core/src/java/org/apache/lucene/store/FSDirectory.java
+++ b/lucene/core/src/java/org/apache/lucene/store/FSDirectory.java
@@ -215,7 +215,7 @@ private static String[] listAll(Path dir, Set skipNames) throws IOExcept
try (DirectoryStream stream = Files.newDirectoryStream(dir)) {
for (Path path : stream) {
String name = path.getFileName().toString();
- if (skipNames != null && skipNames.contains(name) == false) {
+ if (skipNames == null || skipNames.contains(name) == false) {
entries.add(name);
}
}
diff --git a/lucene/core/src/java/org/apache/lucene/util/ByteBlockPool.java b/lucene/core/src/java/org/apache/lucene/util/ByteBlockPool.java
index 1b71440ae9ce..af8e19583bb0 100644
--- a/lucene/core/src/java/org/apache/lucene/util/ByteBlockPool.java
+++ b/lucene/core/src/java/org/apache/lucene/util/ByteBlockPool.java
@@ -324,28 +324,25 @@ public void setBytesRef(BytesRef term, int textStart) {
* the current position.
*/
public void append(final BytesRef bytes) {
- int length = bytes.length;
- if (length == 0) {
- return;
- }
+ int bytesLeft = bytes.length;
int offset = bytes.offset;
- int overflow = (length + byteUpto) - BYTE_BLOCK_SIZE;
- do {
- if (overflow <= 0) {
- System.arraycopy(bytes.bytes, offset, buffer, byteUpto, length);
- byteUpto += length;
+ while (bytesLeft > 0) {
+ int bufferLeft = BYTE_BLOCK_SIZE - byteUpto;
+ if (bytesLeft < bufferLeft) {
+ // fits within current buffer
+ System.arraycopy(bytes.bytes, offset, buffer, byteUpto, bytesLeft);
+ byteUpto += bytesLeft;
break;
} else {
- final int bytesToCopy = length-overflow;
- if (bytesToCopy > 0) {
- System.arraycopy(bytes.bytes, offset, buffer, byteUpto, bytesToCopy);
- offset += bytesToCopy;
- length -= bytesToCopy;
+ // fill up this buffer and move to next one
+ if (bufferLeft > 0) {
+ System.arraycopy(bytes.bytes, offset, buffer, byteUpto, bufferLeft);
}
nextBuffer();
- overflow = overflow - BYTE_BLOCK_SIZE;
+ bytesLeft -= bufferLeft;
+ offset += bufferLeft;
}
- } while(true);
+ }
}
/**
@@ -353,30 +350,18 @@ public void append(final BytesRef bytes) {
* length into the given byte array at offset off.
* Note: this method allows to copy across block boundaries.
*/
- public void readBytes(final long offset, final byte bytes[], final int off, final int length) {
- if (length == 0) {
- return;
- }
- int bytesOffset = off;
- int bytesLength = length;
+ public void readBytes(final long offset, final byte bytes[], int bytesOffset, int bytesLength) {
+ int bytesLeft = bytesLength;
int bufferIndex = (int) (offset >> BYTE_BLOCK_SHIFT);
- byte[] buffer = buffers[bufferIndex];
int pos = (int) (offset & BYTE_BLOCK_MASK);
- int overflow = (pos + length) - BYTE_BLOCK_SIZE;
- do {
- if (overflow <= 0) {
- System.arraycopy(buffer, pos, bytes, bytesOffset, bytesLength);
- break;
- } else {
- final int bytesToCopy = length - overflow;
- System.arraycopy(buffer, pos, bytes, bytesOffset, bytesToCopy);
- pos = 0;
- bytesLength -= bytesToCopy;
- bytesOffset += bytesToCopy;
- buffer = buffers[++bufferIndex];
- overflow = overflow - BYTE_BLOCK_SIZE;
- }
- } while (true);
+ while (bytesLeft > 0) {
+ byte[] buffer = buffers[bufferIndex++];
+ int chunk = Math.min(bytesLeft, BYTE_BLOCK_SIZE - pos);
+ System.arraycopy(buffer, pos, bytes, bytesOffset, chunk);
+ bytesOffset += chunk;
+ bytesLeft -= chunk;
+ pos = 0;
+ }
}
/**
diff --git a/lucene/core/src/java/org/apache/lucene/util/OfflineSorter.java b/lucene/core/src/java/org/apache/lucene/util/OfflineSorter.java
index fa223205a7ed..0bd306633dcc 100644
--- a/lucene/core/src/java/org/apache/lucene/util/OfflineSorter.java
+++ b/lucene/core/src/java/org/apache/lucene/util/OfflineSorter.java
@@ -24,6 +24,12 @@
import java.util.Comparator;
import java.util.List;
import java.util.Locale;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.atomic.AtomicLong;
import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.store.ChecksumIndexInput;
@@ -72,6 +78,9 @@ public class OfflineSorter {
private final int valueLength;
private final String tempFileNamePrefix;
+ private final ExecutorService exec;
+ private final Semaphore partitionsInRAM;
+
/**
* A bit more descriptive unit for constructors.
*
@@ -144,13 +153,13 @@ public class SortInfo {
/** number of lines of data read */
public int lineCount;
/** time spent merging sorted partitions (in milliseconds) */
- public long mergeTime;
+ public final AtomicLong mergeTimeMS = new AtomicLong();
/** time spent sorting data (in milliseconds) */
- public long sortTime;
+ public final AtomicLong sortTimeMS = new AtomicLong();
/** total time spent (in milliseconds) */
- public long totalTime;
+ public long totalTimeMS;
/** time spent in i/o read (in milliseconds) */
- public long readTime;
+ public long readTimeMS;
/** read buffer size (in bytes) */
public final long bufferSize = ramBufferSize.bytes;
@@ -160,17 +169,15 @@ public SortInfo() {}
@Override
public String toString() {
return String.format(Locale.ROOT,
- "time=%.2f sec. total (%.2f reading, %.2f sorting, %.2f merging), lines=%d, temp files=%d, merges=%d, soft ram limit=%.2f MB",
- totalTime / 1000.0d, readTime / 1000.0d, sortTime / 1000.0d, mergeTime / 1000.0d,
- lineCount, tempMergeFiles, mergeRounds,
- (double) bufferSize / MB);
+ "time=%.2f sec. total (%.2f reading, %.2f sorting, %.2f merging), lines=%d, temp files=%d, merges=%d, soft ram limit=%.2f MB",
+ totalTimeMS / 1000.0d, readTimeMS / 1000.0d, sortTimeMS.get() / 1000.0d, mergeTimeMS.get() / 1000.0d,
+ lineCount, tempMergeFiles, mergeRounds,
+ (double) bufferSize / MB);
}
}
private final BufferSize ramBufferSize;
- private final Counter bufferBytesUsed = Counter.newCounter();
- private final SortableBytesRefArray buffer;
SortInfo sortInfo;
private int maxTempFiles;
private final Comparator comparator;
@@ -184,7 +191,7 @@ public String toString() {
* @see BufferSize#automatic()
*/
public OfflineSorter(Directory dir, String tempFileNamePrefix) throws IOException {
- this(dir, tempFileNamePrefix, DEFAULT_COMPARATOR, BufferSize.automatic(), MAX_TEMPFILES, -1);
+ this(dir, tempFileNamePrefix, DEFAULT_COMPARATOR, BufferSize.automatic(), MAX_TEMPFILES, -1, null, 0);
}
/**
@@ -193,14 +200,30 @@ public OfflineSorter(Directory dir, String tempFileNamePrefix) throws IOExceptio
* @see BufferSize#automatic()
*/
public OfflineSorter(Directory dir, String tempFileNamePrefix, Comparator comparator) throws IOException {
- this(dir, tempFileNamePrefix, comparator, BufferSize.automatic(), MAX_TEMPFILES, -1);
+ this(dir, tempFileNamePrefix, comparator, BufferSize.automatic(), MAX_TEMPFILES, -1, null, 0);
}
/**
* All-details constructor. If {@code valueLength} is -1 (the default), the length of each value differs; otherwise,
- * all values have the specified length.
+ * all values have the specified length. If you pass a non-null {@code ExecutorService} then it will be
+ * used to run sorting operations that can be run concurrently, and maxPartitionsInRAM is the maximum
+ * concurrent in-memory partitions. Thus the maximum possible RAM used by this class while sorting is
+ * {@code maxPartitionsInRAM * ramBufferSize}.
*/
- public OfflineSorter(Directory dir, String tempFileNamePrefix, Comparator comparator, BufferSize ramBufferSize, int maxTempfiles, int valueLength) {
+ public OfflineSorter(Directory dir, String tempFileNamePrefix, Comparator comparator,
+ BufferSize ramBufferSize, int maxTempfiles, int valueLength, ExecutorService exec,
+ int maxPartitionsInRAM) {
+ if (exec != null) {
+ this.exec = exec;
+ if (maxPartitionsInRAM <= 0) {
+ throw new IllegalArgumentException("maxPartitionsInRAM must be > 0; got " + maxPartitionsInRAM);
+ }
+ } else {
+ this.exec = new SameThreadExecutorService();
+ maxPartitionsInRAM = 1;
+ }
+ this.partitionsInRAM = new Semaphore(maxPartitionsInRAM);
+
if (ramBufferSize.bytes < ABSOLUTE_MIN_SORT_BUFFER_SIZE) {
throw new IllegalArgumentException(MIN_BUFFER_SIZE_MSG + ": " + ramBufferSize.bytes);
}
@@ -208,14 +231,11 @@ public OfflineSorter(Directory dir, String tempFileNamePrefix, Comparator= 2");
}
- if (valueLength == -1) {
- buffer = new BytesRefArray(bufferBytesUsed);
- } else {
- if (valueLength == 0 || valueLength > Short.MAX_VALUE) {
- throw new IllegalArgumentException("valueLength must be 1 .. " + Short.MAX_VALUE + "; got: " + valueLength);
- }
- buffer = new FixedLengthBytesRefArray(valueLength);
+
+ if (valueLength != -1 && (valueLength == 0 || valueLength > Short.MAX_VALUE)) {
+ throw new IllegalArgumentException("valueLength must be 1 .. " + Short.MAX_VALUE + "; got: " + valueLength);
}
+
this.valueLength = valueLength;
this.ramBufferSize = ramBufferSize;
this.maxTempFiles = maxTempfiles;
@@ -240,9 +260,9 @@ public String getTempFileNamePrefix() {
public String sort(String inputFileName) throws IOException {
sortInfo = new SortInfo();
- sortInfo.totalTime = System.currentTimeMillis();
+ long startMS = System.currentTimeMillis();
- List segments = new ArrayList<>();
+ List> segments = new ArrayList<>();
int[] levelCounts = new int[1];
// So we can remove any partially written temp files on exception:
@@ -250,11 +270,21 @@ public String sort(String inputFileName) throws IOException {
boolean success = false;
try (ByteSequencesReader is = getReader(dir.openChecksumInput(inputFileName, IOContext.READONCE), inputFileName)) {
- int lineCount;
- while ((lineCount = readPartition(is)) > 0) {
- segments.add(sortPartition(trackingDir));
+ while (true) {
+ Partition part = readPartition(is);
+ if (part.count == 0) {
+ if (partitionsInRAM != null) {
+ partitionsInRAM.release();
+ }
+ assert part.exhausted;
+ break;
+ }
+
+ Callable job = new SortPartitionTask(trackingDir, part);
+
+ segments.add(exec.submit(job));
sortInfo.tempMergeFiles++;
- sortInfo.lineCount += lineCount;
+ sortInfo.lineCount += part.count;
levelCounts[0]++;
// Handle intermediate merges; we need a while loop to "cascade" the merge when necessary:
@@ -268,6 +298,10 @@ public String sort(String inputFileName) throws IOException {
levelCounts[mergeLevel] = 0;
mergeLevel++;
}
+
+ if (part.exhausted) {
+ break;
+ }
}
// TODO: we shouldn't have to do this? Can't we return a merged reader to
@@ -286,13 +320,13 @@ public String sort(String inputFileName) throws IOException {
result = out.getName();
}
} else {
- result = segments.get(0);
+ result = getPartition(segments.get(0)).fileName;
}
// We should be explicitly removing all intermediate files ourselves unless there is an exception:
assert trackingDir.getCreatedFiles().size() == 1 && trackingDir.getCreatedFiles().contains(result);
- sortInfo.totalTime = System.currentTimeMillis() - sortInfo.totalTime;
+ sortInfo.totalTimeMS = System.currentTimeMillis() - startMS;
CodecUtil.checkFooter(is.in);
@@ -300,6 +334,8 @@ public String sort(String inputFileName) throws IOException {
return result;
+ } catch (InterruptedException ie) {
+ throw new ThreadInterruptedException(ie);
} finally {
if (success == false) {
IOUtils.deleteFilesIgnoringExceptions(trackingDir, trackingDir.getCreatedFiles());
@@ -307,32 +343,6 @@ public String sort(String inputFileName) throws IOException {
}
}
- /** Sort a single partition in-memory. */
- protected String sortPartition(TrackingDirectoryWrapper trackingDir) throws IOException {
-
- try (IndexOutput tempFile = trackingDir.createTempOutput(tempFileNamePrefix, "sort", IOContext.DEFAULT);
- ByteSequencesWriter out = getWriter(tempFile);) {
-
- BytesRef spare;
-
- long start = System.currentTimeMillis();
- BytesRefIterator iter = buffer.iterator(comparator);
- sortInfo.sortTime += System.currentTimeMillis() - start;
-
- while ((spare = iter.next()) != null) {
- assert spare.length <= Short.MAX_VALUE;
- out.write(spare);
- }
-
- // Clean up the buffer for the next partition.
- buffer.clear();
-
- CodecUtil.writeFooter(out.out);
-
- return tempFile.getName();
- }
- }
-
/** Called on exception, to check whether the checksum is also corrupt in this source, and add that
* information (checksum matched or didn't) as a suppressed exception. */
private void verifyChecksum(Throwable priorException, ByteSequencesReader reader) throws IOException {
@@ -342,122 +352,107 @@ private void verifyChecksum(Throwable priorException, ByteSequencesReader reader
}
/** Merge the most recent {@code maxTempFile} partitions into a new partition. */
- void mergePartitions(Directory trackingDir, List segments) throws IOException {
+ void mergePartitions(Directory trackingDir, List> segments) throws IOException {
long start = System.currentTimeMillis();
-
- List segmentsToMerge;
+ List> segmentsToMerge;
if (segments.size() > maxTempFiles) {
segmentsToMerge = segments.subList(segments.size() - maxTempFiles, segments.size());
} else {
segmentsToMerge = segments;
}
- PriorityQueue queue = new PriorityQueue(segmentsToMerge.size()) {
- @Override
- protected boolean lessThan(FileAndTop a, FileAndTop b) {
- return comparator.compare(a.current, b.current) < 0;
- }
- };
-
- ByteSequencesReader[] streams = new ByteSequencesReader[segmentsToMerge.size()];
-
- String newSegmentName = null;
-
- try (ByteSequencesWriter writer = getWriter(trackingDir.createTempOutput(tempFileNamePrefix, "sort", IOContext.DEFAULT))) {
+ sortInfo.mergeRounds++;
- newSegmentName = writer.out.getName();
-
- // Open streams and read the top for each file
- for (int i = 0; i < segmentsToMerge.size(); i++) {
- streams[i] = getReader(dir.openChecksumInput(segmentsToMerge.get(i), IOContext.READONCE), segmentsToMerge.get(i));
- BytesRef item = null;
- try {
- item = streams[i].next();
- } catch (Throwable t) {
- verifyChecksum(t, streams[i]);
- }
- assert item != null;
- queue.insertWithOverflow(new FileAndTop(i, item));
- }
-
- // Unix utility sort() uses ordered array of files to pick the next line from, updating
- // it as it reads new lines. The PQ used here is a more elegant solution and has
- // a nicer theoretical complexity bound :) The entire sorting process is I/O bound anyway
- // so it shouldn't make much of a difference (didn't check).
- FileAndTop top;
- while ((top = queue.top()) != null) {
- writer.write(top.current);
- try {
- top.current = streams[top.fd].next();
- } catch (Throwable t) {
- verifyChecksum(t, streams[top.fd]);
- }
+ MergePartitionsTask task = new MergePartitionsTask(trackingDir, new ArrayList<>(segmentsToMerge));
- if (top.current != null) {
- queue.updateTop();
- } else {
- queue.pop();
- }
- }
+ segmentsToMerge.clear();
+ segments.add(exec.submit(task));
- CodecUtil.writeFooter(writer.out);
+ sortInfo.tempMergeFiles++;
+ }
- for(ByteSequencesReader reader : streams) {
- CodecUtil.checkFooter(reader.in);
- }
-
- sortInfo.mergeTime += System.currentTimeMillis() - start;
- sortInfo.mergeRounds++;
- } finally {
- IOUtils.close(streams);
+ /** Holds one partition of items, either loaded into memory or based on a file. */
+ private static class Partition {
+ public final SortableBytesRefArray buffer;
+ public final boolean exhausted;
+ public final long count;
+ public final String fileName;
+
+ /** A partition loaded into memory. */
+ public Partition(SortableBytesRefArray buffer, boolean exhausted) {
+ this.buffer = buffer;
+ this.fileName = null;
+ this.count = buffer.size();
+ this.exhausted = exhausted;
}
- IOUtils.deleteFiles(trackingDir, segmentsToMerge);
-
- segmentsToMerge.clear();
- segments.add(newSegmentName);
-
- sortInfo.tempMergeFiles++;
+ /** An on-disk partition. */
+ public Partition(String fileName, long count) {
+ this.buffer = null;
+ this.fileName = fileName;
+ this.count = count;
+ this.exhausted = true;
+ }
}
- /** Read in a single partition of data */
- int readPartition(ByteSequencesReader reader) throws IOException {
- long start = System.currentTimeMillis();
- if (valueLength != -1) {
- int limit = ramBufferSize.bytes / valueLength;
- for(int i=0;i ramBufferSize.bytes) {
+ break;
+ }
}
- buffer.append(item);
}
- } else {
- while (true) {
- BytesRef item = null;
- try {
- item = reader.next();
- } catch (Throwable t) {
- verifyChecksum(t, reader);
- }
- if (item == null) {
- break;
- }
- buffer.append(item);
- // Account for the created objects.
- // (buffer slots do not account to buffer size.)
- if (bufferBytesUsed.get() > ramBufferSize.bytes) {
- break;
- }
+ sortInfo.readTimeMS += System.currentTimeMillis() - start;
+ success = true;
+ return new Partition(buffer, exhausted);
+ } finally {
+ if (success == false && partitionsInRAM != null) {
+ partitionsInRAM.release();
}
}
- sortInfo.readTime += System.currentTimeMillis() - start;
- return buffer.size();
}
static class FileAndTop {
@@ -471,7 +466,7 @@ static class FileAndTop {
}
/** Subclasses can override to change how byte sequences are written to disk. */
- protected ByteSequencesWriter getWriter(IndexOutput out) throws IOException {
+ protected ByteSequencesWriter getWriter(IndexOutput out, long itemCount) throws IOException {
return new ByteSequencesWriter(out);
}
@@ -587,5 +582,148 @@ public void close() throws IOException {
/** Returns the comparator in use to sort entries */
public Comparator getComparator() {
return comparator;
- }
+ }
+
+ /** Sorts one in-memory partition, writes it to disk, and returns the resulting file-based partition. */
+ private class SortPartitionTask implements Callable {
+
+ private final Directory dir;
+ private final Partition part;
+
+ public SortPartitionTask(Directory dir, Partition part) {
+ this.dir = dir;
+ this.part = part;
+ }
+
+ @Override
+ public Partition call() throws IOException {
+ try (IndexOutput tempFile = dir.createTempOutput(tempFileNamePrefix, "sort", IOContext.DEFAULT);
+ ByteSequencesWriter out = getWriter(tempFile, part.buffer.size());) {
+
+ BytesRef spare;
+
+ long startMS = System.currentTimeMillis();
+ BytesRefIterator iter = part.buffer.iterator(comparator);
+ sortInfo.sortTimeMS.addAndGet(System.currentTimeMillis() - startMS);
+
+ int count = 0;
+ while ((spare = iter.next()) != null) {
+ assert spare.length <= Short.MAX_VALUE;
+ out.write(spare);
+ count++;
+ }
+
+ assert count == part.count;
+
+ CodecUtil.writeFooter(out.out);
+ part.buffer.clear();
+
+ return new Partition(tempFile.getName(), part.count);
+ } finally {
+ if (partitionsInRAM != null) {
+ partitionsInRAM.release();
+ }
+ }
+ }
+ }
+
+ private Partition getPartition(Future future) throws IOException {
+ try {
+ return future.get();
+ } catch (InterruptedException ie) {
+ throw new ThreadInterruptedException(ie);
+ } catch (ExecutionException ee) {
+ IOUtils.reThrow(ee.getCause());
+ // oh so soon to go away:
+ return null;
+ }
+ }
+
+ /** Merges multiple file-based partitions to a single on-disk partition. */
+ private class MergePartitionsTask implements Callable {
+ private final Directory dir;
+ private final List> segmentsToMerge;
+
+ public MergePartitionsTask(Directory dir, List> segmentsToMerge) {
+ this.dir = dir;
+ this.segmentsToMerge = segmentsToMerge;
+ }
+
+ @Override
+ public Partition call() throws IOException {
+ long totalCount = 0;
+ for (Future segment : segmentsToMerge) {
+ totalCount += getPartition(segment).count;
+ }
+
+ PriorityQueue queue = new PriorityQueue(segmentsToMerge.size()) {
+ @Override
+ protected boolean lessThan(FileAndTop a, FileAndTop b) {
+ return comparator.compare(a.current, b.current) < 0;
+ }
+ };
+
+ ByteSequencesReader[] streams = new ByteSequencesReader[segmentsToMerge.size()];
+
+ String newSegmentName = null;
+
+ long startMS = System.currentTimeMillis();
+ try (ByteSequencesWriter writer = getWriter(dir.createTempOutput(tempFileNamePrefix, "sort", IOContext.DEFAULT), totalCount)) {
+
+ newSegmentName = writer.out.getName();
+
+ // Open streams and read the top for each file
+ for (int i = 0; i < segmentsToMerge.size(); i++) {
+ Partition segment = getPartition(segmentsToMerge.get(i));
+ streams[i] = getReader(dir.openChecksumInput(segment.fileName, IOContext.READONCE), segment.fileName);
+
+ BytesRef item = null;
+ try {
+ item = streams[i].next();
+ } catch (Throwable t) {
+ verifyChecksum(t, streams[i]);
+ }
+ assert item != null;
+ queue.insertWithOverflow(new FileAndTop(i, item));
+ }
+
+ // Unix utility sort() uses ordered array of files to pick the next line from, updating
+ // it as it reads new lines. The PQ used here is a more elegant solution and has
+ // a nicer theoretical complexity bound :) The entire sorting process is I/O bound anyway
+ // so it shouldn't make much of a difference (didn't check).
+ FileAndTop top;
+ while ((top = queue.top()) != null) {
+ writer.write(top.current);
+ try {
+ top.current = streams[top.fd].next();
+ } catch (Throwable t) {
+ verifyChecksum(t, streams[top.fd]);
+ }
+
+ if (top.current != null) {
+ queue.updateTop();
+ } else {
+ queue.pop();
+ }
+ }
+
+ CodecUtil.writeFooter(writer.out);
+
+ for(ByteSequencesReader reader : streams) {
+ CodecUtil.checkFooter(reader.in);
+ }
+
+ sortInfo.mergeTimeMS.addAndGet(System.currentTimeMillis() - startMS);
+ } finally {
+ IOUtils.close(streams);
+ }
+ List toDelete = new ArrayList<>();
+ for (Future segment : segmentsToMerge) {
+ toDelete.add(getPartition(segment).fileName);
+ }
+ IOUtils.deleteFiles(dir, toDelete);
+
+ return new Partition(newSegmentName, totalCount);
+ }
+ }
}
diff --git a/lucene/core/src/java/org/apache/lucene/util/SameThreadExecutorService.java b/lucene/core/src/java/org/apache/lucene/util/SameThreadExecutorService.java
new file mode 100644
index 000000000000..169b9f83bb31
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/util/SameThreadExecutorService.java
@@ -0,0 +1,69 @@
+/*
+ * 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.util;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.AbstractExecutorService;
+import java.util.concurrent.RejectedExecutionException;
+import java.util.concurrent.TimeUnit;
+
+/** An {@code ExecutorService} that executes tasks immediately in the calling thread during submit. */
+class SameThreadExecutorService extends AbstractExecutorService {
+ private volatile boolean shutdown;
+
+ @Override
+ public void execute(Runnable command) {
+ checkShutdown();
+ command.run();
+ }
+
+ @Override
+ public List shutdownNow() {
+ shutdown();
+ return Collections.emptyList();
+ }
+
+ @Override
+ public void shutdown() {
+ this.shutdown = true;
+ }
+
+ @Override
+ public boolean isTerminated() {
+ // Simplified: we don't check for any threads hanging in execute (we could
+ // introduce an atomic counter, but there seems to be no point).
+ return shutdown == true;
+ }
+
+ @Override
+ public boolean isShutdown() {
+ return shutdown == true;
+ }
+
+ @Override
+ public boolean awaitTermination(long timeout, TimeUnit unit) throws InterruptedException {
+ // See comment in isTerminated();
+ return true;
+ }
+
+ private void checkShutdown() {
+ if (shutdown) {
+ throw new RejectedExecutionException("Executor is shut down.");
+ }
+ }
+}
diff --git a/lucene/core/src/java/org/apache/lucene/util/Version.java b/lucene/core/src/java/org/apache/lucene/util/Version.java
index da6d653bce71..f6e6adcc5ed1 100644
--- a/lucene/core/src/java/org/apache/lucene/util/Version.java
+++ b/lucene/core/src/java/org/apache/lucene/util/Version.java
@@ -101,6 +101,13 @@ public final class Version {
@Deprecated
public static final Version LUCENE_6_5_0 = new Version(6, 5, 0);
+ /**
+ * Match settings and bugs in Lucene's 6.5.1 release.
+ * @deprecated Use latest
+ */
+ @Deprecated
+ public static final Version LUCENE_6_5_1 = new Version(6, 5, 1);
+
/**
* Match settings and bugs in Lucene's 6.6.0 release.
* @deprecated Use latest
diff --git a/lucene/core/src/java/org/apache/lucene/util/bkd/BKDWriter.java b/lucene/core/src/java/org/apache/lucene/util/bkd/BKDWriter.java
index eeb40fa41159..1575a5b28be3 100644
--- a/lucene/core/src/java/org/apache/lucene/util/bkd/BKDWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/util/bkd/BKDWriter.java
@@ -884,11 +884,11 @@ protected int byteAt(BytesRef ref, int i) {
};
}
- OfflineSorter sorter = new OfflineSorter(tempDir, tempFileNamePrefix + "_bkd" + dim, cmp, offlineSorterBufferMB, offlineSorterMaxTempFiles, bytesPerDoc) {
+ OfflineSorter sorter = new OfflineSorter(tempDir, tempFileNamePrefix + "_bkd" + dim, cmp, offlineSorterBufferMB, offlineSorterMaxTempFiles, bytesPerDoc, null, 0) {
/** We write/read fixed-byte-width file that {@link OfflinePointReader} can read. */
@Override
- protected ByteSequencesWriter getWriter(IndexOutput out) {
+ protected ByteSequencesWriter getWriter(IndexOutput out, long count) {
return new ByteSequencesWriter(out) {
@Override
public void write(byte[] bytes, int off, int len) throws IOException {
diff --git a/lucene/core/src/test/org/apache/lucene/analysis/standard/TestStandardAnalyzer.java b/lucene/core/src/test/org/apache/lucene/analysis/standard/TestStandardAnalyzer.java
index 2cc9274ad420..6abbc2bfc5bd 100644
--- a/lucene/core/src/test/org/apache/lucene/analysis/standard/TestStandardAnalyzer.java
+++ b/lucene/core/src/test/org/apache/lucene/analysis/standard/TestStandardAnalyzer.java
@@ -393,4 +393,27 @@ public void testNormalize() {
Analyzer a = new StandardAnalyzer();
assertEquals(new BytesRef("\"\\à3[]()! cz@"), a.normalize("dummy", "\"\\À3[]()! Cz@"));
}
+
+ public void testMaxTokenLengthDefault() throws Exception {
+ StandardAnalyzer a = new StandardAnalyzer();
+
+ StringBuilder bToken = new StringBuilder();
+ // exact max length:
+ for(int i=0;i {});
+ r.leaves().get(0).reader().getReaderCacheHelper().addClosedListener(key -> {});
+ r.leaves().get(0).reader().getCoreCacheHelper().addClosedListener(key -> {});
+
+ // But now we close
+ r.close();
+ expectThrows(AlreadyClosedException.class, () -> r.getReaderCacheHelper().addClosedListener(key -> {}));
+ expectThrows(AlreadyClosedException.class, () -> r.leaves().get(0).reader().getReaderCacheHelper().addClosedListener(key -> {}));
+ expectThrows(AlreadyClosedException.class, () -> r.leaves().get(0).reader().getCoreCacheHelper().addClosedListener(key -> {}));
+
+ dir.close();
}
}
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestIndexSorting.java b/lucene/core/src/test/org/apache/lucene/index/TestIndexSorting.java
index 4ef580bf0717..be3a2af18b74 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestIndexSorting.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestIndexSorting.java
@@ -2403,4 +2403,86 @@ public void testTieBreak() throws Exception {
}
IOUtils.close(r, w, dir);
}
+
+ public void testIndexSortWithSparseField() throws Exception {
+ Directory dir = newDirectory();
+ IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
+ SortField sortField = new SortField("dense_int", SortField.Type.INT, true);
+ Sort indexSort = new Sort(sortField);
+ iwc.setIndexSort(indexSort);
+ IndexWriter w = new IndexWriter(dir, iwc);
+ Field textField = newTextField("sparse_text", "", Field.Store.NO);
+ for (int i = 0; i < 128; i++) {
+ Document doc = new Document();
+ doc.add(new NumericDocValuesField("dense_int", i));
+ if (i < 64) {
+ doc.add(new NumericDocValuesField("sparse_int", i));
+ doc.add(new BinaryDocValuesField("sparse_binary", new BytesRef(Integer.toString(i))));
+ textField.setStringValue("foo");
+ doc.add(textField);
+ }
+ w.addDocument(doc);
+ }
+ w.commit();
+ w.forceMerge(1);
+ DirectoryReader r = DirectoryReader.open(w);
+ assertEquals(1, r.leaves().size());
+ LeafReader leafReader = r.leaves().get(0).reader();
+
+ NumericDocValues denseValues = leafReader.getNumericDocValues("dense_int");
+ NumericDocValues sparseValues = leafReader.getNumericDocValues("sparse_int");
+ BinaryDocValues sparseBinaryValues = leafReader.getBinaryDocValues("sparse_binary");
+ NumericDocValues normsValues = leafReader.getNormValues("sparse_text");
+ for(int docID = 0; docID < 128; docID++) {
+ assertTrue(denseValues.advanceExact(docID));
+ assertEquals(127-docID, (int) denseValues.longValue());
+ if (docID >= 64) {
+ assertTrue(denseValues.advanceExact(docID));
+ assertTrue(sparseValues.advanceExact(docID));
+ assertTrue(sparseBinaryValues.advanceExact(docID));
+ assertTrue(normsValues.advanceExact(docID));
+ assertEquals(124, normsValues.longValue());
+ assertEquals(127-docID, (int) sparseValues.longValue());
+ assertEquals(new BytesRef(Integer.toString(127-docID)), sparseBinaryValues.binaryValue());
+ } else {
+ assertFalse(sparseBinaryValues.advanceExact(docID));
+ assertFalse(sparseValues.advanceExact(docID));
+ assertFalse(normsValues.advanceExact(docID));
+ }
+ }
+ IOUtils.close(r, w, dir);
+ }
+
+ public void testIndexSortOnSparseField() throws Exception {
+ Directory dir = newDirectory();
+ IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
+ SortField sortField = new SortField("sparse", SortField.Type.INT, false);
+ sortField.setMissingValue(Integer.MIN_VALUE);
+ Sort indexSort = new Sort(sortField);
+ iwc.setIndexSort(indexSort);
+ IndexWriter w = new IndexWriter(dir, iwc);
+ for (int i = 0; i < 128; i++) {
+ Document doc = new Document();
+ if (i < 64) {
+ doc.add(new NumericDocValuesField("sparse", i));
+ }
+ w.addDocument(doc);
+ }
+ w.commit();
+ w.forceMerge(1);
+ DirectoryReader r = DirectoryReader.open(w);
+ assertEquals(1, r.leaves().size());
+ LeafReader leafReader = r.leaves().get(0).reader();
+ NumericDocValues sparseValues = leafReader.getNumericDocValues("sparse");
+ for(int docID = 0; docID < 128; docID++) {
+ if (docID >= 64) {
+ assertTrue(sparseValues.advanceExact(docID));
+ assertEquals(docID-64, (int) sparseValues.longValue());
+ } else {
+ assertFalse(sparseValues.advanceExact(docID));
+ }
+ }
+ IOUtils.close(r, w, dir);
+ }
+
}
diff --git a/lucene/core/src/test/org/apache/lucene/store/TestDirectory.java b/lucene/core/src/test/org/apache/lucene/store/TestDirectory.java
index 5e4a5934d345..23c1bcdd6fd0 100644
--- a/lucene/core/src/test/org/apache/lucene/store/TestDirectory.java
+++ b/lucene/core/src/test/org/apache/lucene/store/TestDirectory.java
@@ -14,16 +14,19 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package org.apache.lucene.store;
+package org.apache.lucene.store;
import java.io.IOException;
import java.nio.file.Files;
import java.nio.file.Path;
import java.util.ArrayList;
import java.util.Arrays;
+import java.util.HashSet;
import java.util.List;
+import java.util.Set;
+import org.apache.lucene.mockfile.ExtrasFS;
import org.apache.lucene.util.LuceneTestCase;
public class TestDirectory extends LuceneTestCase {
@@ -137,5 +140,17 @@ public void testNotDirectory() throws Throwable {
fsDir.close();
}
}
+
+ public void testListAll() throws Throwable {
+ Path dir = createTempDir("testdir");
+ assumeFalse("this test does not expect extra files", dir.getFileSystem().provider() instanceof ExtrasFS);
+ Path file1 = Files.createFile(dir.resolve("tempfile1"));
+ Path file2 = Files.createFile(dir.resolve("tempfile2"));
+ Set files = new HashSet<>(Arrays.asList(FSDirectory.listAll(dir)));
+
+ assertTrue(files.size() == 2);
+ assertTrue(files.contains(file1.getFileName().toString()));
+ assertTrue(files.contains(file2.getFileName().toString()));
+ }
}
diff --git a/lucene/core/src/test/org/apache/lucene/util/TestByteBlockPool.java b/lucene/core/src/test/org/apache/lucene/util/TestByteBlockPool.java
index df73687d17c8..475f716acf5e 100644
--- a/lucene/core/src/test/org/apache/lucene/util/TestByteBlockPool.java
+++ b/lucene/core/src/test/org/apache/lucene/util/TestByteBlockPool.java
@@ -18,6 +18,7 @@
import java.io.IOException;
import java.util.ArrayList;
+import java.util.Arrays;
import java.util.List;
public class TestByteBlockPool extends LuceneTestCase {
@@ -34,8 +35,7 @@ public void testReadAndWrite() throws IOException {
final int numValues = atLeast(100);
BytesRefBuilder ref = new BytesRefBuilder();
for (int i = 0; i < numValues; i++) {
- final String value = TestUtil.randomRealisticUnicodeString(random(),
- maxLength);
+ final String value = TestUtil.randomRealisticUnicodeString(random(), maxLength);
list.add(new BytesRef(value));
ref.copyChars(value);
pool.append(ref.get());
@@ -76,5 +76,33 @@ public void testReadAndWrite() throws IOException {
pool.nextBuffer(); // prepare for next iter
}
}
- }
+ }
+
+ public void testLargeRandomBlocks() throws IOException {
+ Counter bytesUsed = Counter.newCounter();
+ ByteBlockPool pool = new ByteBlockPool(new ByteBlockPool.DirectTrackingAllocator(bytesUsed));
+ pool.nextBuffer();
+
+ List items = new ArrayList<>();
+ for (int i=0;i<100;i++) {
+ int size;
+ if (random().nextBoolean()) {
+ size = TestUtil.nextInt(random(), 100, 1000);
+ } else {
+ size = TestUtil.nextInt(random(), 50000, 100000);
+ }
+ byte[] bytes = new byte[size];
+ random().nextBytes(bytes);
+ items.add(bytes);
+ pool.append(new BytesRef(bytes));
+ }
+
+ long position = 0;
+ for (byte[] expected : items) {
+ byte[] actual = new byte[expected.length];
+ pool.readBytes(position, actual, 0, actual.length);
+ assertTrue(Arrays.equals(expected, actual));
+ position += expected.length;
+ }
+ }
}
diff --git a/lucene/core/src/test/org/apache/lucene/util/TestOfflineSorter.java b/lucene/core/src/test/org/apache/lucene/util/TestOfflineSorter.java
index 49ed11013d9c..68ac0a285c40 100644
--- a/lucene/core/src/test/org/apache/lucene/util/TestOfflineSorter.java
+++ b/lucene/core/src/test/org/apache/lucene/util/TestOfflineSorter.java
@@ -24,10 +24,15 @@
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Comparator;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.index.CorruptIndexException;
+import org.apache.lucene.store.ChecksumIndexInput;
import org.apache.lucene.store.CorruptingIndexOutput;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.FilterDirectory;
@@ -72,11 +77,25 @@ public void testSingleLine() throws Exception {
}
}
+ private ExecutorService randomExecutorServiceOrNull() {
+ if (random().nextBoolean()) {
+ return null;
+ } else {
+ return new ThreadPoolExecutor(1, TestUtil.nextInt(random(), 2, 6), Long.MAX_VALUE, TimeUnit.MILLISECONDS,
+ new LinkedBlockingQueue(),
+ new NamedThreadFactory("TestIndexSearcher"));
+ }
+ }
+
public void testIntermediateMerges() throws Exception {
// Sort 20 mb worth of data with 1mb buffer, binary merging.
try (Directory dir = newDirectory()) {
- SortInfo info = checkSort(dir, new OfflineSorter(dir, "foo", OfflineSorter.DEFAULT_COMPARATOR, BufferSize.megabytes(1), 2, -1),
+ ExecutorService exec = randomExecutorServiceOrNull();
+ SortInfo info = checkSort(dir, new OfflineSorter(dir, "foo", OfflineSorter.DEFAULT_COMPARATOR, BufferSize.megabytes(1), 2, -1, exec, TestUtil.nextInt(random(), 1, 4)),
generateRandom((int)OfflineSorter.MB * 20));
+ if (exec != null) {
+ exec.shutdownNow();
+ }
assertTrue(info.mergeRounds > 10);
}
}
@@ -84,8 +103,12 @@ public void testIntermediateMerges() throws Exception {
public void testSmallRandom() throws Exception {
// Sort 20 mb worth of data with 1mb buffer.
try (Directory dir = newDirectory()) {
- SortInfo sortInfo = checkSort(dir, new OfflineSorter(dir, "foo", OfflineSorter.DEFAULT_COMPARATOR, BufferSize.megabytes(1), OfflineSorter.MAX_TEMPFILES, -1),
+ ExecutorService exec = randomExecutorServiceOrNull();
+ SortInfo sortInfo = checkSort(dir, new OfflineSorter(dir, "foo", OfflineSorter.DEFAULT_COMPARATOR, BufferSize.megabytes(1), OfflineSorter.MAX_TEMPFILES, -1, exec, TestUtil.nextInt(random(), 1, 4)),
generateRandom((int)OfflineSorter.MB * 20));
+ if (exec != null) {
+ exec.shutdownNow();
+ }
assertEquals(3, sortInfo.mergeRounds);
}
}
@@ -94,8 +117,12 @@ public void testSmallRandom() throws Exception {
public void testLargerRandom() throws Exception {
// Sort 100MB worth of data with 15mb buffer.
try (Directory dir = newFSDirectory(createTempDir())) {
- checkSort(dir, new OfflineSorter(dir, "foo", OfflineSorter.DEFAULT_COMPARATOR, BufferSize.megabytes(16), OfflineSorter.MAX_TEMPFILES, -1),
+ ExecutorService exec = randomExecutorServiceOrNull();
+ checkSort(dir, new OfflineSorter(dir, "foo", OfflineSorter.DEFAULT_COMPARATOR, BufferSize.megabytes(16), OfflineSorter.MAX_TEMPFILES, -1, exec, TestUtil.nextInt(random(), 1, 4)),
generateRandom((int)OfflineSorter.MB * 100));
+ if (exec != null) {
+ exec.shutdownNow();
+ }
}
}
@@ -357,7 +384,7 @@ public IndexOutput createTempOutput(String prefix, String suffix, IOContext cont
writeAll(unsorted, generateFixed((int) (OfflineSorter.MB * 3)));
CorruptIndexException e = expectThrows(CorruptIndexException.class, () -> {
- new OfflineSorter(dir, "foo", OfflineSorter.DEFAULT_COMPARATOR, BufferSize.megabytes(1), 10, -1).sort(unsorted.getName());
+ new OfflineSorter(dir, "foo", OfflineSorter.DEFAULT_COMPARATOR, BufferSize.megabytes(1), 10, -1, null, 0).sort(unsorted.getName());
});
assertTrue(e.getMessage().contains("checksum failed (hardware problem?)"));
}
@@ -407,7 +434,7 @@ protected void corruptFile() throws IOException {
writeAll(unsorted, generateFixed((int) (OfflineSorter.MB * 3)));
EOFException e = expectThrows(EOFException.class, () -> {
- new OfflineSorter(dir, "foo", OfflineSorter.DEFAULT_COMPARATOR, BufferSize.megabytes(1), 10, -1).sort(unsorted.getName());
+ new OfflineSorter(dir, "foo", OfflineSorter.DEFAULT_COMPARATOR, BufferSize.megabytes(1), 10, -1, null, 0).sort(unsorted.getName());
});
assertEquals(1, e.getSuppressed().length);
assertTrue(e.getSuppressed()[0] instanceof CorruptIndexException);
@@ -429,8 +456,12 @@ public void testFixedLengthHeap() throws Exception {
CodecUtil.writeFooter(out);
}
- OfflineSorter sorter = new OfflineSorter(dir, "foo", OfflineSorter.DEFAULT_COMPARATOR, BufferSize.megabytes(4), OfflineSorter.MAX_TEMPFILES, Integer.BYTES);
+ ExecutorService exec = randomExecutorServiceOrNull();
+ OfflineSorter sorter = new OfflineSorter(dir, "foo", OfflineSorter.DEFAULT_COMPARATOR, BufferSize.megabytes(4), OfflineSorter.MAX_TEMPFILES, Integer.BYTES, exec, TestUtil.nextInt(random(), 1, 4));
sorter.sort(out.getName());
+ if (exec != null) {
+ exec.shutdownNow();
+ }
// 1 MB of ints with 4 MH heap allowed should have been sorted in a single heap partition:
assertEquals(0, sorter.sortInfo.mergeRounds);
dir.close();
@@ -447,7 +478,7 @@ public void testFixedLengthLiesLiesLies() throws Exception {
CodecUtil.writeFooter(out);
}
- OfflineSorter sorter = new OfflineSorter(dir, "foo", OfflineSorter.DEFAULT_COMPARATOR, BufferSize.megabytes(4), OfflineSorter.MAX_TEMPFILES, Long.BYTES);
+ OfflineSorter sorter = new OfflineSorter(dir, "foo", OfflineSorter.DEFAULT_COMPARATOR, BufferSize.megabytes(4), OfflineSorter.MAX_TEMPFILES, Long.BYTES, null, 0);
IllegalArgumentException e = expectThrows(IllegalArgumentException.class, () -> {
sorter.sort(out.getName());
});
@@ -455,18 +486,59 @@ public void testFixedLengthLiesLiesLies() throws Exception {
dir.close();
}
+ // OfflineSorter should not call my BytesSequencesReader.next() again after it already returned null:
+ public void testOverNexting() throws Exception {
+ Directory dir = newDirectory();
+ IndexOutput out = dir.createTempOutput("unsorted", "tmp", IOContext.DEFAULT);
+ try (ByteSequencesWriter w = new OfflineSorter.ByteSequencesWriter(out)) {
+ byte[] bytes = new byte[Integer.BYTES];
+ random().nextBytes(bytes);
+ w.write(bytes);
+ CodecUtil.writeFooter(out);
+ }
+
+ new OfflineSorter(dir, "foo", OfflineSorter.DEFAULT_COMPARATOR, BufferSize.megabytes(4), OfflineSorter.MAX_TEMPFILES, Integer.BYTES, null, 0) {
+ @Override
+ protected ByteSequencesReader getReader(ChecksumIndexInput in, String name) throws IOException {
+ ByteSequencesReader other = super.getReader(in, name);
+
+ return new ByteSequencesReader(in, name) {
+
+ private boolean alreadyEnded;
+
+ @Override
+ public BytesRef next() throws IOException {
+ // if we returned null already, OfflineSorter should not call next() again
+ assertFalse(alreadyEnded);
+ BytesRef result = other.next();
+ if (result == null) {
+ alreadyEnded = true;
+ }
+ return result;
+ }
+
+ @Override
+ public void close() throws IOException {
+ other.close();
+ }
+ };
+ }
+ }.sort(out.getName());
+ dir.close();
+ }
+
public void testInvalidFixedLength() throws Exception {
IllegalArgumentException e;
e = expectThrows(IllegalArgumentException.class,
() -> {
new OfflineSorter(null, "foo", OfflineSorter.DEFAULT_COMPARATOR,
- BufferSize.megabytes(1), OfflineSorter.MAX_TEMPFILES, 0);
+ BufferSize.megabytes(1), OfflineSorter.MAX_TEMPFILES, 0, null, 0);
});
assertEquals("valueLength must be 1 .. 32767; got: 0", e.getMessage());
e = expectThrows(IllegalArgumentException.class,
() -> {
new OfflineSorter(null, "foo", OfflineSorter.DEFAULT_COMPARATOR,
- BufferSize.megabytes(1), OfflineSorter.MAX_TEMPFILES, Integer.MAX_VALUE);
+ BufferSize.megabytes(1), OfflineSorter.MAX_TEMPFILES, Integer.MAX_VALUE, null, 0);
});
assertEquals("valueLength must be 1 .. 32767; got: 2147483647", e.getMessage());
}
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/FacetsConfig.java b/lucene/facet/src/java/org/apache/lucene/facet/FacetsConfig.java
index 96db60f4e421..ea67294be52d 100644
--- a/lucene/facet/src/java/org/apache/lucene/facet/FacetsConfig.java
+++ b/lucene/facet/src/java/org/apache/lucene/facet/FacetsConfig.java
@@ -105,7 +105,7 @@ public FacetsConfig() {
*
* @return The default configuration to be used for dimensions that
* are not yet set in the {@link FacetsConfig} */
- protected DimConfig getDefaultDimConfig(){
+ protected DimConfig getDefaultDimConfig() {
return DEFAULT_DIM_CONFIG;
}
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/sortedset/DefaultSortedSetDocValuesReaderState.java b/lucene/facet/src/java/org/apache/lucene/facet/sortedset/DefaultSortedSetDocValuesReaderState.java
index 6bcfa465b474..cff1cca06d8d 100644
--- a/lucene/facet/src/java/org/apache/lucene/facet/sortedset/DefaultSortedSetDocValuesReaderState.java
+++ b/lucene/facet/src/java/org/apache/lucene/facet/sortedset/DefaultSortedSetDocValuesReaderState.java
@@ -18,6 +18,7 @@
import java.io.IOException;
import java.util.Arrays;
+import java.util.Collection;
import java.util.HashMap;
import java.util.Map;
@@ -33,6 +34,8 @@
import org.apache.lucene.index.MultiDocValues.OrdinalMap;
import org.apache.lucene.index.MultiDocValues;
import org.apache.lucene.index.SortedSetDocValues;
+import org.apache.lucene.util.Accountable;
+import org.apache.lucene.util.Accountables;
import org.apache.lucene.util.BytesRef;
/**
@@ -104,11 +107,46 @@ public DefaultSortedSetDocValuesReaderState(IndexReader reader, String field) th
}
}
+ /**
+ * Return the memory usage of this object in bytes. Negative values are illegal.
+ */
+ @Override
+ public long ramBytesUsed() {
+ synchronized (cachedOrdMaps) {
+ long bytes = 0;
+ for (OrdinalMap map : cachedOrdMaps.values()) {
+ bytes += map.ramBytesUsed();
+ }
+
+ return bytes;
+ }
+ }
+
+ /**
+ * Returns nested resources of this class.
+ * The result should be a point-in-time snapshot (to avoid race conditions).
+ * @see Accountables
+ */
+ @Override
+ public Collection getChildResources() {
+ synchronized (cachedOrdMaps) {
+ return Accountables.namedAccountables("DefaultSortedSetDocValuesReaderState", cachedOrdMaps);
+ }
+ }
+
+ @Override
+ public String toString() {
+ return "DefaultSortedSetDocValuesReaderState(field=" + field + " origReader=" + origReader + ")";
+ }
+
/** Return top-level doc values. */
@Override
public SortedSetDocValues getDocValues() throws IOException {
// TODO: this is dup'd from slow composite reader wrapper ... can we factor it out to share?
OrdinalMap map = null;
+ // TODO: why are we lazy about this? It's better if ctor pays the cost, not first query? Oh, but we
+ // call this method from ctor, ok. Also, we only ever store one entry in the map (for key=field) so
+ // why are we using a map?
synchronized (cachedOrdMaps) {
map = cachedOrdMaps.get(field);
if (map == null) {
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/sortedset/SortedSetDocValuesReaderState.java b/lucene/facet/src/java/org/apache/lucene/facet/sortedset/SortedSetDocValuesReaderState.java
index 83ed3f04f25a..546b3198ed25 100644
--- a/lucene/facet/src/java/org/apache/lucene/facet/sortedset/SortedSetDocValuesReaderState.java
+++ b/lucene/facet/src/java/org/apache/lucene/facet/sortedset/SortedSetDocValuesReaderState.java
@@ -21,6 +21,7 @@
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.SortedSetDocValues;
+import org.apache.lucene.util.Accountable;
/** Wraps a {@link IndexReader} and resolves ords
* using existing {@link SortedSetDocValues} APIs without a
@@ -38,7 +39,7 @@
* so you should create it once and re-use that one instance
* for a given {@link IndexReader}. */
-public abstract class SortedSetDocValuesReaderState {
+public abstract class SortedSetDocValuesReaderState implements Accountable {
/** Holds start/end range of ords, which maps to one
* dimension (someday we may generalize it to map to
diff --git a/lucene/facet/src/test/org/apache/lucene/facet/sortedset/TestSortedSetDocValuesFacets.java b/lucene/facet/src/test/org/apache/lucene/facet/sortedset/TestSortedSetDocValuesFacets.java
index 5aed22b4e841..3ad6e683d4fa 100644
--- a/lucene/facet/src/test/org/apache/lucene/facet/sortedset/TestSortedSetDocValuesFacets.java
+++ b/lucene/facet/src/test/org/apache/lucene/facet/sortedset/TestSortedSetDocValuesFacets.java
@@ -18,6 +18,7 @@
import java.io.IOException;
import java.util.ArrayList;
+import java.util.Collection;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
@@ -39,6 +40,7 @@
import org.apache.lucene.search.TermQuery;
import org.apache.lucene.search.TopDocs;
import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.TestUtil;
@@ -178,6 +180,17 @@ public void testSparseFacets() throws Exception {
assertEquals("dim=b path=[] value=2 childCount=2\n bar1 (1)\n bar2 (1)\n", results.get(1).toString());
assertEquals("dim=c path=[] value=1 childCount=1\n baz1 (1)\n", results.get(2).toString());
+ Collection resources = state.getChildResources();
+ assertTrue(state.toString().contains(FacetsConfig.DEFAULT_INDEX_FIELD_NAME));
+ if (searcher.getIndexReader().leaves().size() > 1) {
+ assertTrue(state.ramBytesUsed() > 0);
+ assertFalse(resources.isEmpty());
+ assertTrue(resources.toString().contains(FacetsConfig.DEFAULT_INDEX_FIELD_NAME));
+ } else {
+ assertEquals(0, state.ramBytesUsed());
+ assertTrue(resources.isEmpty());
+ }
+
searcher.getIndexReader().close();
dir.close();
}
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/AllGroupHeadsCollector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/AllGroupHeadsCollector.java
index b5fbdc36f1a8..503b9529bd65 100644
--- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/AllGroupHeadsCollector.java
+++ b/lucene/grouping/src/java/org/apache/lucene/search/grouping/AllGroupHeadsCollector.java
@@ -18,27 +18,62 @@
import java.io.IOException;
import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.search.FieldComparator;
+import org.apache.lucene.search.LeafFieldComparator;
+import org.apache.lucene.search.Scorer;
import org.apache.lucene.search.SimpleCollector;
+import org.apache.lucene.search.Sort;
+import org.apache.lucene.search.SortField;
import org.apache.lucene.util.FixedBitSet;
/**
- * This collector specializes in collecting the most relevant document (group head) for each group that match the query.
+ * This collector specializes in collecting the most relevant document (group head) for each
+ * group that matches the query.
+ *
+ * Clients should create new collectors by calling {@link #newCollector(GroupSelector, Sort)}
*
* @lucene.experimental
*/
@SuppressWarnings({"unchecked","rawtypes"})
public abstract class AllGroupHeadsCollector extends SimpleCollector {
+ private final GroupSelector groupSelector;
+ protected final Sort sort;
+
protected final int[] reversed;
protected final int compIDXEnd;
- protected final TemporalResult temporalResult;
- protected AllGroupHeadsCollector(int numberOfSorts) {
- this.reversed = new int[numberOfSorts];
- this.compIDXEnd = numberOfSorts - 1;
- temporalResult = new TemporalResult();
+ protected Map> heads = new HashMap<>();
+
+ protected LeafReaderContext context;
+ protected Scorer scorer;
+
+ /**
+ * Create a new AllGroupHeadsCollector based on the type of within-group Sort required
+ * @param selector a GroupSelector to define the groups
+ * @param sort the within-group sort to use to choose the group head document
+ * @param the group value type
+ */
+ public static AllGroupHeadsCollector newCollector(GroupSelector selector, Sort sort) {
+ if (sort.equals(Sort.RELEVANCE))
+ return new ScoringGroupHeadsCollector<>(selector, sort);
+ return new SortingGroupHeadsCollector<>(selector, sort);
+ }
+
+ private AllGroupHeadsCollector(GroupSelector selector, Sort sort) {
+ this.groupSelector = selector;
+ this.sort = sort;
+ this.reversed = new int[sort.getSort().length];
+ final SortField[] sortFields = sort.getSort();
+ for (int i = 0; i < sortFields.length; i++) {
+ reversed[i] = sortFields[i].getReverse() ? -1 : 1;
+ }
+ this.compIDXEnd = this.reversed.length - 1;
}
/**
@@ -78,35 +113,28 @@ public int groupHeadsSize() {
return getCollectedGroupHeads().size();
}
- /**
- * Returns the group head and puts it into {@link #temporalResult}.
- * If the group head wasn't encountered before then it will be added to the collected group heads.
- *
- * The {@link TemporalResult#stop} property will be true
if the group head wasn't encountered before
- * otherwise false
.
- *
- * @param doc The document to retrieve the group head for.
- * @throws IOException If I/O related errors occur
- */
- protected abstract void retrieveGroupHeadAndAddIfNotExist(int doc) throws IOException;
-
/**
* Returns the collected group heads.
* Subsequent calls should return the same group heads.
*
* @return the collected group heads
*/
- protected abstract Collection extends GroupHead> getCollectedGroupHeads();
+ protected Collection extends GroupHead> getCollectedGroupHeads() {
+ return heads.values();
+ }
@Override
public void collect(int doc) throws IOException {
- retrieveGroupHeadAndAddIfNotExist(doc);
- if (temporalResult.stop) {
+ groupSelector.advanceTo(doc);
+ T groupValue = groupSelector.currentValue();
+ if (heads.containsKey(groupValue) == false) {
+ groupValue = groupSelector.copyValue();
+ heads.put(groupValue, newGroupHead(doc, groupValue, context, scorer));
return;
}
- GroupHead groupHead = temporalResult.groupHead;
- // Ok now we need to check if the current doc is more relevant then current doc for this group
+ GroupHead groupHead = heads.get(groupValue);
+ // Ok now we need to check if the current doc is more relevant than top doc for this group
for (int compIDX = 0; ; compIDX++) {
final int c = reversed[compIDX] * groupHead.compare(compIDX, doc);
if (c < 0) {
@@ -125,17 +153,33 @@ public void collect(int doc) throws IOException {
groupHead.updateDocHead(doc);
}
- /**
- * Contains the result of group head retrieval.
- * To prevent new object creations of this class for every collect.
- */
- protected class TemporalResult {
+ @Override
+ public boolean needsScores() {
+ return sort.needsScores();
+ }
- public GroupHead groupHead;
- public boolean stop;
+ @Override
+ protected void doSetNextReader(LeafReaderContext context) throws IOException {
+ groupSelector.setNextReader(context);
+ this.context = context;
+ for (GroupHead head : heads.values()) {
+ head.setNextReader(context);
+ }
+ }
+ @Override
+ public void setScorer(Scorer scorer) throws IOException {
+ this.scorer = scorer;
+ for (GroupHead head : heads.values()) {
+ head.setScorer(scorer);
+ }
}
+ /**
+ * Create a new GroupHead for the given group value, initialized with a doc, context and scorer
+ */
+ protected abstract GroupHead newGroupHead(int doc, T value, LeafReaderContext context, Scorer scorer) throws IOException;
+
/**
* Represents a group head. A group head is the most relevant document for a particular group.
* The relevancy is based is usually based on the sort.
@@ -147,11 +191,29 @@ public static abstract class GroupHead {
public final T groupValue;
public int doc;
- protected GroupHead(T groupValue, int doc) {
+ protected int docBase;
+
+ /**
+ * Create a new GroupHead for the given value
+ */
+ protected GroupHead(T groupValue, int doc, int docBase) {
this.groupValue = groupValue;
- this.doc = doc;
+ this.doc = doc + docBase;
+ this.docBase = docBase;
+ }
+
+ /**
+ * Called for each segment
+ */
+ protected void setNextReader(LeafReaderContext ctx) throws IOException {
+ this.docBase = ctx.docBase;
}
+ /**
+ * Called for each segment
+ */
+ protected abstract void setScorer(Scorer scorer) throws IOException;
+
/**
* Compares the specified document for a specified comparator against the current most relevant document.
*
@@ -173,4 +235,117 @@ protected GroupHead(T groupValue, int doc) {
}
+ /**
+ * General implementation using a {@link FieldComparator} to select the group head
+ */
+ private static class SortingGroupHeadsCollector extends AllGroupHeadsCollector {
+
+ protected SortingGroupHeadsCollector(GroupSelector selector, Sort sort) {
+ super(selector, sort);
+ }
+
+ @Override
+ protected GroupHead newGroupHead(int doc, T value, LeafReaderContext ctx, Scorer scorer) throws IOException {
+ return new SortingGroupHead<>(sort, value, doc, ctx, scorer);
+ }
+ }
+
+ private static class SortingGroupHead extends GroupHead {
+
+ final FieldComparator[] comparators;
+ final LeafFieldComparator[] leafComparators;
+
+ protected SortingGroupHead(Sort sort, T groupValue, int doc, LeafReaderContext context, Scorer scorer) throws IOException {
+ super(groupValue, doc, context.docBase);
+ final SortField[] sortFields = sort.getSort();
+ comparators = new FieldComparator[sortFields.length];
+ leafComparators = new LeafFieldComparator[sortFields.length];
+ for (int i = 0; i < sortFields.length; i++) {
+ comparators[i] = sortFields[i].getComparator(1, i);
+ leafComparators[i] = comparators[i].getLeafComparator(context);
+ leafComparators[i].setScorer(scorer);
+ leafComparators[i].copy(0, doc);
+ leafComparators[i].setBottom(0);
+ }
+ }
+
+ @Override
+ public void setNextReader(LeafReaderContext ctx) throws IOException {
+ super.setNextReader(ctx);
+ for (int i = 0; i < comparators.length; i++) {
+ leafComparators[i] = comparators[i].getLeafComparator(ctx);
+ }
+ }
+
+ @Override
+ protected void setScorer(Scorer scorer) throws IOException {
+ for (LeafFieldComparator c : leafComparators) {
+ c.setScorer(scorer);
+ }
+ }
+
+ @Override
+ public int compare(int compIDX, int doc) throws IOException {
+ return leafComparators[compIDX].compareBottom(doc);
+ }
+
+ @Override
+ public void updateDocHead(int doc) throws IOException {
+ for (LeafFieldComparator comparator : leafComparators) {
+ comparator.copy(0, doc);
+ comparator.setBottom(0);
+ }
+ this.doc = doc + docBase;
+ }
+ }
+
+ /**
+ * Specialized implementation for sorting by score
+ */
+ private static class ScoringGroupHeadsCollector extends AllGroupHeadsCollector {
+
+ protected ScoringGroupHeadsCollector(GroupSelector selector, Sort sort) {
+ super(selector, sort);
+ }
+
+ @Override
+ protected GroupHead newGroupHead(int doc, T value, LeafReaderContext context, Scorer scorer) throws IOException {
+ return new ScoringGroupHead<>(scorer, value, doc, context.docBase);
+ }
+ }
+
+ private static class ScoringGroupHead extends GroupHead {
+
+ private Scorer scorer;
+ private float topScore;
+
+ protected ScoringGroupHead(Scorer scorer, T groupValue, int doc, int docBase) throws IOException {
+ super(groupValue, doc, docBase);
+ assert scorer.docID() == doc;
+ this.scorer = scorer;
+ this.topScore = scorer.score();
+ }
+
+ @Override
+ protected void setScorer(Scorer scorer) {
+ this.scorer = scorer;
+ }
+
+ @Override
+ protected int compare(int compIDX, int doc) throws IOException {
+ assert scorer.docID() == doc;
+ assert compIDX == 0;
+ float score = scorer.score();
+ int c = Float.compare(score, topScore);
+ if (c > 0)
+ topScore = score;
+ return c;
+ }
+
+ @Override
+ protected void updateDocHead(int doc) throws IOException {
+ this.doc = doc + docBase;
+ }
+ }
+
}
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/AllGroupsCollector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/AllGroupsCollector.java
index af697af5a843..8434534df4de 100644
--- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/AllGroupsCollector.java
+++ b/lucene/grouping/src/java/org/apache/lucene/search/grouping/AllGroupsCollector.java
@@ -18,23 +18,34 @@
import java.io.IOException;
import java.util.Collection;
+import java.util.HashSet;
+import java.util.Set;
+import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.search.Scorer;
import org.apache.lucene.search.SimpleCollector;
-import org.apache.lucene.util.BytesRef;
/**
* A collector that collects all groups that match the
* query. Only the group value is collected, and the order
* is undefined. This collector does not determine
* the most relevant document of a group.
- *
- * This is an abstract version. Concrete implementations define
- * what a group actually is and how it is internally collected.
*
* @lucene.experimental
*/
-public abstract class AllGroupsCollector extends SimpleCollector {
+public class AllGroupsCollector extends SimpleCollector {
+
+ private final GroupSelector groupSelector;
+
+ private final Set groups = new HashSet();
+
+ /**
+ * Create a new AllGroupsCollector
+ * @param groupSelector the GroupSelector to determine groups
+ */
+ public AllGroupsCollector(GroupSelector groupSelector) {
+ this.groupSelector = groupSelector;
+ }
/**
* Returns the total number of groups for the executed search.
@@ -49,17 +60,30 @@ public int getGroupCount() {
/**
* Returns the group values
*
- * This is an unordered collections of group values. For each group that matched the query there is a {@link BytesRef}
- * representing a group value.
+ * This is an unordered collections of group values.
*
* @return the group values
*/
- public abstract Collection getGroups();
+ public Collection getGroups() {
+ return groups;
+ }
- // Empty not necessary
@Override
public void setScorer(Scorer scorer) throws IOException {}
+ @Override
+ protected void doSetNextReader(LeafReaderContext context) throws IOException {
+ groupSelector.setNextReader(context);
+ }
+
+ @Override
+ public void collect(int doc) throws IOException {
+ groupSelector.advanceTo(doc);
+ if (groups.contains(groupSelector.currentValue()))
+ return;
+ groups.add(groupSelector.copyValue());
+ }
+
@Override
public boolean needsScores() {
return false; // the result is unaffected by relevancy
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/BlockGroupingCollector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/BlockGroupingCollector.java
index c965042a8553..a50fda103d95 100644
--- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/BlockGroupingCollector.java
+++ b/lucene/grouping/src/java/org/apache/lucene/search/grouping/BlockGroupingCollector.java
@@ -50,7 +50,7 @@
* being that the documents in each group must always be
* indexed as a block. This collector also fills in
* TopGroups.totalGroupCount without requiring the separate
- * {@link org.apache.lucene.search.grouping.term.TermAllGroupsCollector}. However, this collector does
+ * {@link org.apache.lucene.search.grouping.AllGroupsCollector}. However, this collector does
* not fill in the groupValue of each group; this field
* will always be null.
*
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/DistinctValuesCollector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/DistinctValuesCollector.java
index 54d752c58766..103b0d26a707 100644
--- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/DistinctValuesCollector.java
+++ b/lucene/grouping/src/java/org/apache/lucene/search/grouping/DistinctValuesCollector.java
@@ -16,10 +16,14 @@
*/
package org.apache.lucene.search.grouping;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
+import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.search.SimpleCollector;
/**
@@ -27,33 +31,99 @@
*
* @lucene.experimental
*/
-public abstract class DistinctValuesCollector extends SimpleCollector {
+public class DistinctValuesCollector extends SecondPassGroupingCollector {
+
+ /**
+ * Create a DistinctValuesCollector
+ * @param groupSelector the group selector to determine the top-level groups
+ * @param groups the top-level groups to collect for
+ * @param valueSelector a group selector to determine which values to collect per-group
+ */
+ public DistinctValuesCollector(GroupSelector groupSelector, Collection> groups,
+ GroupSelector valueSelector) {
+ super(groupSelector, groups, new DistinctValuesReducer<>(valueSelector));
+ }
+
+ private static class ValuesCollector extends SimpleCollector {
+
+ final GroupSelector valueSelector;
+ final Set values = new HashSet<>();
+
+ private ValuesCollector(GroupSelector valueSelector) {
+ this.valueSelector = valueSelector;
+ }
+
+ @Override
+ public void collect(int doc) throws IOException {
+ if (valueSelector.advanceTo(doc) == GroupSelector.State.ACCEPT) {
+ R value = valueSelector.currentValue();
+ if (values.contains(value) == false)
+ values.add(valueSelector.copyValue());
+ }
+ else {
+ if (values.contains(null) == false)
+ values.add(null);
+ }
+ }
+
+ @Override
+ protected void doSetNextReader(LeafReaderContext context) throws IOException {
+ valueSelector.setNextReader(context);
+ }
+
+ @Override
+ public boolean needsScores() {
+ return false;
+ }
+ }
+
+ private static class DistinctValuesReducer extends GroupReducer> {
+
+ final GroupSelector valueSelector;
+
+ private DistinctValuesReducer(GroupSelector valueSelector) {
+ this.valueSelector = valueSelector;
+ }
+
+ @Override
+ public boolean needsScores() {
+ return false;
+ }
+
+ @Override
+ protected ValuesCollector newCollector() {
+ return new ValuesCollector<>(valueSelector);
+ }
+ }
/**
* Returns all unique values for each top N group.
*
* @return all unique values for each top N group
*/
- public abstract List> getGroups();
+ public List> getGroups() {
+ List> counts = new ArrayList<>();
+ for (SearchGroup group : groups) {
+ @SuppressWarnings("unchecked")
+ ValuesCollector vc = (ValuesCollector) groupReducer.getCollector(group.groupValue);
+ counts.add(new GroupCount<>(group.groupValue, vc.values));
+ }
+ return counts;
+ }
/**
* Returned by {@link DistinctValuesCollector#getGroups()},
* representing the value and set of distinct values for the group.
*/
- public static class GroupCount {
+ public static class GroupCount {
public final T groupValue;
- public final Set uniqueValues;
+ public final Set uniqueValues;
- public GroupCount(T groupValue) {
+ public GroupCount(T groupValue, Set values) {
this.groupValue = groupValue;
- this.uniqueValues = new HashSet<>();
+ this.uniqueValues = values;
}
}
- @Override
- public boolean needsScores() {
- return false; // not needed to fetch all values
- }
-
}
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/FirstPassGroupingCollector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/FirstPassGroupingCollector.java
index 02bb1a276379..bd47adbb1f7f 100644
--- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/FirstPassGroupingCollector.java
+++ b/lucene/grouping/src/java/org/apache/lucene/search/grouping/FirstPassGroupingCollector.java
@@ -33,15 +33,16 @@
/** FirstPassGroupingCollector is the first of two passes necessary
* to collect grouped hits. This pass gathers the top N sorted
- * groups. Concrete subclasses define what a group is and how it
- * is internally collected.
+ * groups. Groups are defined by a {@link GroupSelector}
*
* See {@link org.apache.lucene.search.grouping} for more
* details including a full code example.
*
* @lucene.experimental
*/
-abstract public class FirstPassGroupingCollector extends SimpleCollector {
+public class FirstPassGroupingCollector extends SimpleCollector {
+
+ private final GroupSelector groupSelector;
private final FieldComparator>[] comparators;
private final LeafFieldComparator[] leafComparators;
@@ -60,16 +61,18 @@ abstract public class FirstPassGroupingCollector extends SimpleCollector {
/**
* Create the first pass collector.
*
- * @param groupSort The {@link Sort} used to sort the
+ * @param groupSelector a GroupSelector used to defined groups
+ * @param groupSort The {@link Sort} used to sort the
* groups. The top sorted document within each group
* according to groupSort, determines how that group
* sorts against other groups. This must be non-null,
* ie, if you want to groupSort by relevance use
* Sort.RELEVANCE.
- * @param topNGroups How many top groups to keep.
+ * @param topNGroups How many top groups to keep.
*/
@SuppressWarnings({"unchecked", "rawtypes"})
- public FirstPassGroupingCollector(Sort groupSort, int topNGroups) {
+ public FirstPassGroupingCollector(GroupSelector groupSelector, Sort groupSort, int topNGroups) {
+ this.groupSelector = groupSelector;
if (topNGroups < 1) {
throw new IllegalArgumentException("topNGroups must be >= 1 (got " + topNGroups + ")");
}
@@ -133,7 +136,7 @@ public Collection> getTopGroups(int groupOffset, boolean fillFiel
if (upto++ < groupOffset) {
continue;
}
- //System.out.println(" group=" + (group.groupValue == null ? "null" : group.groupValue.utf8ToString()));
+ // System.out.println(" group=" + (group.groupValue == null ? "null" : group.groupValue.toString()));
SearchGroup searchGroup = new SearchGroup<>();
searchGroup.groupValue = group.groupValue;
if (fillFields) {
@@ -155,14 +158,11 @@ public void setScorer(Scorer scorer) throws IOException {
}
}
- @Override
- public void collect(int doc) throws IOException {
- //System.out.println("FP.collect doc=" + doc);
-
+ private boolean isCompetitive(int doc) throws IOException {
// If orderedGroups != null we already have collected N groups and
// can short circuit by comparing this document to the bottom group,
// without having to find what group this document belongs to.
-
+
// Even if this document belongs to a group in the top N, we'll know that
// we don't have to update that group.
@@ -173,7 +173,7 @@ public void collect(int doc) throws IOException {
final int c = reversed[compIDX] * leafComparators[compIDX].compareBottom(doc);
if (c < 0) {
// Definitely not competitive. So don't even bother to continue
- return;
+ return false;
} else if (c > 0) {
// Definitely competitive.
break;
@@ -181,15 +181,24 @@ public void collect(int doc) throws IOException {
// Here c=0. If we're at the last comparator, this doc is not
// competitive, since docs are visited in doc Id order, which means
// this doc cannot compete with any other document in the queue.
- return;
+ return false;
}
}
}
+ return true;
+ }
+
+ @Override
+ public void collect(int doc) throws IOException {
+
+ if (isCompetitive(doc) == false)
+ return;
// TODO: should we add option to mean "ignore docs that
// don't have the group field" (instead of stuffing them
// under null group)?
- final T groupValue = getDocGroupValue(doc);
+ groupSelector.advanceTo(doc);
+ T groupValue = groupSelector.currentValue();
final CollectedSearchGroup group = groupMap.get(groupValue);
@@ -207,7 +216,7 @@ public void collect(int doc) throws IOException {
// Add a new CollectedSearchGroup:
CollectedSearchGroup sg = new CollectedSearchGroup<>();
- sg.groupValue = copyDocGroupValue(groupValue, null);
+ sg.groupValue = groupSelector.copyValue();
sg.comparatorSlot = groupMap.size();
sg.topDoc = docBase + doc;
for (LeafFieldComparator fc : leafComparators) {
@@ -233,7 +242,7 @@ public void collect(int doc) throws IOException {
groupMap.remove(bottomGroup.groupValue);
// reuse the removed CollectedSearchGroup
- bottomGroup.groupValue = copyDocGroupValue(groupValue, bottomGroup.groupValue);
+ bottomGroup.groupValue = groupSelector.copyValue();
bottomGroup.topDoc = docBase + doc;
for (LeafFieldComparator fc : leafComparators) {
@@ -338,25 +347,15 @@ protected void doSetNextReader(LeafReaderContext readerContext) throws IOExcepti
for (int i=0; i getGroupSelector() {
+ return groupSelector;
+ }
}
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/GroupReducer.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/GroupReducer.java
new file mode 100644
index 000000000000..4366e9123f78
--- /dev/null
+++ b/lucene/grouping/src/java/org/apache/lucene/search/grouping/GroupReducer.java
@@ -0,0 +1,112 @@
+/*
+ * 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.grouping;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.search.Collector;
+import org.apache.lucene.search.LeafCollector;
+import org.apache.lucene.search.Scorer;
+
+/**
+ * Concrete implementations of this class define what to collect for individual
+ * groups during the second-pass of a grouping search.
+ *
+ * Each group is assigned a Collector returned by {@link #newCollector()}, and
+ * {@link LeafCollector#collect(int)} is called for each document that is in
+ * a group
+ *
+ * @see SecondPassGroupingCollector
+ *
+ * @param the type of the value used for grouping
+ * @param the type of {@link Collector} used to reduce each group
+ */
+public abstract class GroupReducer {
+
+ private final Map> groups = new HashMap<>();
+
+ /**
+ * Define which groups should be reduced.
+ *
+ * Called by {@link SecondPassGroupingCollector}
+ */
+ public void setGroups(Collection> groups) {
+ for (SearchGroup group : groups) {
+ this.groups.put(group.groupValue, new GroupCollector<>(newCollector()));
+ }
+ }
+
+ /**
+ * Whether or not this reducer requires collected documents to be scored
+ */
+ public abstract boolean needsScores();
+
+ /**
+ * Creates a new Collector for each group
+ */
+ protected abstract C newCollector();
+
+ /**
+ * Get the Collector for a given group
+ */
+ public final C getCollector(T value) {
+ return groups.get(value).collector;
+ }
+
+ /**
+ * Collect a given document into a given group
+ * @throws IOException on error
+ */
+ public final void collect(T value, int doc) throws IOException {
+ GroupCollector collector = groups.get(value);
+ collector.leafCollector.collect(doc);
+ }
+
+ /**
+ * Set the Scorer on all group collectors
+ */
+ public final void setScorer(Scorer scorer) throws IOException {
+ for (GroupCollector collector : groups.values()) {
+ collector.leafCollector.setScorer(scorer);
+ }
+ }
+
+ /**
+ * Called when the parent {@link SecondPassGroupingCollector} moves to a new segment
+ */
+ public final void setNextReader(LeafReaderContext ctx) throws IOException {
+ for (GroupCollector collector : groups.values()) {
+ collector.leafCollector = collector.collector.getLeafCollector(ctx);
+ }
+ }
+
+ private static final class GroupCollector {
+
+ final C collector;
+ LeafCollector leafCollector;
+
+ private GroupCollector(C collector) {
+ this.collector = collector;
+ }
+ }
+
+}
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/GroupSelector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/GroupSelector.java
new file mode 100644
index 000000000000..dbb09329f8ae
--- /dev/null
+++ b/lucene/grouping/src/java/org/apache/lucene/search/grouping/GroupSelector.java
@@ -0,0 +1,73 @@
+/*
+ * 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.grouping;
+
+import java.io.IOException;
+import java.util.Collection;
+
+import org.apache.lucene.index.LeafReaderContext;
+
+/**
+ * Defines a group, for use by grouping collectors
+ *
+ * A GroupSelector acts as an iterator over documents. For each segment, clients
+ * should call {@link #setNextReader(LeafReaderContext)}, and then {@link #advanceTo(int)}
+ * for each matching document.
+ *
+ * @param the type of the group value
+ */
+public abstract class GroupSelector {
+
+ /**
+ * What to do with the current value
+ */
+ public enum State { SKIP, ACCEPT }
+
+ /**
+ * Set the LeafReaderContext
+ */
+ public abstract void setNextReader(LeafReaderContext readerContext) throws IOException;
+
+ /**
+ * Advance the GroupSelector's iterator to the given document
+ */
+ public abstract State advanceTo(int doc) throws IOException;
+
+ /**
+ * Get the group value of the current document
+ *
+ * N.B. this object may be reused, for a persistent version use {@link #copyValue()}
+ */
+ public abstract T currentValue();
+
+ /**
+ * @return a copy of the group value of the current document
+ */
+ public abstract T copyValue();
+
+ /**
+ * Set a restriction on the group values returned by this selector
+ *
+ * If the selector is positioned on a document whose group value is not contained
+ * within this set, then {@link #advanceTo(int)} will return {@link State#SKIP}
+ *
+ * @param groups a set of {@link SearchGroup} objects to limit selections to
+ */
+ public abstract void setGroups(Collection> groups);
+
+}
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/Grouper.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/Grouper.java
deleted file mode 100644
index 2ff79a1331b2..000000000000
--- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/Grouper.java
+++ /dev/null
@@ -1,56 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.lucene.search.grouping;
-
-import java.io.IOException;
-import java.util.Collection;
-
-import org.apache.lucene.search.Sort;
-
-/**
- * A factory object to create first and second-pass collectors, run by a {@link GroupingSearch}
- * @param the type the group value
- */
-public abstract class Grouper {
-
- /**
- * Create a first-pass collector
- * @param sort the order in which groups should be returned
- * @param count how many groups to return
- */
- public abstract FirstPassGroupingCollector getFirstPassCollector(Sort sort, int count) throws IOException;
-
- /**
- * Create an {@link AllGroupsCollector}
- */
- public abstract AllGroupsCollector getAllGroupsCollector();
-
- /**
- * Create an {@link AllGroupHeadsCollector}
- * @param sort a within-group sort order to determine which doc is the group head
- */
- public abstract AllGroupHeadsCollector getGroupHeadsCollector(Sort sort);
-
- /**
- * Create a second-pass collector
- */
- public abstract SecondPassGroupingCollector getSecondPassCollector(
- Collection> groups, Sort groupSort, Sort withinGroupSort,
- int maxDocsPerGroup, boolean getScores, boolean getMaxScores, boolean fillSortFields) throws IOException;
-
-}
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/GroupingSearch.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/GroupingSearch.java
index f4319d578d35..a36917d742ac 100644
--- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/GroupingSearch.java
+++ b/lucene/grouping/src/java/org/apache/lucene/search/grouping/GroupingSearch.java
@@ -30,8 +30,6 @@
import org.apache.lucene.search.Sort;
import org.apache.lucene.search.SortField;
import org.apache.lucene.search.Weight;
-import org.apache.lucene.search.grouping.function.FunctionGrouper;
-import org.apache.lucene.search.grouping.term.TermGrouper;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.mutable.MutableValue;
@@ -43,7 +41,7 @@
*/
public class GroupingSearch {
- private final Grouper grouper;
+ private final GroupSelector grouper;
private final Query groupEndDocs;
private Sort groupSort = Sort.RELEVANCE;
@@ -71,11 +69,7 @@ public class GroupingSearch {
* @param groupField The name of the field to group by.
*/
public GroupingSearch(String groupField) {
- this(new TermGrouper(groupField, 128), null);
- }
-
- public GroupingSearch(String groupField, int initialSize) {
- this(new TermGrouper(groupField, initialSize), null);
+ this(new TermGroupSelector(groupField), null);
}
/**
@@ -86,7 +80,7 @@ public GroupingSearch(String groupField, int initialSize) {
* @param valueSourceContext The context of the specified groupFunction
*/
public GroupingSearch(ValueSource groupFunction, Map, ?> valueSourceContext) {
- this(new FunctionGrouper(groupFunction, valueSourceContext), null);
+ this(new ValueSourceGroupSelector(groupFunction, valueSourceContext), null);
}
/**
@@ -99,7 +93,7 @@ public GroupingSearch(Query groupEndDocs) {
this(null, groupEndDocs);
}
- private GroupingSearch(Grouper grouper, Query groupEndDocs) {
+ private GroupingSearch(GroupSelector grouper, Query groupEndDocs) {
this.grouper = grouper;
this.groupEndDocs = groupEndDocs;
}
@@ -129,10 +123,10 @@ public TopGroups search(IndexSearcher searcher, Query query, int groupOff
protected TopGroups groupByFieldOrFunction(IndexSearcher searcher, Query query, int groupOffset, int groupLimit) throws IOException {
int topN = groupOffset + groupLimit;
- final FirstPassGroupingCollector firstPassCollector = grouper.getFirstPassCollector(groupSort, topN);
- final AllGroupsCollector allGroupsCollector = allGroups ? grouper.getAllGroupsCollector() : null;
+ final FirstPassGroupingCollector firstPassCollector = new FirstPassGroupingCollector(grouper, groupSort, topN);
+ final AllGroupsCollector allGroupsCollector = allGroups ? new AllGroupsCollector(grouper) : null;
final AllGroupHeadsCollector allGroupHeadsCollector
- = allGroupHeads ? grouper.getGroupHeadsCollector(sortWithinGroup) : null;
+ = allGroupHeads ? AllGroupHeadsCollector.newCollector(grouper, sortWithinGroup) : null;
final Collector firstRound = MultiCollector.wrap(firstPassCollector, allGroupsCollector, allGroupHeadsCollector);
@@ -158,8 +152,8 @@ protected TopGroups groupByFieldOrFunction(IndexSearcher searcher, Query query,
}
int topNInsideGroup = groupDocsOffset + groupDocsLimit;
- SecondPassGroupingCollector secondPassCollector
- = grouper.getSecondPassCollector(topSearchGroups, groupSort, sortWithinGroup, topNInsideGroup,
+ TopGroupsCollector secondPassCollector
+ = new TopGroupsCollector(grouper, topSearchGroups, groupSort, sortWithinGroup, topNInsideGroup,
includeScores, includeMaxScore, fillSortFields);
if (cachedCollector != null && cachedCollector.isCached()) {
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/SecondPassGroupingCollector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/SecondPassGroupingCollector.java
index f8feb756f540..c54c8eee4a81 100644
--- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/SecondPassGroupingCollector.java
+++ b/lucene/grouping/src/java/org/apache/lucene/search/grouping/SecondPassGroupingCollector.java
@@ -18,152 +18,82 @@
import java.io.IOException;
import java.util.Collection;
-import java.util.HashMap;
-import java.util.Map;
import java.util.Objects;
import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.search.LeafCollector;
import org.apache.lucene.search.Scorer;
import org.apache.lucene.search.SimpleCollector;
-import org.apache.lucene.search.Sort;
-import org.apache.lucene.search.TopDocs;
-import org.apache.lucene.search.TopDocsCollector;
-import org.apache.lucene.search.TopFieldCollector;
-import org.apache.lucene.search.TopScoreDocCollector;
/**
- * SecondPassGroupingCollector is the second of two passes
- * necessary to collect grouped docs. This pass gathers the
- * top N documents per top group computed from the
- * first pass. Concrete subclasses define what a group is and how it
- * is internally collected.
+ * SecondPassGroupingCollector runs over an already collected set of
+ * groups, further applying a {@link GroupReducer} to each group
*
- * See {@link org.apache.lucene.search.grouping} for more
- * details including a full code example.
+ * @see TopGroupsCollector
+ * @see DistinctValuesCollector
*
* @lucene.experimental
*/
-public abstract class SecondPassGroupingCollector extends SimpleCollector {
+public class SecondPassGroupingCollector extends SimpleCollector {
- private final Collection> groups;
- private final Sort groupSort;
- private final Sort withinGroupSort;
- private final int maxDocsPerGroup;
- private final boolean needsScores;
- protected final Map> groupMap;
+ protected final GroupSelector groupSelector;
+ protected final Collection> groups;
+ protected final GroupReducer groupReducer;
- protected SearchGroupDocs[] groupDocs;
+ protected int totalHitCount;
+ protected int totalGroupedHitCount;
- private int totalHitCount;
- private int totalGroupedHitCount;
-
- public SecondPassGroupingCollector(Collection> groups, Sort groupSort, Sort withinGroupSort,
- int maxDocsPerGroup, boolean getScores, boolean getMaxScores, boolean fillSortFields)
- throws IOException {
+ /**
+ * Create a new SecondPassGroupingCollector
+ * @param groupSelector the GroupSelector that defines groups for this search
+ * @param groups the groups to collect documents for
+ * @param reducer the reducer to apply to each group
+ */
+ public SecondPassGroupingCollector(GroupSelector groupSelector, Collection> groups, GroupReducer reducer) {
//System.out.println("SP init");
if (groups.isEmpty()) {
throw new IllegalArgumentException("no groups to collect (groups is empty)");
}
+ this.groupSelector = Objects.requireNonNull(groupSelector);
+ this.groupSelector.setGroups(groups);
+
this.groups = Objects.requireNonNull(groups);
- this.groupSort = Objects.requireNonNull(groupSort);
- this.withinGroupSort = Objects.requireNonNull(withinGroupSort);
- this.maxDocsPerGroup = maxDocsPerGroup;
- this.needsScores = getScores || getMaxScores || withinGroupSort.needsScores();
+ this.groupReducer = reducer;
+ reducer.setGroups(groups);
+ }
- this.groupMap = new HashMap<>(groups.size());
- for (SearchGroup group : groups) {
- //System.out.println(" prep group=" + (group.groupValue == null ? "null" : group.groupValue.utf8ToString()));
- final TopDocsCollector> collector;
- if (withinGroupSort.equals(Sort.RELEVANCE)) { // optimize to use TopScoreDocCollector
- // Sort by score
- collector = TopScoreDocCollector.create(maxDocsPerGroup);
- } else {
- // Sort by fields
- collector = TopFieldCollector.create(withinGroupSort, maxDocsPerGroup, fillSortFields, getScores, getMaxScores);
- }
- groupMap.put(group.groupValue, new SearchGroupDocs<>(group.groupValue, collector));
- }
+ /**
+ * @return the GroupSelector used in this collector
+ */
+ public GroupSelector getGroupSelector() {
+ return groupSelector;
}
@Override
public boolean needsScores() {
- return needsScores;
+ return groupReducer.needsScores();
}
@Override
public void setScorer(Scorer scorer) throws IOException {
- for (SearchGroupDocs group : groupMap.values()) {
- group.leafCollector.setScorer(scorer);
- }
+ groupReducer.setScorer(scorer);
}
@Override
public void collect(int doc) throws IOException {
totalHitCount++;
- SearchGroupDocs group = retrieveGroup(doc);
- if (group != null) {
- totalGroupedHitCount++;
- group.leafCollector.collect(doc);
- }
+ if (groupSelector.advanceTo(doc) == GroupSelector.State.SKIP)
+ return;
+ totalGroupedHitCount++;
+ T value = groupSelector.currentValue();
+ groupReducer.collect(value, doc);
}
- /**
- * Returns the group the specified doc belongs to or null
if no group could be retrieved.
- *
- * @param doc The specified doc
- * @return the group the specified doc belongs to or null
if no group could be retrieved
- * @throws IOException If an I/O related error occurred
- */
- protected abstract SearchGroupDocs retrieveGroup(int doc) throws IOException;
-
@Override
protected void doSetNextReader(LeafReaderContext readerContext) throws IOException {
- //System.out.println("SP.setNextReader");
- for (SearchGroupDocs group : groupMap.values()) {
- group.leafCollector = group.collector.getLeafCollector(readerContext);
- }
- }
-
- public TopGroups getTopGroups(int withinGroupOffset) {
- @SuppressWarnings({"unchecked","rawtypes"})
- final GroupDocs[] groupDocsResult = (GroupDocs[]) new GroupDocs[groups.size()];
-
- int groupIDX = 0;
- float maxScore = Float.MIN_VALUE;
- for(SearchGroup> group : groups) {
- final SearchGroupDocs groupDocs = groupMap.get(group.groupValue);
- final TopDocs topDocs = groupDocs.collector.topDocs(withinGroupOffset, maxDocsPerGroup);
- groupDocsResult[groupIDX++] = new GroupDocs<>(Float.NaN,
- topDocs.getMaxScore(),
- topDocs.totalHits,
- topDocs.scoreDocs,
- groupDocs.groupValue,
- group.sortValues);
- maxScore = Math.max(maxScore, topDocs.getMaxScore());
- }
-
- return new TopGroups<>(groupSort.getSort(),
- withinGroupSort.getSort(),
- totalHitCount, totalGroupedHitCount, groupDocsResult,
- maxScore);
+ groupReducer.setNextReader(readerContext);
+ groupSelector.setNextReader(readerContext);
}
-
- // TODO: merge with SearchGroup or not?
- // ad: don't need to build a new hashmap
- // disad: blows up the size of SearchGroup if we need many of them, and couples implementations
- public class SearchGroupDocs {
-
- public final T groupValue;
- public final TopDocsCollector> collector;
- public LeafCollector leafCollector;
-
- public SearchGroupDocs(T groupValue, TopDocsCollector> collector) {
- this.groupValue = groupValue;
- this.collector = collector;
- }
- }
}
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermGroupFacetCollector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/TermGroupFacetCollector.java
similarity index 97%
rename from lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermGroupFacetCollector.java
rename to lucene/grouping/src/java/org/apache/lucene/search/grouping/TermGroupFacetCollector.java
index cee327cbf473..39d28a579962 100644
--- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermGroupFacetCollector.java
+++ b/lucene/grouping/src/java/org/apache/lucene/search/grouping/TermGroupFacetCollector.java
@@ -14,7 +14,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package org.apache.lucene.search.grouping.term;
+package org.apache.lucene.search.grouping;
import java.io.IOException;
import java.util.ArrayList;
@@ -25,7 +25,6 @@
import org.apache.lucene.index.SortedDocValues;
import org.apache.lucene.index.SortedSetDocValues;
import org.apache.lucene.index.TermsEnum;
-import org.apache.lucene.search.grouping.GroupFacetCollector;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefBuilder;
import org.apache.lucene.util.SentinelIntSet;
@@ -401,15 +400,15 @@ protected void nextTerm() throws IOException {
}
}
}
-}
-class GroupedFacetHit {
+ private static class GroupedFacetHit {
- final BytesRef groupValue;
- final BytesRef facetValue;
+ final BytesRef groupValue;
+ final BytesRef facetValue;
- GroupedFacetHit(BytesRef groupValue, BytesRef facetValue) {
- this.groupValue = groupValue;
- this.facetValue = facetValue;
+ GroupedFacetHit(BytesRef groupValue, BytesRef facetValue) {
+ this.groupValue = groupValue;
+ this.facetValue = facetValue;
+ }
}
}
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/TermGroupSelector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/TermGroupSelector.java
new file mode 100644
index 000000000000..5b8f77c2ce83
--- /dev/null
+++ b/lucene/grouping/src/java/org/apache/lucene/search/grouping/TermGroupSelector.java
@@ -0,0 +1,114 @@
+/*
+ * 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.grouping;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.lucene.index.DocValues;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.SortedDocValues;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefHash;
+
+/**
+ * A GroupSelector implementation that groups via SortedDocValues
+ */
+public class TermGroupSelector extends GroupSelector {
+
+ private final String field;
+ private final BytesRefHash values = new BytesRefHash();
+ private final Map ordsToGroupIds = new HashMap<>();
+
+ private SortedDocValues docValues;
+ private int groupId;
+
+ private boolean secondPass;
+ private boolean includeEmpty;
+
+ /**
+ * Create a new TermGroupSelector
+ * @param field the SortedDocValues field to use for grouping
+ */
+ public TermGroupSelector(String field) {
+ this.field = field;
+ }
+
+ @Override
+ public void setNextReader(LeafReaderContext readerContext) throws IOException {
+ this.docValues = DocValues.getSorted(readerContext.reader(), field);
+ this.ordsToGroupIds.clear();
+ BytesRef scratch = new BytesRef();
+ for (int i = 0; i < values.size(); i++) {
+ values.get(i, scratch);
+ int ord = this.docValues.lookupTerm(scratch);
+ if (ord >= 0)
+ ordsToGroupIds.put(ord, i);
+ }
+ }
+
+ @Override
+ public State advanceTo(int doc) throws IOException {
+ if (this.docValues.advanceExact(doc) == false) {
+ groupId = -1;
+ return includeEmpty ? State.ACCEPT : State.SKIP;
+ }
+ int ord = docValues.ordValue();
+ if (ordsToGroupIds.containsKey(ord)) {
+ groupId = ordsToGroupIds.get(ord);
+ return State.ACCEPT;
+ }
+ if (secondPass)
+ return State.SKIP;
+ groupId = values.add(docValues.binaryValue());
+ ordsToGroupIds.put(ord, groupId);
+ return State.ACCEPT;
+ }
+
+ private BytesRef scratch = new BytesRef();
+
+ @Override
+ public BytesRef currentValue() {
+ if (groupId == -1)
+ return null;
+ values.get(groupId, scratch);
+ return scratch;
+ }
+
+ @Override
+ public BytesRef copyValue() {
+ if (groupId == -1)
+ return null;
+ return BytesRef.deepCopyOf(currentValue());
+ }
+
+ @Override
+ public void setGroups(Collection> searchGroups) {
+ this.values.clear();
+ this.values.reinit();
+ for (SearchGroup sg : searchGroups) {
+ if (sg.groupValue == null)
+ includeEmpty = true;
+ else
+ this.values.add(sg.groupValue);
+ }
+ this.secondPass = true;
+ }
+}
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/TopGroupsCollector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/TopGroupsCollector.java
new file mode 100644
index 000000000000..b6c71d37a8d2
--- /dev/null
+++ b/lucene/grouping/src/java/org/apache/lucene/search/grouping/TopGroupsCollector.java
@@ -0,0 +1,116 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.search.grouping;
+
+import java.util.Collection;
+import java.util.Objects;
+import java.util.function.Supplier;
+
+import org.apache.lucene.search.Sort;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.search.TopDocsCollector;
+import org.apache.lucene.search.TopFieldCollector;
+import org.apache.lucene.search.TopScoreDocCollector;
+
+/**
+ * A second-pass collector that collects the TopDocs for each group, and
+ * returns them as a {@link TopGroups} object
+ *
+ * @param the type of the group value
+ */
+public class TopGroupsCollector extends SecondPassGroupingCollector {
+
+ private final Sort groupSort;
+ private final Sort withinGroupSort;
+ private final int maxDocsPerGroup;
+
+ /**
+ * Create a new TopGroupsCollector
+ * @param groupSelector the group selector used to define groups
+ * @param groups the groups to collect TopDocs for
+ * @param groupSort the order in which groups are returned
+ * @param withinGroupSort the order in which documents are sorted in each group
+ * @param maxDocsPerGroup the maximum number of docs to collect for each group
+ * @param getScores if true, record the scores of all docs in each group
+ * @param getMaxScores if true, record the maximum score for each group
+ * @param fillSortFields if true, record the sort field values for all docs
+ */
+ public TopGroupsCollector(GroupSelector groupSelector, Collection> groups, Sort groupSort, Sort withinGroupSort,
+ int maxDocsPerGroup, boolean getScores, boolean getMaxScores, boolean fillSortFields) {
+ super(groupSelector, groups,
+ new TopDocsReducer<>(withinGroupSort, maxDocsPerGroup, getScores, getMaxScores, fillSortFields));
+ this.groupSort = Objects.requireNonNull(groupSort);
+ this.withinGroupSort = Objects.requireNonNull(withinGroupSort);
+ this.maxDocsPerGroup = maxDocsPerGroup;
+
+ }
+
+ private static class TopDocsReducer extends GroupReducer> {
+
+ private final Supplier> supplier;
+ private final boolean needsScores;
+
+ TopDocsReducer(Sort withinGroupSort,
+ int maxDocsPerGroup, boolean getScores, boolean getMaxScores, boolean fillSortFields) {
+ this.needsScores = getScores || getMaxScores || withinGroupSort.needsScores();
+ this.supplier = withinGroupSort == Sort.RELEVANCE ?
+ () -> TopScoreDocCollector.create(maxDocsPerGroup) :
+ () -> TopFieldCollector.create(withinGroupSort, maxDocsPerGroup, fillSortFields, getScores, getMaxScores);
+ }
+
+ @Override
+ public boolean needsScores() {
+ return needsScores;
+ }
+
+ @Override
+ protected TopDocsCollector> newCollector() {
+ return supplier.get();
+ }
+ }
+
+ /**
+ * Get the TopGroups recorded by this collector
+ * @param withinGroupOffset the offset within each group to start collecting documents
+ */
+ public TopGroups getTopGroups(int withinGroupOffset) {
+ @SuppressWarnings({"unchecked","rawtypes"})
+ final GroupDocs[] groupDocsResult = (GroupDocs[]) new GroupDocs[groups.size()];
+
+ int groupIDX = 0;
+ float maxScore = Float.MIN_VALUE;
+ for(SearchGroup group : groups) {
+ TopDocsCollector> collector = (TopDocsCollector>) groupReducer.getCollector(group.groupValue);
+ final TopDocs topDocs = collector.topDocs(withinGroupOffset, maxDocsPerGroup);
+ groupDocsResult[groupIDX++] = new GroupDocs<>(Float.NaN,
+ topDocs.getMaxScore(),
+ topDocs.totalHits,
+ topDocs.scoreDocs,
+ group.groupValue,
+ group.sortValues);
+ maxScore = Math.max(maxScore, topDocs.getMaxScore());
+ }
+
+ return new TopGroups<>(groupSort.getSort(),
+ withinGroupSort.getSort(),
+ totalHitCount, totalGroupedHitCount, groupDocsResult,
+ maxScore);
+ }
+
+
+}
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/ValueSourceGroupSelector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/ValueSourceGroupSelector.java
new file mode 100644
index 000000000000..249016018855
--- /dev/null
+++ b/lucene/grouping/src/java/org/apache/lucene/search/grouping/ValueSourceGroupSelector.java
@@ -0,0 +1,86 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.search.grouping;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.queries.function.FunctionValues;
+import org.apache.lucene.queries.function.ValueSource;
+import org.apache.lucene.util.mutable.MutableValue;
+
+/**
+ * A GroupSelector that groups via a ValueSource
+ */
+public class ValueSourceGroupSelector extends GroupSelector {
+
+ private final ValueSource valueSource;
+ private final Map, ?> context;
+
+ private Set secondPassGroups;
+
+ /**
+ * Create a new ValueSourceGroupSelector
+ * @param valueSource the ValueSource to group by
+ * @param context a context map for the ValueSource
+ */
+ public ValueSourceGroupSelector(ValueSource valueSource, Map, ?> context) {
+ this.valueSource = valueSource;
+ this.context = context;
+ }
+
+ private FunctionValues.ValueFiller filler;
+
+ @Override
+ public void setNextReader(LeafReaderContext readerContext) throws IOException {
+ FunctionValues values = valueSource.getValues(context, readerContext);
+ this.filler = values.getValueFiller();
+ }
+
+ @Override
+ public State advanceTo(int doc) throws IOException {
+ this.filler.fillValue(doc);
+ if (secondPassGroups != null) {
+ if (secondPassGroups.contains(filler.getValue()) == false)
+ return State.SKIP;
+ }
+ return State.ACCEPT;
+ }
+
+ @Override
+ public MutableValue currentValue() {
+ return filler.getValue();
+ }
+
+ @Override
+ public MutableValue copyValue() {
+ return filler.getValue().duplicate();
+ }
+
+ @Override
+ public void setGroups(Collection> searchGroups) {
+ secondPassGroups = new HashSet<>();
+ for (SearchGroup group : searchGroups) {
+ secondPassGroups.add(group.groupValue);
+ }
+ }
+}
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionAllGroupHeadsCollector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionAllGroupHeadsCollector.java
deleted file mode 100644
index f4d46682497a..000000000000
--- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionAllGroupHeadsCollector.java
+++ /dev/null
@@ -1,159 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.lucene.search.grouping.function;
-
-import java.io.IOException;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.Map;
-
-import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.queries.function.FunctionValues;
-import org.apache.lucene.queries.function.ValueSource;
-import org.apache.lucene.search.FieldComparator;
-import org.apache.lucene.search.LeafFieldComparator;
-import org.apache.lucene.search.Scorer;
-import org.apache.lucene.search.Sort;
-import org.apache.lucene.search.SortField;
-import org.apache.lucene.search.grouping.AllGroupHeadsCollector;
-import org.apache.lucene.util.mutable.MutableValue;
-
-/**
- * An implementation of {@link AllGroupHeadsCollector} for retrieving the most relevant groups when grouping
- * by {@link ValueSource}.
- *
- * @lucene.experimental
- */
-public class FunctionAllGroupHeadsCollector extends AllGroupHeadsCollector {
-
- private final ValueSource groupBy;
- private final Map, ?> vsContext;
- private final Map groups;
- private final Sort sortWithinGroup;
-
- private FunctionValues.ValueFiller filler;
- private MutableValue mval;
- private LeafReaderContext readerContext;
- private Scorer scorer;
-
- /**
- * Constructs a {@link FunctionAllGroupHeadsCollector} instance.
- *
- * @param groupBy The {@link ValueSource} to group by
- * @param vsContext The ValueSource context
- * @param sortWithinGroup The sort within a group
- */
- public FunctionAllGroupHeadsCollector(ValueSource groupBy, Map, ?> vsContext, Sort sortWithinGroup) {
- super(sortWithinGroup.getSort().length);
- groups = new HashMap<>();
- this.sortWithinGroup = sortWithinGroup;
- this.groupBy = groupBy;
- this.vsContext = vsContext;
-
- final SortField[] sortFields = sortWithinGroup.getSort();
- for (int i = 0; i < sortFields.length; i++) {
- reversed[i] = sortFields[i].getReverse() ? -1 : 1;
- }
- }
-
- @Override
- protected void retrieveGroupHeadAndAddIfNotExist(int doc) throws IOException {
- filler.fillValue(doc);
- FunctionGroupHead groupHead = groups.get(mval);
- if (groupHead == null) {
- MutableValue groupValue = mval.duplicate();
- groupHead = new FunctionGroupHead(groupValue, sortWithinGroup, doc);
- groups.put(groupValue, groupHead);
- temporalResult.stop = true;
- } else {
- temporalResult.stop = false;
- }
- this.temporalResult.groupHead = groupHead;
- }
-
- @Override
- protected Collection getCollectedGroupHeads() {
- return groups.values();
- }
-
- @Override
- public void setScorer(Scorer scorer) throws IOException {
- this.scorer = scorer;
- for (FunctionGroupHead groupHead : groups.values()) {
- for (LeafFieldComparator comparator : groupHead.leafComparators) {
- comparator.setScorer(scorer);
- }
- }
- }
-
- @Override
- protected void doSetNextReader(LeafReaderContext context) throws IOException {
- this.readerContext = context;
- FunctionValues values = groupBy.getValues(vsContext, context);
- filler = values.getValueFiller();
- mval = filler.getValue();
-
- for (FunctionGroupHead groupHead : groups.values()) {
- for (int i = 0; i < groupHead.comparators.length; i++) {
- groupHead.leafComparators[i] = groupHead.comparators[i].getLeafComparator(context);
- }
- }
- }
-
- /** Holds current head document for a single group.
- *
- * @lucene.experimental */
- public class FunctionGroupHead extends AllGroupHeadsCollector.GroupHead {
-
- final FieldComparator>[] comparators;
- final LeafFieldComparator[] leafComparators;
-
- @SuppressWarnings({"unchecked","rawtypes"})
- private FunctionGroupHead(MutableValue groupValue, Sort sort, int doc) throws IOException {
- super(groupValue, doc + readerContext.docBase);
- final SortField[] sortFields = sort.getSort();
- comparators = new FieldComparator[sortFields.length];
- leafComparators = new LeafFieldComparator[sortFields.length];
- for (int i = 0; i < sortFields.length; i++) {
- comparators[i] = sortFields[i].getComparator(1, i);
- leafComparators[i] = comparators[i].getLeafComparator(readerContext);
- leafComparators[i].setScorer(scorer);
- leafComparators[i].copy(0, doc);
- leafComparators[i].setBottom(0);
- }
- }
-
- @Override
- public int compare(int compIDX, int doc) throws IOException {
- return leafComparators[compIDX].compareBottom(doc);
- }
-
- @Override
- public void updateDocHead(int doc) throws IOException {
- for (LeafFieldComparator comparator : leafComparators) {
- comparator.copy(0, doc);
- comparator.setBottom(0);
- }
- this.doc = doc + readerContext.docBase;
- }
- }
-
- @Override
- public boolean needsScores() {
- return sortWithinGroup.needsScores();
- }
-}
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionAllGroupsCollector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionAllGroupsCollector.java
deleted file mode 100644
index 1609d4d7bcff..000000000000
--- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionAllGroupsCollector.java
+++ /dev/null
@@ -1,82 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.lucene.search.grouping.function;
-
-import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.queries.function.FunctionValues;
-import org.apache.lucene.queries.function.ValueSource;
-import org.apache.lucene.search.grouping.AllGroupsCollector;
-import org.apache.lucene.util.mutable.MutableValue;
-
-import java.io.IOException;
-import java.util.Collection;
-import java.util.Map;
-import java.util.SortedSet;
-import java.util.TreeSet;
-
-/**
- * A collector that collects all groups that match the
- * query. Only the group value is collected, and the order
- * is undefined. This collector does not determine
- * the most relevant document of a group.
- *
- * Implementation detail: Uses {@link ValueSource} and {@link FunctionValues} to retrieve the
- * field values to group by.
- *
- * @lucene.experimental
- */
-public class FunctionAllGroupsCollector extends AllGroupsCollector {
-
- private final Map, ?> vsContext;
- private final ValueSource groupBy;
- private final SortedSet groups = new TreeSet<>();
-
- private FunctionValues.ValueFiller filler;
- private MutableValue mval;
-
- /**
- * Constructs a {@link FunctionAllGroupsCollector} instance.
- *
- * @param groupBy The {@link ValueSource} to group by
- * @param vsContext The ValueSource context
- */
- public FunctionAllGroupsCollector(ValueSource groupBy, Map, ?> vsContext) {
- this.vsContext = vsContext;
- this.groupBy = groupBy;
- }
-
- @Override
- public Collection getGroups() {
- return groups;
- }
-
- @Override
- public void collect(int doc) throws IOException {
- filler.fillValue(doc);
- if (!groups.contains(mval)) {
- groups.add(mval.duplicate());
- }
- }
-
- @Override
- protected void doSetNextReader(LeafReaderContext context) throws IOException {
- FunctionValues values = groupBy.getValues(vsContext, context);
- filler = values.getValueFiller();
- mval = filler.getValue();
- }
-
-}
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionDistinctValuesCollector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionDistinctValuesCollector.java
deleted file mode 100644
index 69ead076d6fc..000000000000
--- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionDistinctValuesCollector.java
+++ /dev/null
@@ -1,85 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.lucene.search.grouping.function;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.LinkedHashMap;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.queries.function.FunctionValues;
-import org.apache.lucene.queries.function.ValueSource;
-import org.apache.lucene.search.grouping.DistinctValuesCollector;
-import org.apache.lucene.search.grouping.SearchGroup;
-import org.apache.lucene.util.mutable.MutableValue;
-
-/**
- * Function based implementation of {@link DistinctValuesCollector}.
- *
- * @lucene.experimental
- */
-public class FunctionDistinctValuesCollector extends DistinctValuesCollector {
-
- private final Map, ?> vsContext;
- private final ValueSource groupSource;
- private final ValueSource countSource;
- private final Map> groupMap;
-
- private FunctionValues.ValueFiller groupFiller;
- private FunctionValues.ValueFiller countFiller;
- private MutableValue groupMval;
- private MutableValue countMval;
-
- public FunctionDistinctValuesCollector(Map, ?> vsContext, ValueSource groupSource, ValueSource countSource, Collection> groups) {
- this.vsContext = vsContext;
- this.groupSource = groupSource;
- this.countSource = countSource;
- groupMap = new LinkedHashMap<>();
- for (SearchGroup group : groups) {
- groupMap.put(group.groupValue, new GroupCount<>(group.groupValue));
- }
- }
-
- @Override
- public List> getGroups() {
- return new ArrayList<>(groupMap.values());
- }
-
- @Override
- public void collect(int doc) throws IOException {
- groupFiller.fillValue(doc);
- GroupCount groupCount = groupMap.get(groupMval);
- if (groupCount != null) {
- countFiller.fillValue(doc);
- groupCount.uniqueValues.add(countMval.duplicate());
- }
- }
-
- @Override
- protected void doSetNextReader(LeafReaderContext context) throws IOException {
- FunctionValues values = groupSource.getValues(vsContext, context);
- groupFiller = values.getValueFiller();
- groupMval = groupFiller.getValue();
- values = countSource.getValues(vsContext, context);
- countFiller = values.getValueFiller();
- countMval = countFiller.getValue();
- }
-
-}
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionFirstPassGroupingCollector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionFirstPassGroupingCollector.java
deleted file mode 100644
index 85376e6a066f..000000000000
--- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionFirstPassGroupingCollector.java
+++ /dev/null
@@ -1,86 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.lucene.search.grouping.function;
-
-import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.queries.function.FunctionValues;
-import org.apache.lucene.queries.function.ValueSource;
-import org.apache.lucene.search.Sort;
-import org.apache.lucene.search.grouping.FirstPassGroupingCollector;
-import org.apache.lucene.util.mutable.MutableValue;
-
-import java.io.IOException;
-import java.util.Map;
-
-/**
- * Concrete implementation of {@link FirstPassGroupingCollector} that groups based on
- * {@link ValueSource} instances.
- *
- * @lucene.experimental
- */
-public class FunctionFirstPassGroupingCollector extends FirstPassGroupingCollector {
-
- private final ValueSource groupByVS;
- private final Map, ?> vsContext;
-
- private FunctionValues.ValueFiller filler;
- private MutableValue mval;
-
- /**
- * Creates a first pass collector.
- *
- * @param groupByVS The {@link ValueSource} instance to group by
- * @param vsContext The ValueSource context
- * @param groupSort The {@link Sort} used to sort the
- * groups. The top sorted document within each group
- * according to groupSort, determines how that group
- * sorts against other groups. This must be non-null,
- * ie, if you want to groupSort by relevance use
- * Sort.RELEVANCE.
- * @param topNGroups How many top groups to keep.
- * @throws IOException When I/O related errors occur
- */
- public FunctionFirstPassGroupingCollector(ValueSource groupByVS, Map, ?> vsContext, Sort groupSort, int topNGroups) throws IOException {
- super(groupSort, topNGroups);
- this.groupByVS = groupByVS;
- this.vsContext = vsContext;
- }
-
- @Override
- protected MutableValue getDocGroupValue(int doc) throws IOException {
- filler.fillValue(doc);
- return mval;
- }
-
- @Override
- protected MutableValue copyDocGroupValue(MutableValue groupValue, MutableValue reuse) {
- if (reuse != null) {
- reuse.copy(groupValue);
- return reuse;
- }
- return groupValue.duplicate();
- }
-
- @Override
- protected void doSetNextReader(LeafReaderContext readerContext) throws IOException {
- super.doSetNextReader(readerContext);
- FunctionValues values = groupByVS.getValues(vsContext, readerContext);
- filler = values.getValueFiller();
- mval = filler.getValue();
- }
-
-}
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionGrouper.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionGrouper.java
deleted file mode 100644
index 5204dc266570..000000000000
--- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionGrouper.java
+++ /dev/null
@@ -1,69 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.lucene.search.grouping.function;
-
-import java.io.IOException;
-import java.util.Collection;
-import java.util.Map;
-
-import org.apache.lucene.queries.function.ValueSource;
-import org.apache.lucene.search.Sort;
-import org.apache.lucene.search.grouping.AllGroupHeadsCollector;
-import org.apache.lucene.search.grouping.AllGroupsCollector;
-import org.apache.lucene.search.grouping.FirstPassGroupingCollector;
-import org.apache.lucene.search.grouping.SecondPassGroupingCollector;
-import org.apache.lucene.search.grouping.Grouper;
-import org.apache.lucene.search.grouping.SearchGroup;
-import org.apache.lucene.util.mutable.MutableValue;
-
-/**
- * Collector factory for grouping by ValueSource
- */
-public class FunctionGrouper extends Grouper {
-
- private final ValueSource valueSource;
- private final Map, ?> context;
-
- /**
- * Create a Grouper for the provided ValueSource and context
- */
- public FunctionGrouper(ValueSource valueSource, Map, ?> context) {
- this.valueSource = valueSource;
- this.context = context;
- }
-
- @Override
- public FirstPassGroupingCollector getFirstPassCollector(Sort sort, int count) throws IOException {
- return new FunctionFirstPassGroupingCollector(valueSource, context, sort, count);
- }
-
- @Override
- public AllGroupHeadsCollector getGroupHeadsCollector(Sort sort) {
- return new FunctionAllGroupHeadsCollector(valueSource, context, sort);
- }
-
- @Override
- public AllGroupsCollector getAllGroupsCollector() {
- return new FunctionAllGroupsCollector(valueSource, context);
- }
-
- @Override
- public SecondPassGroupingCollector getSecondPassCollector(Collection> searchGroups, Sort groupSort, Sort withinGroupSort, int maxDocsPerGroup, boolean getScores, boolean getMaxScores, boolean fillSortFields) throws IOException {
- return new FunctionSecondPassGroupingCollector(searchGroups, groupSort, withinGroupSort, maxDocsPerGroup, getScores, getMaxScores, fillSortFields, valueSource, context);
- }
-}
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionSecondPassGroupingCollector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionSecondPassGroupingCollector.java
deleted file mode 100644
index 45f2b37887ec..000000000000
--- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionSecondPassGroupingCollector.java
+++ /dev/null
@@ -1,80 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.lucene.search.grouping.function;
-
-import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.queries.function.FunctionValues;
-import org.apache.lucene.queries.function.ValueSource;
-import org.apache.lucene.search.Sort;
-import org.apache.lucene.search.grouping.SecondPassGroupingCollector;
-import org.apache.lucene.search.grouping.SearchGroup;
-import org.apache.lucene.util.mutable.MutableValue;
-import org.apache.lucene.search.grouping.TopGroups; //javadoc
-
-import java.io.IOException;
-import java.util.Collection;
-import java.util.Map;
-
-/**
- * Concrete implementation of {@link SecondPassGroupingCollector} that groups based on
- * {@link ValueSource} instances.
- *
- * @lucene.experimental
- */
-public class FunctionSecondPassGroupingCollector extends SecondPassGroupingCollector {
-
- private final ValueSource groupByVS;
- private final Map, ?> vsContext;
-
- private FunctionValues.ValueFiller filler;
- private MutableValue mval;
-
- /**
- * Constructs a {@link FunctionSecondPassGroupingCollector} instance.
- *
- * @param searchGroups The {@link SearchGroup} instances collected during the first phase.
- * @param groupSort The group sort
- * @param withinGroupSort The sort inside a group
- * @param maxDocsPerGroup The maximum number of documents to collect inside a group
- * @param getScores Whether to include the scores
- * @param getMaxScores Whether to include the maximum score
- * @param fillSortFields Whether to fill the sort values in {@link TopGroups#withinGroupSort}
- * @param groupByVS The {@link ValueSource} to group by
- * @param vsContext The value source context
- * @throws IOException IOException When I/O related errors occur
- */
- public FunctionSecondPassGroupingCollector(Collection> searchGroups, Sort groupSort, Sort withinGroupSort, int maxDocsPerGroup, boolean getScores, boolean getMaxScores, boolean fillSortFields, ValueSource groupByVS, Map, ?> vsContext) throws IOException {
- super(searchGroups, groupSort, withinGroupSort, maxDocsPerGroup, getScores, getMaxScores, fillSortFields);
- this.groupByVS = groupByVS;
- this.vsContext = vsContext;
- }
-
- @Override
- protected SearchGroupDocs retrieveGroup(int doc) throws IOException {
- filler.fillValue(doc);
- return groupMap.get(mval);
- }
-
- @Override
- protected void doSetNextReader(LeafReaderContext readerContext) throws IOException {
- super.doSetNextReader(readerContext);
- FunctionValues values = groupByVS.getValues(vsContext, readerContext);
- filler = values.getValueFiller();
- mval = filler.getValue();
- }
-
-}
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/package-info.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/package-info.java
index 824a98e31bf5..7e3745e61f28 100644
--- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/package-info.java
+++ b/lucene/grouping/src/java/org/apache/lucene/search/grouping/package-info.java
@@ -62,9 +62,9 @@
*
*
* The implementation is two-pass: the first pass ({@link
- * org.apache.lucene.search.grouping.term.TermFirstPassGroupingCollector})
+ * org.apache.lucene.search.grouping.FirstPassGroupingCollector})
* gathers the top groups, and the second pass ({@link
- * org.apache.lucene.search.grouping.term.TermSecondPassGroupingCollector})
+ * org.apache.lucene.search.grouping.SecondPassGroupingCollector})
* gathers documents within those groups. If the search is costly to
* run you may want to use the {@link
* org.apache.lucene.search.CachingCollector} class, which
@@ -73,18 +73,17 @@
* hold all hits. Results are returned as a {@link
* org.apache.lucene.search.grouping.TopGroups} instance.
*
- *
- * This module abstracts away what defines group and how it is collected. All grouping collectors
- * are abstract and have currently term based implementations. One can implement
- * collectors that for example group on multiple fields.
- *
+ * Groups are defined by {@link org.apache.lucene.search.grouping.GroupSelector}
+ * implementations:
+ *
+ * - {@link org.apache.lucene.search.grouping.TermGroupSelector} groups based on
+ * the value of a {@link org.apache.lucene.index.SortedDocValues} field
+ * - {@link org.apache.lucene.search.grouping.ValueSourceGroupSelector} groups based on
+ * the value of a {@link org.apache.lucene.queries.function.ValueSource}
+ *
*
* Known limitations:
*
- * - For the two-pass grouping search, the group field must be a
- * indexed as a {@link org.apache.lucene.document.SortedDocValuesField}).
- *
- Although Solr support grouping by function and this module has abstraction of what a group is, there are currently only
- * implementations for grouping based on terms.
*
- Sharding is not directly supported, though is not too
* difficult, if you can merge the top groups and top documents per
* group yourself.
@@ -174,14 +173,15 @@
* have to separately retrieve it (for example using stored
* fields,
FieldCache
, etc.).
*
- * Another collector is the TermAllGroupHeadsCollector
that can be used to retrieve all most relevant
+ *
Another collector is the AllGroupHeadsCollector
that can be used to retrieve all most relevant
* documents per group. Also known as group heads. This can be useful in situations when one wants to compute group
* based facets / statistics on the complete query result. The collector can be executed during the first or second
* phase. This collector can also be used with the GroupingSearch
convenience utility, but when if one only
* wants to compute the most relevant documents per group it is better to just use the collector as done here below.
*
*
- * AbstractAllGroupHeadsCollector c = TermAllGroupHeadsCollector.create(groupField, sortWithinGroup);
+ * TermGroupSelector grouper = new TermGroupSelector(groupField);
+ * AllGroupHeadsCollector c = AllGroupHeadsCollector.newCollector(grouper, sortWithinGroup);
* s.search(new TermQuery(new Term("content", searchTerm)), c);
* // Return all group heads as int array
* int[] groupHeadsArray = c.retrieveGroupHeads()
@@ -189,12 +189,6 @@
* int maxDoc = s.maxDoc();
* FixedBitSet groupHeadsBitSet = c.retrieveGroupHeads(maxDoc)
*
- *
- * For each of the above collector types there is also a variant that works with ValueSource
instead of
- * of fields. Concretely this means that these variants can work with functions. These variants are slower than
- * there term based counter parts. These implementations are located in the
- * org.apache.lucene.search.grouping.function
package, but can also be used with the
- * GroupingSearch
convenience utility
- *
+ *
*/
package org.apache.lucene.search.grouping;
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermAllGroupHeadsCollector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermAllGroupHeadsCollector.java
deleted file mode 100644
index 54e23993e586..000000000000
--- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermAllGroupHeadsCollector.java
+++ /dev/null
@@ -1,767 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.lucene.search.grouping.term;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.lucene.index.DocValues;
-import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.index.SortedDocValues;
-import org.apache.lucene.search.FieldComparator;
-import org.apache.lucene.search.LeafFieldComparator;
-import org.apache.lucene.search.Scorer;
-import org.apache.lucene.search.Sort;
-import org.apache.lucene.search.SortField;
-import org.apache.lucene.search.grouping.AllGroupHeadsCollector;
-import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.BytesRefBuilder;
-import org.apache.lucene.util.SentinelIntSet;
-
-/**
- * A base implementation of {@link AllGroupHeadsCollector} for retrieving the most relevant groups when grouping
- * on a string based group field. More specifically this all concrete implementations of this base implementation
- * use {@link SortedDocValues}.
- *
- * @lucene.experimental
- */
-public abstract class TermAllGroupHeadsCollector extends AllGroupHeadsCollector {
-
- private static final int DEFAULT_INITIAL_SIZE = 128;
-
- final String groupField;
-
- SortedDocValues groupIndex;
- LeafReaderContext readerContext;
-
- protected TermAllGroupHeadsCollector(String groupField, int numberOfSorts) {
- super(numberOfSorts);
- this.groupField = groupField;
- }
-
- /**
- * Creates an AbstractAllGroupHeadsCollector
instance based on the supplied arguments.
- * This factory method decides with implementation is best suited.
- *
- * Delegates to {@link #create(String, org.apache.lucene.search.Sort, int)} with an initialSize of 128.
- *
- * @param groupField The field to group by
- * @param sortWithinGroup The sort within each group
- * @return an AbstractAllGroupHeadsCollector
instance based on the supplied arguments
- */
- public static AllGroupHeadsCollector create(String groupField, Sort sortWithinGroup) {
- return create(groupField, sortWithinGroup, DEFAULT_INITIAL_SIZE);
- }
-
- /**
- * Creates an AbstractAllGroupHeadsCollector
instance based on the supplied arguments.
- * This factory method decides with implementation is best suited.
- *
- * @param groupField The field to group by
- * @param sortWithinGroup The sort within each group
- * @param initialSize The initial allocation size of the internal int set and group list which should roughly match
- * the total number of expected unique groups. Be aware that the heap usage is
- * 4 bytes * initialSize.
- * @return an AbstractAllGroupHeadsCollector
instance based on the supplied arguments
- */
- public static AllGroupHeadsCollector create(String groupField, Sort sortWithinGroup, int initialSize) {
- boolean sortAllScore = true;
- boolean sortAllFieldValue = true;
-
- for (SortField sortField : sortWithinGroup.getSort()) {
- if (sortField.getType() == SortField.Type.SCORE) {
- sortAllFieldValue = false;
- } else if (needGeneralImpl(sortField)) {
- return new GeneralAllGroupHeadsCollector(groupField, sortWithinGroup);
- } else {
- sortAllScore = false;
- }
- }
-
- if (sortAllScore) {
- return new ScoreAllGroupHeadsCollector(groupField, sortWithinGroup, initialSize);
- } else if (sortAllFieldValue) {
- return new OrdAllGroupHeadsCollector(groupField, sortWithinGroup, initialSize);
- } else {
- return new OrdScoreAllGroupHeadsCollector(groupField, sortWithinGroup, initialSize);
- }
- }
-
- // Returns when a sort field needs the general impl.
- private static boolean needGeneralImpl(SortField sortField) {
- SortField.Type sortType = sortField.getType();
- // Note (MvG): We can also make an optimized impl when sorting is SortField.DOC
- return sortType != SortField.Type.STRING_VAL && sortType != SortField.Type.STRING && sortType != SortField.Type.SCORE;
- }
-
- // A general impl that works for any group sort.
- static class GeneralAllGroupHeadsCollector extends TermAllGroupHeadsCollector {
-
- private final Sort sortWithinGroup;
- private final Map groups;
-
- Scorer scorer;
-
- GeneralAllGroupHeadsCollector(String groupField, Sort sortWithinGroup) {
- super(groupField, sortWithinGroup.getSort().length);
- this.sortWithinGroup = sortWithinGroup;
- groups = new HashMap<>();
-
- final SortField[] sortFields = sortWithinGroup.getSort();
- for (int i = 0; i < sortFields.length; i++) {
- reversed[i] = sortFields[i].getReverse() ? -1 : 1;
- }
- }
-
- protected int getOrdForDoc(int doc) throws IOException {
- int curDocID = groupIndex.docID();
- if (curDocID < doc) {
- curDocID = groupIndex.advance(doc);
- }
-
- if (curDocID == doc) {
- return groupIndex.ordValue();
- } else {
- return -1;
- }
- }
-
- @Override
- protected void retrieveGroupHeadAndAddIfNotExist(int doc) throws IOException {
- int ord = getOrdForDoc(doc);
-
- BytesRef groupValue;
- if (ord == -1) {
- groupValue = null;
- } else {
- groupValue = groupIndex.lookupOrd(ord);
- }
-
- GroupHead groupHead = groups.get(groupValue);
- if (groupHead == null) {
- groupValue = groupValue == null ? null : BytesRef.deepCopyOf(groupValue);
- groupHead = new GroupHead(groupValue, sortWithinGroup, doc);
- groups.put(groupValue, groupHead);
- temporalResult.stop = true;
- } else {
- temporalResult.stop = false;
- }
- temporalResult.groupHead = groupHead;
- }
-
- @Override
- protected Collection getCollectedGroupHeads() {
- return groups.values();
- }
-
- @Override
- protected void doSetNextReader(LeafReaderContext context) throws IOException {
- this.readerContext = context;
- groupIndex = DocValues.getSorted(context.reader(), groupField);
-
- for (GroupHead groupHead : groups.values()) {
- for (int i = 0; i < groupHead.comparators.length; i++) {
- groupHead.leafComparators[i] = groupHead.comparators[i].getLeafComparator(context);
- }
- }
- }
-
- @Override
- public boolean needsScores() {
- return sortWithinGroup.needsScores();
- }
-
- @Override
- public void setScorer(Scorer scorer) throws IOException {
- this.scorer = scorer;
- for (GroupHead groupHead : groups.values()) {
- for (LeafFieldComparator comparator : groupHead.leafComparators) {
- comparator.setScorer(scorer);
- }
- }
- }
-
- class GroupHead extends AllGroupHeadsCollector.GroupHead {
-
- @SuppressWarnings({"unchecked", "rawtypes"})
- final FieldComparator[] comparators;
-
- final LeafFieldComparator[] leafComparators;
-
- @SuppressWarnings({"unchecked", "rawtypes"})
- GroupHead(BytesRef groupValue, Sort sort, int doc) throws IOException {
- super(groupValue, doc + readerContext.docBase);
- final SortField[] sortFields = sort.getSort();
- comparators = new FieldComparator[sortFields.length];
- leafComparators = new LeafFieldComparator[sortFields.length];
- for (int i = 0; i < sortFields.length; i++) {
- comparators[i] = sortFields[i].getComparator(1, i);
- leafComparators[i] = comparators[i].getLeafComparator(readerContext);
- leafComparators[i].setScorer(scorer);
- leafComparators[i].copy(0, doc);
- leafComparators[i].setBottom(0);
- }
- }
-
- @Override
- public int compare(int compIDX, int doc) throws IOException {
- return leafComparators[compIDX].compareBottom(doc);
- }
-
- @Override
- public void updateDocHead(int doc) throws IOException {
- for (LeafFieldComparator comparator : leafComparators) {
- comparator.copy(0, doc);
- comparator.setBottom(0);
- }
- this.doc = doc + readerContext.docBase;
- }
- }
- }
-
-
- // AbstractAllGroupHeadsCollector optimized for ord fields and scores.
- static class OrdScoreAllGroupHeadsCollector extends TermAllGroupHeadsCollector {
-
- private final SentinelIntSet ordSet;
- private final List collectedGroups;
- final SortField[] fields;
-
- SortedDocValues[] sortsIndex;
- Scorer scorer;
- private GroupHead[] segmentGroupHeads;
-
- OrdScoreAllGroupHeadsCollector(String groupField, Sort sortWithinGroup, int initialSize) {
- super(groupField, sortWithinGroup.getSort().length);
- ordSet = new SentinelIntSet(initialSize, -2);
- collectedGroups = new ArrayList<>(initialSize);
-
- final SortField[] sortFields = sortWithinGroup.getSort();
- fields = new SortField[sortFields.length];
- sortsIndex = new SortedDocValues[sortFields.length];
- for (int i = 0; i < sortFields.length; i++) {
- reversed[i] = sortFields[i].getReverse() ? -1 : 1;
- fields[i] = sortFields[i];
- }
- }
-
- @Override
- protected Collection getCollectedGroupHeads() {
- return collectedGroups;
- }
-
- @Override
- public boolean needsScores() {
- return true;
- }
-
- @Override
- public void setScorer(Scorer scorer) throws IOException {
- this.scorer = scorer;
- }
-
- private int getOrdForDoc(int doc) throws IOException {
- int curDocID = groupIndex.docID();
- if (curDocID < doc) {
- curDocID = groupIndex.advance(doc);
- }
-
- if (curDocID == doc) {
- return groupIndex.ordValue();
- } else {
- return -1;
- }
- }
-
- @Override
- protected void retrieveGroupHeadAndAddIfNotExist(int doc) throws IOException {
- int key = getOrdForDoc(doc);
- GroupHead groupHead;
- if (!ordSet.exists(key)) {
- ordSet.put(key);
- final BytesRef term;
- if (key == -1) {
- term = null;
- } else {
- term = BytesRef.deepCopyOf(groupIndex.lookupOrd(key));
- }
- groupHead = new GroupHead(doc, term);
- collectedGroups.add(groupHead);
- segmentGroupHeads[key+1] = groupHead;
- temporalResult.stop = true;
- } else {
- temporalResult.stop = false;
- groupHead = segmentGroupHeads[key+1];
- }
- temporalResult.groupHead = groupHead;
- }
-
- @Override
- protected void doSetNextReader(LeafReaderContext context) throws IOException {
- this.readerContext = context;
- groupIndex = DocValues.getSorted(context.reader(), groupField);
- for (int i = 0; i < fields.length; i++) {
- if (fields[i].getType() == SortField.Type.SCORE) {
- continue;
- }
-
- sortsIndex[i] = DocValues.getSorted(context.reader(), fields[i].getField());
- }
-
- // Clear ordSet and fill it with previous encountered groups that can occur in the current segment.
- ordSet.clear();
- segmentGroupHeads = new GroupHead[groupIndex.getValueCount()+1];
- for (GroupHead collectedGroup : collectedGroups) {
- int ord;
- if (collectedGroup.groupValue == null) {
- ord = -1;
- } else {
- ord = groupIndex.lookupTerm(collectedGroup.groupValue);
- }
- if (collectedGroup.groupValue == null || ord >= 0) {
- ordSet.put(ord);
- segmentGroupHeads[ord+1] = collectedGroup;
-
- for (int i = 0; i < sortsIndex.length; i++) {
- if (fields[i].getType() == SortField.Type.SCORE) {
- continue;
- }
- int sortOrd;
- if (collectedGroup.sortValues[i] == null) {
- sortOrd = -1;
- } else {
- sortOrd = sortsIndex[i].lookupTerm(collectedGroup.sortValues[i].get());
- }
- collectedGroup.sortOrds[i] = sortOrd;
- }
- }
- }
- }
-
- void setDoc(int docID) throws IOException {
- for (int i = 0; i < sortsIndex.length; i++) {
- SortedDocValues values = sortsIndex[i];
- if (values != null && docID > values.docID()) {
- values.advance(docID);
- }
- }
- }
-
- class GroupHead extends AllGroupHeadsCollector.GroupHead {
-
- BytesRefBuilder[] sortValues;
- int[] sortOrds;
- float[] scores;
-
- GroupHead(int doc, BytesRef groupValue) throws IOException {
- super(groupValue, doc + readerContext.docBase);
- sortValues = new BytesRefBuilder[sortsIndex.length];
- sortOrds = new int[sortsIndex.length];
- scores = new float[sortsIndex.length];
- setDoc(doc);
- for (int i = 0; i < sortsIndex.length; i++) {
- if (fields[i].getType() == SortField.Type.SCORE) {
- scores[i] = scorer.score();
- } else {
- if (doc == sortsIndex[i].docID()) {
- sortOrds[i] = sortsIndex[i].ordValue();
- } else {
- sortOrds[i] = -1;
- }
- sortValues[i] = new BytesRefBuilder();
- if (sortOrds[i] != -1) {
- sortValues[i].copyBytes(sortsIndex[i].binaryValue());
- }
- }
- }
- }
-
- @Override
- public int compare(int compIDX, int doc) throws IOException {
- if (fields[compIDX].getType() == SortField.Type.SCORE) {
- float score = scorer.score();
- if (scores[compIDX] < score) {
- return 1;
- } else if (scores[compIDX] > score) {
- return -1;
- }
- return 0;
- } else {
- if (sortsIndex[compIDX].docID() < doc) {
- sortsIndex[compIDX].advance(doc);
- }
- if (sortOrds[compIDX] < 0) {
- // The current segment doesn't contain the sort value we encountered before. Therefore the ord is negative.
- final BytesRef term;
- if (sortsIndex[compIDX].docID() == doc) {
- term = sortsIndex[compIDX].binaryValue();
- } else {
- term = new BytesRef(BytesRef.EMPTY_BYTES);
- }
- return sortValues[compIDX].get().compareTo(term);
- } else {
- int ord;
- if (sortsIndex[compIDX].docID() == doc) {
- ord = sortsIndex[compIDX].ordValue();
- } else {
- ord = -1;
- }
- return sortOrds[compIDX] - ord;
- }
- }
- }
-
- @Override
- public void updateDocHead(int doc) throws IOException {
- setDoc(doc);
- for (int i = 0; i < sortsIndex.length; i++) {
- if (fields[i].getType() == SortField.Type.SCORE) {
- scores[i] = scorer.score();
- } else {
- if (sortsIndex[i].docID() == doc) {
- sortOrds[i] = sortsIndex[i].ordValue();
- sortValues[i].copyBytes(sortsIndex[i].binaryValue());
- } else {
- sortOrds[i] = -1;
- sortValues[i].clear();
- }
- }
- }
- this.doc = doc + readerContext.docBase;
- }
- }
- }
-
-
- // AbstractAllGroupHeadsCollector optimized for ord fields.
- static class OrdAllGroupHeadsCollector extends TermAllGroupHeadsCollector {
-
- private final SentinelIntSet ordSet;
- private final List collectedGroups;
- private final SortField[] fields;
-
- SortedDocValues[] sortsIndex;
- GroupHead[] segmentGroupHeads;
-
- OrdAllGroupHeadsCollector(String groupField, Sort sortWithinGroup, int initialSize) {
- super(groupField, sortWithinGroup.getSort().length);
- ordSet = new SentinelIntSet(initialSize, -2);
- collectedGroups = new ArrayList<>(initialSize);
-
- final SortField[] sortFields = sortWithinGroup.getSort();
- fields = new SortField[sortFields.length];
- sortsIndex = new SortedDocValues[sortFields.length];
- for (int i = 0; i < sortFields.length; i++) {
- reversed[i] = sortFields[i].getReverse() ? -1 : 1;
- fields[i] = sortFields[i];
- }
- }
-
- @Override
- protected Collection getCollectedGroupHeads() {
- return collectedGroups;
- }
-
- @Override
- public boolean needsScores() {
- return false;
- }
-
- @Override
- public void setScorer(Scorer scorer) throws IOException {
- }
-
- @Override
- protected void retrieveGroupHeadAndAddIfNotExist(int doc) throws IOException {
- if (doc > groupIndex.docID()) {
- groupIndex.advance(doc);
- }
-
- int key;
- if (doc == groupIndex.docID()) {
- key = groupIndex.ordValue();
- } else {
- key = -1;
- }
-
- GroupHead groupHead;
- if (!ordSet.exists(key)) {
- ordSet.put(key);
- final BytesRef term;
- if (key == -1) {
- term = null;
- } else {
- term = BytesRef.deepCopyOf(groupIndex.lookupOrd(key));
- }
- groupHead = new GroupHead(doc, term);
- collectedGroups.add(groupHead);
- segmentGroupHeads[key+1] = groupHead;
- temporalResult.stop = true;
- } else {
- temporalResult.stop = false;
- groupHead = segmentGroupHeads[key+1];
- }
- temporalResult.groupHead = groupHead;
- }
-
- @Override
- protected void doSetNextReader(LeafReaderContext context) throws IOException {
- this.readerContext = context;
- groupIndex = DocValues.getSorted(context.reader(), groupField);
- for (int i = 0; i < fields.length; i++) {
- sortsIndex[i] = DocValues.getSorted(context.reader(), fields[i].getField());
- }
-
- // Clear ordSet and fill it with previous encountered groups that can occur in the current segment.
- ordSet.clear();
- segmentGroupHeads = new GroupHead[groupIndex.getValueCount()+1];
- for (GroupHead collectedGroup : collectedGroups) {
- int groupOrd;
- if (collectedGroup.groupValue == null) {
- groupOrd = -1;
- } else {
- groupOrd = groupIndex.lookupTerm(collectedGroup.groupValue);
- }
- if (collectedGroup.groupValue == null || groupOrd >= 0) {
- ordSet.put(groupOrd);
- segmentGroupHeads[groupOrd+1] = collectedGroup;
-
- for (int i = 0; i < sortsIndex.length; i++) {
- int sortOrd;
- if (collectedGroup.sortOrds[i] == -1) {
- sortOrd = -1;
- } else {
- sortOrd = sortsIndex[i].lookupTerm(collectedGroup.sortValues[i].get());
- }
- collectedGroup.sortOrds[i] = sortOrd;
- }
- }
- }
- }
-
- void setDoc(int docID) throws IOException {
- for (int i = 0; i < sortsIndex.length; i++) {
- SortedDocValues values = sortsIndex[i];
- if (docID > values.docID()) {
- values.advance(docID);
- }
- }
- }
-
- class GroupHead extends AllGroupHeadsCollector.GroupHead {
-
- BytesRefBuilder[] sortValues;
- int[] sortOrds;
-
- GroupHead(int doc, BytesRef groupValue) throws IOException {
- super(groupValue, doc + readerContext.docBase);
- sortValues = new BytesRefBuilder[sortsIndex.length];
- sortOrds = new int[sortsIndex.length];
- setDoc(doc);
- for (int i = 0; i < sortsIndex.length; i++) {
- if (doc == sortsIndex[i].docID()) {
- sortOrds[i] = sortsIndex[i].ordValue();
- } else {
- sortOrds[i] = -1;
- }
- sortValues[i] = new BytesRefBuilder();
- if (sortOrds[i] != -1) {
- sortValues[i].copyBytes(sortsIndex[i].binaryValue());
- }
- }
- }
-
- @Override
- public int compare(int compIDX, int doc) throws IOException {
- if (sortsIndex[compIDX].docID() < doc) {
- sortsIndex[compIDX].advance(doc);
- }
- if (sortOrds[compIDX] < 0) {
- // The current segment doesn't contain the sort value we encountered before. Therefore the ord is negative.
- final BytesRef term;
- if (sortsIndex[compIDX].docID() == doc) {
- term = sortsIndex[compIDX].binaryValue();
- } else {
- term = new BytesRef(BytesRef.EMPTY_BYTES);
- }
- return sortValues[compIDX].get().compareTo(term);
- } else {
- int ord;
- if (sortsIndex[compIDX].docID() == doc) {
- ord = sortsIndex[compIDX].ordValue();
- } else {
- ord = -1;
- }
- return sortOrds[compIDX] - ord;
- }
- }
-
- @Override
- public void updateDocHead(int doc) throws IOException {
- setDoc(doc);
- for (int i = 0; i < sortsIndex.length; i++) {
- if (sortsIndex[i].docID() == doc) {
- sortOrds[i] = sortsIndex[i].ordValue();
- sortValues[i].copyBytes(sortsIndex[i].binaryValue());
- } else {
- sortOrds[i] = -1;
- sortValues[i].clear();
- }
- }
- this.doc = doc + readerContext.docBase;
- }
-
- }
-
- }
-
-
- // AbstractAllGroupHeadsCollector optimized for scores.
- static class ScoreAllGroupHeadsCollector extends TermAllGroupHeadsCollector {
-
- final SentinelIntSet ordSet;
- final List collectedGroups;
- final SortField[] fields;
-
- Scorer scorer;
- GroupHead[] segmentGroupHeads;
-
- ScoreAllGroupHeadsCollector(String groupField, Sort sortWithinGroup, int initialSize) {
- super(groupField, sortWithinGroup.getSort().length);
- ordSet = new SentinelIntSet(initialSize, -2);
- collectedGroups = new ArrayList<>(initialSize);
-
- final SortField[] sortFields = sortWithinGroup.getSort();
- fields = new SortField[sortFields.length];
- for (int i = 0; i < sortFields.length; i++) {
- reversed[i] = sortFields[i].getReverse() ? -1 : 1;
- fields[i] = sortFields[i];
- }
- }
-
- @Override
- protected Collection getCollectedGroupHeads() {
- return collectedGroups;
- }
-
- @Override
- public boolean needsScores() {
- return true;
- }
-
- @Override
- public void setScorer(Scorer scorer) throws IOException {
- this.scorer = scorer;
- }
-
- @Override
- protected void retrieveGroupHeadAndAddIfNotExist(int doc) throws IOException {
- if (doc > groupIndex.docID()) {
- groupIndex.advance(doc);
- }
-
- int key;
- if (doc == groupIndex.docID()) {
- key = groupIndex.ordValue();
- } else {
- key = -1;
- }
-
- GroupHead groupHead;
- if (!ordSet.exists(key)) {
- ordSet.put(key);
- final BytesRef term;
- if (key == -1) {
- term = null;
- } else {
- term = BytesRef.deepCopyOf(groupIndex.lookupOrd(key));
- }
- groupHead = new GroupHead(doc, term);
- collectedGroups.add(groupHead);
- segmentGroupHeads[key+1] = groupHead;
- temporalResult.stop = true;
- } else {
- temporalResult.stop = false;
- groupHead = segmentGroupHeads[key+1];
- }
- temporalResult.groupHead = groupHead;
- }
-
- @Override
- protected void doSetNextReader(LeafReaderContext context) throws IOException {
- this.readerContext = context;
- groupIndex = DocValues.getSorted(context.reader(), groupField);
-
- // Clear ordSet and fill it with previous encountered groups that can occur in the current segment.
- ordSet.clear();
- segmentGroupHeads = new GroupHead[groupIndex.getValueCount()+1];
- for (GroupHead collectedGroup : collectedGroups) {
- int ord;
- if (collectedGroup.groupValue == null) {
- ord = -1;
- } else {
- ord = groupIndex.lookupTerm(collectedGroup.groupValue);
- }
- if (collectedGroup.groupValue == null || ord >= 0) {
- ordSet.put(ord);
- segmentGroupHeads[ord+1] = collectedGroup;
- }
- }
- }
-
- class GroupHead extends AllGroupHeadsCollector.GroupHead {
-
- float[] scores;
-
- GroupHead(int doc, BytesRef groupValue) throws IOException {
- super(groupValue, doc + readerContext.docBase);
- scores = new float[fields.length];
- float score = scorer.score();
- for (int i = 0; i < scores.length; i++) {
- scores[i] = score;
- }
- }
-
- @Override
- public int compare(int compIDX, int doc) throws IOException {
- float score = scorer.score();
- if (scores[compIDX] < score) {
- return 1;
- } else if (scores[compIDX] > score) {
- return -1;
- }
- return 0;
- }
-
- @Override
- public void updateDocHead(int doc) throws IOException {
- float score = scorer.score();
- for (int i = 0; i < scores.length; i++) {
- scores[i] = score;
- }
- this.doc = doc + readerContext.docBase;
- }
-
- }
-
- }
-
-}
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermAllGroupsCollector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermAllGroupsCollector.java
deleted file mode 100644
index 125555a0940b..000000000000
--- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermAllGroupsCollector.java
+++ /dev/null
@@ -1,128 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.lucene.search.grouping.term;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.List;
-
-import org.apache.lucene.index.DocValues;
-import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.index.SortedDocValues;
-import org.apache.lucene.search.grouping.AllGroupsCollector;
-import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.SentinelIntSet;
-
-/**
- * A collector that collects all groups that match the
- * query. Only the group value is collected, and the order
- * is undefined. This collector does not determine
- * the most relevant document of a group.
- *
- * Implementation detail: an int hash set (SentinelIntSet)
- * is used to detect if a group is already added to the
- * total count. For each segment the int set is cleared and filled
- * with previous counted groups that occur in the new
- * segment.
- *
- * @lucene.experimental
- */
-public class TermAllGroupsCollector extends AllGroupsCollector {
-
- private static final int DEFAULT_INITIAL_SIZE = 128;
-
- private final String groupField;
- private final SentinelIntSet ordSet;
- private final List groups;
-
- private SortedDocValues index;
-
- /**
- * Expert: Constructs a {@link AllGroupsCollector}
- *
- * @param groupField The field to group by
- * @param initialSize The initial allocation size of the
- * internal int set and group list
- * which should roughly match the total
- * number of expected unique groups. Be aware that the
- * heap usage is 4 bytes * initialSize.
- */
- public TermAllGroupsCollector(String groupField, int initialSize) {
- ordSet = new SentinelIntSet(initialSize, -2);
- groups = new ArrayList<>(initialSize);
- this.groupField = groupField;
- }
-
- /**
- * Constructs a {@link AllGroupsCollector}. This sets the
- * initial allocation size for the internal int set and group
- * list to 128.
- *
- * @param groupField The field to group by
- */
- public TermAllGroupsCollector(String groupField) {
- this(groupField, DEFAULT_INITIAL_SIZE);
- }
-
- @Override
- public void collect(int doc) throws IOException {
- if (doc > index.docID()) {
- index.advance(doc);
- }
- int key;
- if (doc == index.docID()) {
- key = index.ordValue();
- } else {
- key = -1;
- }
- if (!ordSet.exists(key)) {
- ordSet.put(key);
- final BytesRef term;
- if (key == -1) {
- term = null;
- } else {
- term = BytesRef.deepCopyOf(index.lookupOrd(key));
- }
- groups.add(term);
- }
- }
-
- @Override
- public Collection getGroups() {
- return groups;
- }
-
- @Override
- protected void doSetNextReader(LeafReaderContext context) throws IOException {
- index = DocValues.getSorted(context.reader(), groupField);
-
- // Clear ordSet and fill it with previous encountered groups that can occur in the current segment.
- ordSet.clear();
- for (BytesRef countedGroup : groups) {
- if (countedGroup == null) {
- ordSet.put(-1);
- } else {
- int ord = index.lookupTerm(countedGroup);
- if (ord >= 0) {
- ordSet.put(ord);
- }
- }
- }
- }
-
-}
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermDistinctValuesCollector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermDistinctValuesCollector.java
deleted file mode 100644
index e5356a311b46..000000000000
--- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermDistinctValuesCollector.java
+++ /dev/null
@@ -1,162 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.lucene.search.grouping.term;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.List;
-
-import org.apache.lucene.index.DocValues;
-import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.index.SortedDocValues;
-import org.apache.lucene.search.grouping.DistinctValuesCollector;
-import org.apache.lucene.search.grouping.SearchGroup;
-import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.SentinelIntSet;
-
-/**
- * A term based implementation of {@link DistinctValuesCollector} that relies
- * on {@link SortedDocValues} to count the distinct values per group.
- *
- * @lucene.experimental
- */
-public class TermDistinctValuesCollector extends DistinctValuesCollector {
-
- private final String groupField;
- private final String countField;
- private final List groups;
- private final SentinelIntSet ordSet;
- private final TermGroupCount groupCounts[];
-
- private SortedDocValues groupFieldTermIndex;
- private SortedDocValues countFieldTermIndex;
-
- /**
- * Constructs {@link TermDistinctValuesCollector} instance.
- *
- * @param groupField The field to group by
- * @param countField The field to count distinct values for
- * @param groups The top N groups, collected during the first phase search
- */
- public TermDistinctValuesCollector(String groupField, String countField, Collection> groups) {
- this.groupField = groupField;
- this.countField = countField;
- this.groups = new ArrayList<>(groups.size());
- for (SearchGroup group : groups) {
- this.groups.add(new TermGroupCount(group.groupValue));
- }
- ordSet = new SentinelIntSet(groups.size(), -2);
- groupCounts = new TermGroupCount[ordSet.keys.length];
- }
-
- @Override
- public void collect(int doc) throws IOException {
- if (doc > groupFieldTermIndex.docID()) {
- groupFieldTermIndex.advance(doc);
- }
- int ord;
- if (doc == groupFieldTermIndex.docID()) {
- ord = groupFieldTermIndex.ordValue();
- } else {
- ord = -1;
- }
- int slot = ordSet.find(ord);
- if (slot < 0) {
- return;
- }
-
- TermGroupCount gc = groupCounts[slot];
- if (doc > countFieldTermIndex.docID()) {
- countFieldTermIndex.advance(doc);
- }
-
- int countOrd;
- if (doc == countFieldTermIndex.docID()) {
- countOrd = countFieldTermIndex.ordValue();
- } else {
- countOrd = -1;
- }
-
- if (doesNotContainOrd(countOrd, gc.ords)) {
- if (countOrd == -1) {
- gc.uniqueValues.add(null);
- } else {
- BytesRef term = BytesRef.deepCopyOf(countFieldTermIndex.lookupOrd(countOrd));
- gc.uniqueValues.add(term);
- }
-
- gc.ords = Arrays.copyOf(gc.ords, gc.ords.length + 1);
- gc.ords[gc.ords.length - 1] = countOrd;
- if (gc.ords.length > 1) {
- Arrays.sort(gc.ords);
- }
- }
- }
-
- private boolean doesNotContainOrd(int ord, int[] ords) {
- if (ords.length == 0) {
- return true;
- } else if (ords.length == 1) {
- return ord != ords[0];
- }
- return Arrays.binarySearch(ords, ord) < 0;
- }
-
- @Override
- public List> getGroups() {
- return new ArrayList<>(groups);
- }
-
- @Override
- protected void doSetNextReader(LeafReaderContext context) throws IOException {
- groupFieldTermIndex = DocValues.getSorted(context.reader(), groupField);
- countFieldTermIndex = DocValues.getSorted(context.reader(), countField);
- ordSet.clear();
- for (TermGroupCount group : groups) {
- int groupOrd = group.groupValue == null ? -1 : groupFieldTermIndex.lookupTerm(group.groupValue);
- if (group.groupValue != null && groupOrd < 0) {
- continue;
- }
-
- groupCounts[ordSet.put(groupOrd)] = group;
- group.ords = new int[group.uniqueValues.size()];
- Arrays.fill(group.ords, -2);
- int i = 0;
- for (BytesRef value : group.uniqueValues) {
- int countOrd = value == null ? -1 : countFieldTermIndex.lookupTerm(value);
- if (value == null || countOrd >= 0) {
- group.ords[i++] = countOrd;
- }
- }
- }
- }
-
- /** Holds distinct values for a single group.
- *
- * @lucene.experimental */
- public static class TermGroupCount extends DistinctValuesCollector.GroupCount {
-
- int[] ords;
-
- TermGroupCount(BytesRef groupValue) {
- super(groupValue);
- }
- }
-
-}
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermFirstPassGroupingCollector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermFirstPassGroupingCollector.java
deleted file mode 100644
index 3c35fa8b22c8..000000000000
--- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermFirstPassGroupingCollector.java
+++ /dev/null
@@ -1,96 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.lucene.search.grouping.term;
-
-import java.io.IOException;
-
-import org.apache.lucene.index.DocValues;
-import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.index.SortedDocValues;
-import org.apache.lucene.search.Sort;
-import org.apache.lucene.search.grouping.FirstPassGroupingCollector;
-import org.apache.lucene.util.ArrayUtil;
-import org.apache.lucene.util.BytesRef;
-
-/**
- * Concrete implementation of {@link FirstPassGroupingCollector} that groups based on
- * field values and more specifically uses {@link SortedDocValues}
- * to collect groups.
- *
- * @lucene.experimental
- */
-public class TermFirstPassGroupingCollector extends FirstPassGroupingCollector {
-
- private SortedDocValues index;
-
- private String groupField;
-
- /**
- * Create the first pass collector.
- *
- * @param groupField The field used to group
- * documents. This field must be single-valued and
- * indexed (DocValues is used to access its value
- * per-document).
- * @param groupSort The {@link Sort} used to sort the
- * groups. The top sorted document within each group
- * according to groupSort, determines how that group
- * sorts against other groups. This must be non-null,
- * ie, if you want to groupSort by relevance use
- * Sort.RELEVANCE.
- * @param topNGroups How many top groups to keep.
- * @throws IOException When I/O related errors occur
- */
- public TermFirstPassGroupingCollector(String groupField, Sort groupSort, int topNGroups) throws IOException {
- super(groupSort, topNGroups);
- this.groupField = groupField;
- }
-
- @Override
- protected BytesRef getDocGroupValue(int doc) throws IOException {
- if (doc > index.docID()) {
- index.advance(doc);
- }
- if (doc == index.docID()) {
- return index.binaryValue();
- } else {
- return null;
- }
- }
-
- @Override
- protected BytesRef copyDocGroupValue(BytesRef groupValue, BytesRef reuse) {
- if (groupValue == null) {
- return null;
- } else if (reuse != null) {
- reuse.bytes = ArrayUtil.grow(reuse.bytes, groupValue.length);
- reuse.offset = 0;
- reuse.length = groupValue.length;
- System.arraycopy(groupValue.bytes, groupValue.offset, reuse.bytes, 0, groupValue.length);
- return reuse;
- } else {
- return BytesRef.deepCopyOf(groupValue);
- }
- }
-
- @Override
- protected void doSetNextReader(LeafReaderContext readerContext) throws IOException {
- super.doSetNextReader(readerContext);
- index = DocValues.getSorted(readerContext.reader(), groupField);
- }
-
-}
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermGrouper.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermGrouper.java
deleted file mode 100644
index 246ee7853aa1..000000000000
--- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermGrouper.java
+++ /dev/null
@@ -1,81 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.lucene.search.grouping.term;
-
-import java.io.IOException;
-import java.util.Collection;
-
-import org.apache.lucene.search.Sort;
-import org.apache.lucene.search.grouping.AllGroupHeadsCollector;
-import org.apache.lucene.search.grouping.AllGroupsCollector;
-import org.apache.lucene.search.grouping.FirstPassGroupingCollector;
-import org.apache.lucene.search.grouping.SecondPassGroupingCollector;
-import org.apache.lucene.search.grouping.Grouper;
-import org.apache.lucene.search.grouping.SearchGroup;
-import org.apache.lucene.util.BytesRef;
-
-/**
- * Collector factory for grouping by term
- */
-public class TermGrouper extends Grouper {
-
- private final String field;
- private final int initialSize;
-
- /**
- * Create a new TermGrouper
- * @param field the field to group on
- */
- public TermGrouper(String field) {
- this(field, 128);
- }
-
- /**
- * Create a new TermGrouper
- * @param field the field to group on
- * @param initialSize the initial size of various internal datastructures
- */
- public TermGrouper(String field, int initialSize) {
- this.field = field;
- this.initialSize = initialSize;
- }
-
- @Override
- public FirstPassGroupingCollector getFirstPassCollector(Sort sort, int count) throws IOException {
- return new TermFirstPassGroupingCollector(field, sort, count);
- }
-
- @Override
- public AllGroupHeadsCollector getGroupHeadsCollector(Sort sort) {
- return TermAllGroupHeadsCollector.create(field, sort, initialSize);
- }
-
- @Override
- public AllGroupsCollector getAllGroupsCollector() {
- return new TermAllGroupsCollector(field, initialSize);
- }
-
- @Override
- public SecondPassGroupingCollector getSecondPassCollector(
- Collection> groups, Sort groupSort, Sort withinGroupSort,
- int maxDocsPerGroup, boolean getScores, boolean getMaxScores, boolean fillSortFields) throws IOException {
- return new TermSecondPassGroupingCollector(field, groups, groupSort, withinGroupSort, maxDocsPerGroup, getScores, getMaxScores, fillSortFields);
- }
-
-
-}
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermSecondPassGroupingCollector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermSecondPassGroupingCollector.java
deleted file mode 100644
index 75d221024ba4..000000000000
--- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermSecondPassGroupingCollector.java
+++ /dev/null
@@ -1,91 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.lucene.search.grouping.term;
-
-import java.io.IOException;
-import java.util.Collection;
-
-import org.apache.lucene.index.DocValues;
-import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.index.SortedDocValues;
-import org.apache.lucene.search.Sort;
-import org.apache.lucene.search.grouping.SecondPassGroupingCollector;
-import org.apache.lucene.search.grouping.SearchGroup;
-import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.SentinelIntSet;
-
-/**
- * Concrete implementation of {@link SecondPassGroupingCollector} that groups based on
- * field values and more specifically uses {@link SortedDocValues}
- * to collect grouped docs.
- *
- * @lucene.experimental
- */
-public class TermSecondPassGroupingCollector extends SecondPassGroupingCollector {
-
- private final String groupField;
- private final SentinelIntSet ordSet;
-
- private SortedDocValues index;
-
- @SuppressWarnings({"unchecked", "rawtypes"})
- public TermSecondPassGroupingCollector(String groupField, Collection> groups, Sort groupSort, Sort withinGroupSort,
- int maxDocsPerGroup, boolean getScores, boolean getMaxScores, boolean fillSortFields)
- throws IOException {
- super(groups, groupSort, withinGroupSort, maxDocsPerGroup, getScores, getMaxScores, fillSortFields);
- this.groupField = groupField;
- this.ordSet = new SentinelIntSet(groupMap.size(), -2);
- super.groupDocs = (SearchGroupDocs[]) new SearchGroupDocs[ordSet.keys.length];
- }
-
- @Override
- protected void doSetNextReader(LeafReaderContext readerContext) throws IOException {
- super.doSetNextReader(readerContext);
- index = DocValues.getSorted(readerContext.reader(), groupField);
-
- // Rebuild ordSet
- ordSet.clear();
- for (SearchGroupDocs group : groupMap.values()) {
-// System.out.println(" group=" + (group.groupValue == null ? "null" : group.groupValue.utf8ToString()));
- int ord = group.groupValue == null ? -1 : index.lookupTerm(group.groupValue);
- if (group.groupValue == null || ord >= 0) {
- groupDocs[ordSet.put(ord)] = group;
- }
- }
- }
-
- @Override
- protected SearchGroupDocs